From ae0741b8d6f203d881de12f9f5d8f0d326f5093c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 21 Mar 2026 07:43:26 -0600 Subject: [PATCH 01/46] feat: add standalone shuffle benchmark binary for profiling Add a `shuffle_bench` binary that benchmarks shuffle write and read performance independently from Spark, making it easy to profile with tools like `cargo flamegraph`, `perf`, or `instruments`. Supports reading Parquet files (e.g. TPC-H/TPC-DS) or generating synthetic data with configurable schema. Covers different scenarios including compression codecs, partition counts, partitioning schemes, and memory-constrained spilling. --- native/Cargo.lock | 88 +++- native/core/Cargo.toml | 5 + native/core/src/bin/shuffle_bench.rs | 725 +++++++++++++++++++++++++++ 3 files changed, 816 insertions(+), 2 deletions(-) create mode 100644 native/core/src/bin/shuffle_bench.rs diff --git a/native/Cargo.lock b/native/Cargo.lock index 5f99c614b3..f43b41dd9a 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -96,12 +96,56 @@ version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" +[[package]] +name = "anstream" +version = "0.6.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43d5b281e737544384e969a5ccad3f1cdd24b48086a0fc1b2a5262a26b8f4f4a" +dependencies = [ + "anstyle", + "anstyle-parse", + "anstyle-query", + "anstyle-wincon", + "colorchoice", + "is_terminal_polyfill", + "utf8parse", +] + [[package]] name = "anstyle" version = "1.0.13" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5192cca8006f1fd4f7237516f40fa183bb07f8fbdfedaa0036de5ea9b0b45e78" +[[package]] +name = "anstyle-parse" +version = "0.2.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4e7644824f0aa2c7b9384579234ef10eb7efb6a0deb83f9630a49594dd9c15c2" +dependencies = [ + "utf8parse", +] + +[[package]] +name = "anstyle-query" +version = "1.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40c48f72fd53cd289104fc64099abca73db4166ad86ea0b4341abe65af83dadc" +dependencies = [ + "windows-sys 0.60.2", +] + +[[package]] +name = "anstyle-wincon" +version = "3.0.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "291e6a250ff86cd4a820112fb8898808a366d8f9f58ce16d1f538353ad55747d" +dependencies = [ + "anstyle", + "once_cell_polyfill", + "windows-sys 0.60.2", +] + [[package]] name = "anyhow" version = "1.0.102" @@ -1331,6 +1375,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2797f34da339ce31042b27d23607e051786132987f595b02ba4f6a6dffb7030a" dependencies = [ "clap_builder", + "clap_derive", ] [[package]] @@ -1339,8 +1384,22 @@ version = "4.5.60" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "24a241312cea5059b13574bb9b3861cabf758b879c15190b37b6d6fd63ab6876" dependencies = [ + "anstream", "anstyle", "clap_lex", + "strsim", +] + +[[package]] +name = "clap_derive" +version = "4.5.55" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a92793da1a46a5f2a02a6f4c46c6496b28c43638adea8306fcb0caa1634f24e5" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 2.0.117", ] [[package]] @@ -1358,6 +1417,12 @@ dependencies = [ "cc", ] +[[package]] +name = "colorchoice" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d07550c9036bf2ae0c684c4297d503f838287c83c53686d05370d0e139ae570" + [[package]] name = "combine" version = "4.6.7" @@ -1834,6 +1899,7 @@ dependencies = [ "aws-config", "aws-credential-types", "bytes", + "clap", "crc32fast", "criterion", "datafusion", @@ -1885,7 +1951,7 @@ dependencies = [ [[package]] name = "datafusion-comet-common" -version = "0.14.0" +version = "0.15.0" dependencies = [ "arrow", "datafusion", @@ -1911,7 +1977,7 @@ dependencies = [ [[package]] name = "datafusion-comet-jni-bridge" -version = "0.14.0" +version = "0.15.0" dependencies = [ "arrow", "assertables", @@ -3609,6 +3675,12 @@ dependencies = [ "windows-sys 0.61.2", ] +[[package]] +name = "is_terminal_polyfill" +version = "1.70.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a6cb138bb79a146c1bd460005623e142ef0181e3d0219cb493e02f7d08a35695" + [[package]] name = "itertools" version = "0.13.0" @@ -4289,6 +4361,12 @@ version = "1.21.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9f7c3e4beb33f85d45ae3e3a1792185706c8e16d043238c593331cc7cd313b50" +[[package]] +name = "once_cell_polyfill" +version = "1.70.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "384b8ab6d37215f3c5301a95a4accb5d64aa607f1fcb26a11b5303878451b4fe" + [[package]] name = "oorandom" version = "11.1.5" @@ -6339,6 +6417,12 @@ version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b6c140620e7ffbb22c2dee59cafe6084a59b5ffc27a8859a5f0d494b5d52b6be" +[[package]] +name = "utf8parse" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" + [[package]] name = "uuid" version = "1.22.0" diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 3f305a631d..3df9e55719 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -72,6 +72,7 @@ url = { workspace = true } aws-config = { workspace = true } aws-credential-types = { workspace = true } parking_lot = "0.12.5" +clap = { version = "4", features = ["derive"] } datafusion-comet-objectstore-hdfs = { path = "../hdfs", optional = true, default-features = false, features = ["hdfs"] } reqwest = { version = "0.12", default-features = false, features = ["rustls-tls-native-roots", "http2"] } object_store_opendal = {version = "0.55.0", optional = true} @@ -113,6 +114,10 @@ name = "comet" # "rlib" is for benchmarking with criterion. crate-type = ["cdylib", "rlib"] +[[bin]] +name = "shuffle_bench" +path = "src/bin/shuffle_bench.rs" + [[bench]] name = "parquet_read" harness = false diff --git a/native/core/src/bin/shuffle_bench.rs b/native/core/src/bin/shuffle_bench.rs new file mode 100644 index 0000000000..c1498161f7 --- /dev/null +++ b/native/core/src/bin/shuffle_bench.rs @@ -0,0 +1,725 @@ +// 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. + +//! Standalone shuffle benchmark tool for profiling Comet shuffle write and read +//! outside of Spark. +//! +//! # Usage +//! +//! Read from Parquet files (e.g. TPC-H lineitem): +//! ```sh +//! cargo run --release --bin shuffle_bench -- \ +//! --input /data/tpch-sf100/lineitem/ \ +//! --partitions 200 \ +//! --codec zstd --zstd-level 1 \ +//! --hash-columns 0,3 \ +//! --read-back +//! ``` +//! +//! Generate synthetic data: +//! ```sh +//! cargo run --release --bin shuffle_bench -- \ +//! --generate --gen-rows 10000000 --gen-string-cols 4 --gen-int-cols 4 \ +//! --gen-decimal-cols 2 --gen-avg-string-len 32 \ +//! --partitions 200 --codec lz4 --read-back +//! ``` +//! +//! Profile with flamegraph: +//! ```sh +//! cargo flamegraph --release --bin shuffle_bench -- \ +//! --input /data/tpch-sf100/lineitem/ \ +//! --partitions 200 --codec zstd --zstd-level 1 +//! ``` + +use arrow::array::builder::{Date32Builder, Decimal128Builder, Int64Builder, StringBuilder}; +use arrow::array::RecordBatch; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use clap::Parser; +use comet::execution::shuffle::{ + read_ipc_compressed, CometPartitioning, CompressionCodec, ShuffleWriterExec, +}; +use datafusion::datasource::memory::MemorySourceConfig; +use datafusion::datasource::source::DataSourceExec; +use datafusion::execution::config::SessionConfig; +use datafusion::execution::runtime_env::RuntimeEnvBuilder; +use datafusion::physical_expr::expressions::Column; +use datafusion::physical_plan::common::collect; +use datafusion::physical_plan::ExecutionPlan; +use datafusion::prelude::SessionContext; +use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; +use rand::RngExt; +use std::fs; +use std::path::PathBuf; +use std::sync::Arc; +use std::time::Instant; + +#[derive(Parser, Debug)] +#[command( + name = "shuffle_bench", + about = "Standalone benchmark for Comet shuffle write and read performance" +)] +struct Args { + /// Path to input Parquet file or directory of Parquet files + #[arg(long)] + input: Option, + + /// Generate synthetic data instead of reading from Parquet + #[arg(long, default_value_t = false)] + generate: bool, + + /// Number of rows to generate (requires --generate) + #[arg(long, default_value_t = 1_000_000)] + gen_rows: usize, + + /// Number of Int64 columns to generate + #[arg(long, default_value_t = 4)] + gen_int_cols: usize, + + /// Number of Utf8 string columns to generate + #[arg(long, default_value_t = 2)] + gen_string_cols: usize, + + /// Number of Decimal128 columns to generate + #[arg(long, default_value_t = 2)] + gen_decimal_cols: usize, + + /// Number of Date32 columns to generate + #[arg(long, default_value_t = 1)] + gen_date_cols: usize, + + /// Average string length for generated string columns + #[arg(long, default_value_t = 24)] + gen_avg_string_len: usize, + + /// Batch size for reading Parquet or generating data + #[arg(long, default_value_t = 8192)] + batch_size: usize, + + /// Number of output shuffle partitions + #[arg(long, default_value_t = 200)] + partitions: usize, + + /// Partitioning scheme: hash, single, round-robin + #[arg(long, default_value = "hash")] + partitioning: String, + + /// Column indices to hash on (comma-separated, e.g. "0,3") + #[arg(long, default_value = "0")] + hash_columns: String, + + /// Compression codec: none, lz4, zstd, snappy + #[arg(long, default_value = "zstd")] + codec: String, + + /// Zstd compression level (1-22) + #[arg(long, default_value_t = 1)] + zstd_level: i32, + + /// Memory limit in bytes (triggers spilling when exceeded) + #[arg(long)] + memory_limit: Option, + + /// Also benchmark reading back the shuffle output + #[arg(long, default_value_t = false)] + read_back: bool, + + /// Number of iterations to run + #[arg(long, default_value_t = 1)] + iterations: usize, + + /// Number of warmup iterations before timing + #[arg(long, default_value_t = 0)] + warmup: usize, + + /// Output directory for shuffle data/index files + #[arg(long, default_value = "/tmp/comet_shuffle_bench")] + output_dir: PathBuf, + + /// Write buffer size in bytes + #[arg(long, default_value_t = 1048576)] + write_buffer_size: usize, +} + +fn main() { + let args = Args::parse(); + + // Validate args + if args.input.is_none() && !args.generate { + eprintln!("Error: must specify either --input or --generate"); + std::process::exit(1); + } + + // Create output directory + fs::create_dir_all(&args.output_dir).expect("Failed to create output directory"); + + let data_file = args.output_dir.join("data.out"); + let index_file = args.output_dir.join("index.out"); + + // Load data + let load_start = Instant::now(); + let batches = if let Some(ref input_path) = args.input { + load_parquet(input_path, args.batch_size) + } else { + generate_data(&args) + }; + let load_elapsed = load_start.elapsed(); + + let schema = batches[0].schema(); + let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum(); + let total_bytes: usize = batches.iter().map(|b| b.get_array_memory_size()).sum(); + + println!("=== Shuffle Benchmark ==="); + println!( + "Data source: {}", + if args.input.is_some() { + "parquet" + } else { + "generated" + } + ); + println!( + "Schema: {} columns ({} fields)", + schema.fields().len(), + describe_schema(&schema) + ); + println!("Total rows: {}", format_number(total_rows)); + println!("Total size: {}", format_bytes(total_bytes)); + println!("Batches: {}", batches.len()); + println!( + "Rows/batch: ~{}", + if batches.is_empty() { + 0 + } else { + total_rows / batches.len() + } + ); + println!("Load time: {:.3}s", load_elapsed.as_secs_f64()); + println!(); + + let codec = parse_codec(&args.codec, args.zstd_level); + let hash_col_indices = parse_hash_columns(&args.hash_columns); + + println!("Partitioning: {}", args.partitioning); + println!("Partitions: {}", args.partitions); + println!("Codec: {:?}", codec); + println!("Hash columns: {:?}", hash_col_indices); + if let Some(mem_limit) = args.memory_limit { + println!("Memory limit: {}", format_bytes(mem_limit)); + } + println!( + "Iterations: {} (warmup: {})", + args.iterations, args.warmup + ); + println!(); + + // Run warmup + timed iterations + let total_iters = args.warmup + args.iterations; + let mut write_times = Vec::with_capacity(args.iterations); + let mut read_times = Vec::with_capacity(args.iterations); + let mut data_file_sizes = Vec::with_capacity(args.iterations); + + for i in 0..total_iters { + let is_warmup = i < args.warmup; + let label = if is_warmup { + format!("warmup {}/{}", i + 1, args.warmup) + } else { + format!("iter {}/{}", i - args.warmup + 1, args.iterations) + }; + + // Write phase + let write_elapsed = run_shuffle_write( + &batches, + &schema, + &codec, + &hash_col_indices, + &args, + data_file.to_str().unwrap(), + index_file.to_str().unwrap(), + ); + let data_size = fs::metadata(&data_file).map(|m| m.len()).unwrap_or(0); + + if !is_warmup { + write_times.push(write_elapsed); + data_file_sizes.push(data_size); + } + + print!(" [{label}] write: {:.3}s", write_elapsed); + print!(" output: {}", format_bytes(data_size as usize)); + + // Read phase + if args.read_back { + let read_elapsed = run_shuffle_read( + data_file.to_str().unwrap(), + index_file.to_str().unwrap(), + args.partitions, + ); + if !is_warmup { + read_times.push(read_elapsed); + } + print!(" read: {:.3}s", read_elapsed); + } + println!(); + } + + // Print summary + if args.iterations > 0 { + println!(); + println!("=== Results ==="); + + let avg_write = write_times.iter().sum::() / write_times.len() as f64; + let avg_data_size = data_file_sizes.iter().sum::() / data_file_sizes.len() as u64; + let write_throughput_rows = total_rows as f64 / avg_write; + let write_throughput_bytes = total_bytes as f64 / avg_write; + let compression_ratio = if avg_data_size > 0 { + total_bytes as f64 / avg_data_size as f64 + } else { + 0.0 + }; + + println!("Write:"); + println!(" avg time: {:.3}s", avg_write); + if write_times.len() > 1 { + let min = write_times.iter().cloned().fold(f64::INFINITY, f64::min); + let max = write_times + .iter() + .cloned() + .fold(f64::NEG_INFINITY, f64::max); + println!(" min/max: {:.3}s / {:.3}s", min, max); + } + println!( + " throughput: {}/s ({} rows/s)", + format_bytes(write_throughput_bytes as usize), + format_number(write_throughput_rows as usize) + ); + println!( + " output size: {}", + format_bytes(avg_data_size as usize) + ); + println!(" compression: {:.2}x", compression_ratio); + + if !read_times.is_empty() { + let avg_read = read_times.iter().sum::() / read_times.len() as f64; + let read_throughput_bytes = avg_data_size as f64 / avg_read; + + println!("Read:"); + println!(" avg time: {:.3}s", avg_read); + if read_times.len() > 1 { + let min = read_times.iter().cloned().fold(f64::INFINITY, f64::min); + let max = read_times.iter().cloned().fold(f64::NEG_INFINITY, f64::max); + println!(" min/max: {:.3}s / {:.3}s", min, max); + } + println!( + " throughput: {}/s (from compressed)", + format_bytes(read_throughput_bytes as usize) + ); + } + } + + // Cleanup + let _ = fs::remove_file(&data_file); + let _ = fs::remove_file(&index_file); +} + +fn load_parquet(path: &PathBuf, batch_size: usize) -> Vec { + let mut batches = Vec::new(); + + let paths = if path.is_dir() { + let mut files: Vec = fs::read_dir(path) + .expect("Failed to read input directory") + .filter_map(|entry| { + let entry = entry.ok()?; + let p = entry.path(); + if p.extension().and_then(|e| e.to_str()) == Some("parquet") { + Some(p) + } else { + None + } + }) + .collect(); + files.sort(); + if files.is_empty() { + panic!("No .parquet files found in {}", path.display()); + } + files + } else { + vec![path.clone()] + }; + + for file_path in &paths { + let file = fs::File::open(file_path) + .unwrap_or_else(|e| panic!("Failed to open {}: {}", file_path.display(), e)); + let builder = ParquetRecordBatchReaderBuilder::try_new(file).unwrap_or_else(|e| { + panic!( + "Failed to read Parquet metadata from {}: {}", + file_path.display(), + e + ) + }); + let reader = builder + .with_batch_size(batch_size) + .build() + .unwrap_or_else(|e| { + panic!( + "Failed to build Parquet reader for {}: {}", + file_path.display(), + e + ) + }); + for batch_result in reader { + let batch = batch_result.unwrap_or_else(|e| { + panic!("Failed to read batch from {}: {}", file_path.display(), e) + }); + if batch.num_rows() > 0 { + batches.push(batch); + } + } + } + + if batches.is_empty() { + panic!("No data read from input"); + } + + println!( + "Loaded {} batches from {} file(s)", + batches.len(), + paths.len() + ); + batches +} + +fn generate_data(args: &Args) -> Vec { + let mut fields = Vec::new(); + let mut col_idx = 0; + + // Int64 columns + for _ in 0..args.gen_int_cols { + fields.push(Field::new( + format!("int_col_{col_idx}"), + DataType::Int64, + true, + )); + col_idx += 1; + } + // String columns + for _ in 0..args.gen_string_cols { + fields.push(Field::new( + format!("str_col_{col_idx}"), + DataType::Utf8, + true, + )); + col_idx += 1; + } + // Decimal columns + for _ in 0..args.gen_decimal_cols { + fields.push(Field::new( + format!("dec_col_{col_idx}"), + DataType::Decimal128(18, 2), + true, + )); + col_idx += 1; + } + // Date columns + for _ in 0..args.gen_date_cols { + fields.push(Field::new( + format!("date_col_{col_idx}"), + DataType::Date32, + true, + )); + col_idx += 1; + } + + let schema = Arc::new(Schema::new(fields)); + let mut batches = Vec::new(); + let mut rng = rand::rng(); + let mut remaining = args.gen_rows; + + while remaining > 0 { + let batch_rows = remaining.min(args.batch_size); + remaining -= batch_rows; + + let mut columns: Vec> = Vec::new(); + + // Int64 columns + for _ in 0..args.gen_int_cols { + let mut builder = Int64Builder::with_capacity(batch_rows); + for _ in 0..batch_rows { + if rng.random_range(0..100) < 5 { + builder.append_null(); + } else { + builder.append_value(rng.random_range(0..1_000_000i64)); + } + } + columns.push(Arc::new(builder.finish())); + } + // String columns + for _ in 0..args.gen_string_cols { + let mut builder = + StringBuilder::with_capacity(batch_rows, batch_rows * args.gen_avg_string_len); + for _ in 0..batch_rows { + if rng.random_range(0..100) < 5 { + builder.append_null(); + } else { + let len = rng.random_range(1..args.gen_avg_string_len * 2); + let s: String = (0..len) + .map(|_| rng.random_range(b'a'..=b'z') as char) + .collect(); + builder.append_value(&s); + } + } + columns.push(Arc::new(builder.finish())); + } + // Decimal columns + for _ in 0..args.gen_decimal_cols { + let mut builder = Decimal128Builder::with_capacity(batch_rows) + .with_precision_and_scale(18, 2) + .unwrap(); + for _ in 0..batch_rows { + if rng.random_range(0..100) < 5 { + builder.append_null(); + } else { + builder.append_value(rng.random_range(0..100_000_000i128)); + } + } + columns.push(Arc::new(builder.finish())); + } + // Date columns + for _ in 0..args.gen_date_cols { + let mut builder = Date32Builder::with_capacity(batch_rows); + for _ in 0..batch_rows { + if rng.random_range(0..100) < 5 { + builder.append_null(); + } else { + builder.append_value(rng.random_range(0..20000i32)); + } + } + columns.push(Arc::new(builder.finish())); + } + + let batch = RecordBatch::try_new(Arc::clone(&schema), columns).unwrap(); + batches.push(batch); + } + + println!( + "Generated {} batches ({} rows)", + batches.len(), + args.gen_rows + ); + batches +} + +fn run_shuffle_write( + batches: &[RecordBatch], + schema: &SchemaRef, + codec: &CompressionCodec, + hash_col_indices: &[usize], + args: &Args, + data_file: &str, + index_file: &str, +) -> f64 { + let partitioning = build_partitioning( + &args.partitioning, + args.partitions, + hash_col_indices, + schema, + ); + + let partitions = &[batches.to_vec()]; + let exec = ShuffleWriterExec::try_new( + Arc::new(DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new(partitions, Arc::clone(schema), None).unwrap(), + ))), + partitioning, + codec.clone(), + data_file.to_string(), + index_file.to_string(), + false, + args.write_buffer_size, + ) + .expect("Failed to create ShuffleWriterExec"); + + let config = SessionConfig::new().with_batch_size(args.batch_size); + let mut runtime_builder = RuntimeEnvBuilder::new(); + if let Some(mem_limit) = args.memory_limit { + runtime_builder = runtime_builder.with_memory_limit(mem_limit, 1.0); + } + let runtime_env = Arc::new(runtime_builder.build().unwrap()); + let ctx = SessionContext::new_with_config_rt(config, runtime_env); + let task_ctx = ctx.task_ctx(); + + let start = Instant::now(); + let stream = exec.execute(0, task_ctx).unwrap(); + let rt = tokio::runtime::Runtime::new().unwrap(); + rt.block_on(collect(stream)).unwrap(); + start.elapsed().as_secs_f64() +} + +fn run_shuffle_read(data_file: &str, index_file: &str, num_partitions: usize) -> f64 { + let start = Instant::now(); + + // Read index file to get partition offsets + let index_bytes = fs::read(index_file).expect("Failed to read index file"); + let num_offsets = index_bytes.len() / 8; + let offsets: Vec = (0..num_offsets) + .map(|i| { + let bytes: [u8; 8] = index_bytes[i * 8..(i + 1) * 8].try_into().unwrap(); + i64::from_le_bytes(bytes) + }) + .collect(); + + // Read data file + let data_bytes = fs::read(data_file).expect("Failed to read data file"); + + let mut total_rows = 0usize; + let mut total_batches = 0usize; + + // Decode each partition's data + for p in 0..num_partitions.min(offsets.len().saturating_sub(1)) { + let start_offset = offsets[p] as usize; + let end_offset = offsets[p + 1] as usize; + + if start_offset >= end_offset { + continue; // Empty partition + } + + // Read all IPC blocks within this partition + let mut offset = start_offset; + while offset < end_offset { + // First 8 bytes: IPC length + let ipc_length = + u64::from_le_bytes(data_bytes[offset..offset + 8].try_into().unwrap()) as usize; + + // Skip 8-byte length prefix, then 8 bytes of field_count + codec header + let block_data = &data_bytes[offset + 16..offset + 8 + ipc_length]; + let batch = read_ipc_compressed(block_data).expect("Failed to decode shuffle block"); + total_rows += batch.num_rows(); + total_batches += 1; + + offset += 8 + ipc_length; + } + } + + let elapsed = start.elapsed().as_secs_f64(); + eprintln!( + " read back {} rows in {} batches from {} partitions", + format_number(total_rows), + total_batches, + num_partitions + ); + elapsed +} + +fn build_partitioning( + scheme: &str, + num_partitions: usize, + hash_col_indices: &[usize], + schema: &SchemaRef, +) -> CometPartitioning { + match scheme { + "single" => CometPartitioning::SinglePartition, + "round-robin" => CometPartitioning::RoundRobin(num_partitions, 0), + "hash" => { + let exprs: Vec> = hash_col_indices + .iter() + .map(|&idx| { + let field = schema.field(idx); + Arc::new(Column::new(field.name(), idx)) + as Arc + }) + .collect(); + CometPartitioning::Hash(exprs, num_partitions) + } + other => { + eprintln!("Unknown partitioning scheme: {other}. Using hash."); + build_partitioning("hash", num_partitions, hash_col_indices, schema) + } + } +} + +fn parse_codec(codec: &str, zstd_level: i32) -> CompressionCodec { + match codec.to_lowercase().as_str() { + "none" => CompressionCodec::None, + "lz4" => CompressionCodec::Lz4Frame, + "zstd" => CompressionCodec::Zstd(zstd_level), + "snappy" => CompressionCodec::Snappy, + other => { + eprintln!("Unknown codec: {other}. Using zstd."); + CompressionCodec::Zstd(zstd_level) + } + } +} + +fn parse_hash_columns(s: &str) -> Vec { + s.split(',') + .filter(|s| !s.is_empty()) + .map(|s| s.trim().parse::().expect("Invalid column index")) + .collect() +} + +fn describe_schema(schema: &Schema) -> String { + let mut counts = std::collections::HashMap::new(); + for field in schema.fields() { + let type_name = match field.data_type() { + DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::UInt8 + | DataType::UInt16 + | DataType::UInt32 + | DataType::UInt64 => "int", + DataType::Float16 | DataType::Float32 | DataType::Float64 => "float", + DataType::Utf8 | DataType::LargeUtf8 => "string", + DataType::Boolean => "bool", + DataType::Date32 | DataType::Date64 => "date", + DataType::Decimal128(_, _) | DataType::Decimal256(_, _) => "decimal", + DataType::Timestamp(_, _) => "timestamp", + DataType::Binary | DataType::LargeBinary | DataType::FixedSizeBinary(_) => "binary", + _ => "other", + }; + *counts.entry(type_name).or_insert(0) += 1; + } + let mut parts: Vec = counts + .into_iter() + .map(|(k, v)| format!("{}x{}", v, k)) + .collect(); + parts.sort(); + parts.join(", ") +} + +fn format_number(n: usize) -> String { + let s = n.to_string(); + let mut result = String::new(); + for (i, c) in s.chars().rev().enumerate() { + if i > 0 && i % 3 == 0 { + result.push(','); + } + result.push(c); + } + result.chars().rev().collect() +} + +fn format_bytes(bytes: usize) -> String { + if bytes >= 1024 * 1024 * 1024 { + format!("{:.2} GiB", bytes as f64 / (1024.0 * 1024.0 * 1024.0)) + } else if bytes >= 1024 * 1024 { + format!("{:.2} MiB", bytes as f64 / (1024.0 * 1024.0)) + } else if bytes >= 1024 { + format!("{:.2} KiB", bytes as f64 / 1024.0) + } else { + format!("{bytes} B") + } +} From 9b5b305cb125a76e22f55ac61ee4994ff9e9f484 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 21 Mar 2026 09:05:16 -0600 Subject: [PATCH 02/46] feat: add --limit option to shuffle benchmark (default 1M rows) --- native/core/src/bin/shuffle_bench.rs | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/native/core/src/bin/shuffle_bench.rs b/native/core/src/bin/shuffle_bench.rs index c1498161f7..9b963c5803 100644 --- a/native/core/src/bin/shuffle_bench.rs +++ b/native/core/src/bin/shuffle_bench.rs @@ -152,6 +152,10 @@ struct Args { /// Write buffer size in bytes #[arg(long, default_value_t = 1048576)] write_buffer_size: usize, + + /// Maximum number of rows to use (default: 1,000,000) + #[arg(long, default_value_t = 1_000_000)] + limit: usize, } fn main() { @@ -178,6 +182,26 @@ fn main() { }; let load_elapsed = load_start.elapsed(); + // Apply row limit + let batches = { + let mut limited = Vec::new(); + let mut rows_so_far = 0usize; + for batch in batches { + if rows_so_far >= args.limit { + break; + } + let remaining = args.limit - rows_so_far; + if batch.num_rows() <= remaining { + rows_so_far += batch.num_rows(); + limited.push(batch); + } else { + limited.push(batch.slice(0, remaining)); + rows_so_far += remaining; + } + } + limited + }; + let schema = batches[0].schema(); let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum(); let total_bytes: usize = batches.iter().map(|b| b.get_array_memory_size()).sum(); From e1ab490c57b04387e4885284811ccf2ad993f4a6 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 21 Mar 2026 09:19:01 -0600 Subject: [PATCH 03/46] perf: apply limit during parquet read to avoid scanning all files --- native/core/src/bin/shuffle_bench.rs | 43 ++++++++++++---------------- 1 file changed, 18 insertions(+), 25 deletions(-) diff --git a/native/core/src/bin/shuffle_bench.rs b/native/core/src/bin/shuffle_bench.rs index 9b963c5803..17b1a9a6ff 100644 --- a/native/core/src/bin/shuffle_bench.rs +++ b/native/core/src/bin/shuffle_bench.rs @@ -176,32 +176,12 @@ fn main() { // Load data let load_start = Instant::now(); let batches = if let Some(ref input_path) = args.input { - load_parquet(input_path, args.batch_size) + load_parquet(input_path, args.batch_size, args.limit) } else { generate_data(&args) }; let load_elapsed = load_start.elapsed(); - // Apply row limit - let batches = { - let mut limited = Vec::new(); - let mut rows_so_far = 0usize; - for batch in batches { - if rows_so_far >= args.limit { - break; - } - let remaining = args.limit - rows_so_far; - if batch.num_rows() <= remaining { - rows_so_far += batch.num_rows(); - limited.push(batch); - } else { - limited.push(batch.slice(0, remaining)); - rows_so_far += remaining; - } - } - limited - }; - let schema = batches[0].schema(); let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum(); let total_bytes: usize = batches.iter().map(|b| b.get_array_memory_size()).sum(); @@ -358,8 +338,9 @@ fn main() { let _ = fs::remove_file(&index_file); } -fn load_parquet(path: &PathBuf, batch_size: usize) -> Vec { +fn load_parquet(path: &PathBuf, batch_size: usize, limit: usize) -> Vec { let mut batches = Vec::new(); + let mut total_rows = 0usize; let paths = if path.is_dir() { let mut files: Vec = fs::read_dir(path) @@ -383,7 +364,7 @@ fn load_parquet(path: &PathBuf, batch_size: usize) -> Vec { vec![path.clone()] }; - for file_path in &paths { + 'outer: for file_path in &paths { let file = fs::File::open(file_path) .unwrap_or_else(|e| panic!("Failed to open {}: {}", file_path.display(), e)); let builder = ParquetRecordBatchReaderBuilder::try_new(file).unwrap_or_else(|e| { @@ -407,8 +388,19 @@ fn load_parquet(path: &PathBuf, batch_size: usize) -> Vec { let batch = batch_result.unwrap_or_else(|e| { panic!("Failed to read batch from {}: {}", file_path.display(), e) }); - if batch.num_rows() > 0 { + if batch.num_rows() == 0 { + continue; + } + let remaining = limit - total_rows; + if batch.num_rows() <= remaining { + total_rows += batch.num_rows(); batches.push(batch); + } else { + batches.push(batch.slice(0, remaining)); + total_rows += remaining; + } + if total_rows >= limit { + break 'outer; } } } @@ -418,8 +410,9 @@ fn load_parquet(path: &PathBuf, batch_size: usize) -> Vec { } println!( - "Loaded {} batches from {} file(s)", + "Loaded {} batches ({} rows) from {} file(s)", batches.len(), + format_number(total_rows), paths.len() ); batches From ca36cbd579d5c512e9617eb54cd052ec0d2c244d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 23 Mar 2026 06:39:47 -0700 Subject: [PATCH 04/46] chore: add comment explaining parquet/rand deps in shuffle crate --- native/shuffle/Cargo.toml | 1 + 1 file changed, 1 insertion(+) diff --git a/native/shuffle/Cargo.toml b/native/shuffle/Cargo.toml index 1a64a36ff2..1337ea3b15 100644 --- a/native/shuffle/Cargo.toml +++ b/native/shuffle/Cargo.toml @@ -43,6 +43,7 @@ itertools = "0.14.0" jni = "0.21" log = "0.4" lz4_flex = { version = "0.13.0", default-features = false, features = ["frame"] } +# parquet and rand are only used by the shuffle_bench binary (shuffle-bench feature) parquet = { workspace = true, optional = true } rand = { workspace = true, optional = true } simd-adler32 = "0.3.7" From 6e8bed28902e7256e1e1b3e914439e0a5abf3622 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 26 Mar 2026 07:13:41 -0700 Subject: [PATCH 05/46] perf: add max_buffered_batches config and stream shuffle bench from parquet - Add `spark.comet.exec.shuffle.maxBufferedBatches` config to limit the number of batches buffered before spilling, allowing earlier spilling to reduce peak memory usage on executors - Fix too-many-open-files: close spill file FD after each spill and reopen in append mode, rather than holding one FD open per partition - Refactor shuffle_bench to stream directly from Parquet instead of loading all input data into memory; remove synthetic data generation - Add --max-buffered-batches CLI arg to shuffle_bench - Add shuffle benchmark documentation to README --- .../scala/org/apache/comet/CometConf.scala | 12 + native/core/src/execution/planner.rs | 2 + native/proto/src/proto/operator.proto | 3 + native/shuffle/README.md | 43 ++ native/shuffle/benches/shuffle_writer.rs | 1 + native/shuffle/src/bin/shuffle_bench.rs | 432 +++++------------- .../src/partitioners/multi_partition.rs | 11 + native/shuffle/src/shuffle_writer.rs | 11 + .../shuffle/src/writers/partition_writer.rs | 29 +- .../shuffle/CometNativeShuffleWriter.scala | 2 + 10 files changed, 209 insertions(+), 337 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index bfe90181ff..2ad9c1f609 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -534,6 +534,18 @@ object CometConf extends ShimCometConf { .checkValue(v => v > 0, "Write buffer size must be positive") .createWithDefault(1) + val COMET_SHUFFLE_MAX_BUFFERED_BATCHES: ConfigEntry[Int] = + conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.maxBufferedBatches") + .category(CATEGORY_SHUFFLE) + .doc("Maximum number of batches to buffer in memory before spilling to disk during " + + "native shuffle. Setting this to a small value causes earlier spilling, which reduces " + + "peak memory usage on executors at the cost of more disk I/O. " + + "The default value of 0 disables this limit and spills only when the memory pool is " + + "exhausted.") + .intConf + .checkValue(v => v >= 0, "Max buffered batches must be non-negative") + .createWithDefault(0) + val COMET_SHUFFLE_PREFER_DICTIONARY_RATIO: ConfigEntry[Double] = conf( "spark.comet.shuffle.preferDictionary.ratio") .category(CATEGORY_SHUFFLE) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 5af31fcc22..b6e05fccd1 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1352,6 +1352,7 @@ impl PhysicalPlanner { }?; let write_buffer_size = writer.write_buffer_size as usize; + let max_buffered_batches = writer.max_buffered_batches as usize; let shuffle_writer = Arc::new(ShuffleWriterExec::try_new( Arc::clone(&child.native_plan), partitioning, @@ -1360,6 +1361,7 @@ impl PhysicalPlanner { writer.output_index_file.clone(), writer.tracing_enabled, write_buffer_size, + max_buffered_batches, )?); Ok(( diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 344b9f0f21..5e23aad061 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -294,6 +294,9 @@ message ShuffleWriter { // Size of the write buffer in bytes used when writing shuffle data to disk. // Larger values may improve write performance but use more memory. int32 write_buffer_size = 8; + // Maximum number of batches to buffer before spilling to disk. + // 0 means no limit (spill only when memory pool is exhausted). + int32 max_buffered_batches = 9; } message ParquetWriter { diff --git a/native/shuffle/README.md b/native/shuffle/README.md index 8fba6b0323..0333ddbe8e 100644 --- a/native/shuffle/README.md +++ b/native/shuffle/README.md @@ -23,3 +23,46 @@ This crate provides the shuffle writer and reader implementation for Apache Data of the [Apache DataFusion Comet] subproject. [Apache DataFusion Comet]: https://github.com/apache/datafusion-comet/ + +## Shuffle Benchmark Tool + +A standalone benchmark binary (`shuffle_bench`) is included for profiling shuffle write and read +performance outside of Spark. It streams input data directly from Parquet files. + +### Basic usage + +```sh +cargo run --release --features shuffle-bench --bin shuffle_bench -- \ + --input /data/tpch-sf100/lineitem/ \ + --partitions 200 \ + --codec zstd --zstd-level 1 \ + --hash-columns 0,3 +``` + +### Options + +| Option | Default | Description | +|---|---|---| +| `--input` | *(required)* | Path to a Parquet file or directory of Parquet files | +| `--partitions` | `200` | Number of output shuffle partitions | +| `--partitioning` | `hash` | Partitioning scheme: `hash`, `single`, `round-robin` | +| `--hash-columns` | `0` | Comma-separated column indices to hash on (e.g. `0,3`) | +| `--codec` | `zstd` | Compression codec: `none`, `lz4`, `zstd`, `snappy` | +| `--zstd-level` | `1` | Zstd compression level (1–22) | +| `--batch-size` | `8192` | Batch size for reading Parquet data | +| `--memory-limit` | *(none)* | Memory limit in bytes; triggers spilling when exceeded | +| `--max-buffered-batches` | `0` | Max batches to buffer before spilling (0 = memory-pool-only) | +| `--write-buffer-size` | `1048576` | Write buffer size in bytes | +| `--limit` | `0` | Limit rows processed per iteration (0 = no limit) | +| `--iterations` | `1` | Number of timed iterations | +| `--warmup` | `0` | Number of warmup iterations before timing | +| `--read-back` | `false` | Also benchmark reading back the shuffle output | +| `--output-dir` | `/tmp/comet_shuffle_bench` | Directory for temporary shuffle output files | + +### Profiling with flamegraph + +```sh +cargo flamegraph --release --features shuffle-bench --bin shuffle_bench -- \ + --input /data/tpch-sf100/lineitem/ \ + --partitions 200 --codec zstd --zstd-level 1 +``` diff --git a/native/shuffle/benches/shuffle_writer.rs b/native/shuffle/benches/shuffle_writer.rs index 27abd919fa..8ff1f024d5 100644 --- a/native/shuffle/benches/shuffle_writer.rs +++ b/native/shuffle/benches/shuffle_writer.rs @@ -153,6 +153,7 @@ fn create_shuffle_writer_exec( "/tmp/index.out".to_string(), false, 1024 * 1024, + 0, // max_buffered_batches: no limit ) .unwrap() } diff --git a/native/shuffle/src/bin/shuffle_bench.rs b/native/shuffle/src/bin/shuffle_bench.rs index 373d0f92b9..0f04954ffa 100644 --- a/native/shuffle/src/bin/shuffle_bench.rs +++ b/native/shuffle/src/bin/shuffle_bench.rs @@ -16,11 +16,10 @@ // under the License. //! Standalone shuffle benchmark tool for profiling Comet shuffle write and read -//! outside of Spark. +//! outside of Spark. Streams input directly from Parquet files. //! //! # Usage //! -//! Read from Parquet files (e.g. TPC-H lineitem): //! ```sh //! cargo run --release --bin shuffle_bench -- \ //! --input /data/tpch-sf100/lineitem/ \ @@ -30,14 +29,6 @@ //! --read-back //! ``` //! -//! Generate synthetic data: -//! ```sh -//! cargo run --release --bin shuffle_bench -- \ -//! --generate --gen-rows 10000000 --gen-string-cols 4 --gen-int-cols 4 \ -//! --gen-decimal-cols 2 --gen-avg-string-len 32 \ -//! --partitions 200 --codec lz4 --read-back -//! ``` -//! //! Profile with flamegraph: //! ```sh //! cargo flamegraph --release --bin shuffle_bench -- \ @@ -45,23 +36,19 @@ //! --partitions 200 --codec zstd --zstd-level 1 //! ``` -use arrow::array::builder::{Date32Builder, Decimal128Builder, Int64Builder, StringBuilder}; -use arrow::array::RecordBatch; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow::datatypes::{DataType, SchemaRef}; use clap::Parser; -use datafusion::datasource::memory::MemorySourceConfig; -use datafusion::datasource::source::DataSourceExec; use datafusion::execution::config::SessionConfig; use datafusion::execution::runtime_env::RuntimeEnvBuilder; use datafusion::physical_expr::expressions::Column; +use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::common::collect; use datafusion::physical_plan::ExecutionPlan; -use datafusion::prelude::SessionContext; +use datafusion::prelude::{ParquetReadOptions, SessionContext}; use datafusion_comet_shuffle::{ read_ipc_compressed, CometPartitioning, CompressionCodec, ShuffleWriterExec, }; use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; -use rand::RngExt; use std::fs; use std::path::PathBuf; use std::sync::Arc; @@ -75,37 +62,9 @@ use std::time::Instant; struct Args { /// Path to input Parquet file or directory of Parquet files #[arg(long)] - input: Option, - - /// Generate synthetic data instead of reading from Parquet - #[arg(long, default_value_t = false)] - generate: bool, - - /// Number of rows to generate (requires --generate) - #[arg(long, default_value_t = 1_000_000)] - gen_rows: usize, - - /// Number of Int64 columns to generate - #[arg(long, default_value_t = 4)] - gen_int_cols: usize, + input: PathBuf, - /// Number of Utf8 string columns to generate - #[arg(long, default_value_t = 2)] - gen_string_cols: usize, - - /// Number of Decimal128 columns to generate - #[arg(long, default_value_t = 2)] - gen_decimal_cols: usize, - - /// Number of Date32 columns to generate - #[arg(long, default_value_t = 1)] - gen_date_cols: usize, - - /// Average string length for generated string columns - #[arg(long, default_value_t = 24)] - gen_avg_string_len: usize, - - /// Batch size for reading Parquet or generating data + /// Batch size for reading Parquet data #[arg(long, default_value_t = 8192)] batch_size: usize, @@ -153,70 +112,37 @@ struct Args { #[arg(long, default_value_t = 1048576)] write_buffer_size: usize, - /// Maximum number of rows to use (default: 1,000,000) - #[arg(long, default_value_t = 1_000_000)] + /// Maximum number of batches to buffer before spilling (0 = no limit) + #[arg(long, default_value_t = 0)] + max_buffered_batches: usize, + + /// Limit rows processed per iteration (0 = no limit) + #[arg(long, default_value_t = 0)] limit: usize, } fn main() { let args = Args::parse(); - // Validate args - if args.input.is_none() && !args.generate { - eprintln!("Error: must specify either --input or --generate"); - std::process::exit(1); - } - // Create output directory fs::create_dir_all(&args.output_dir).expect("Failed to create output directory"); - let data_file = args.output_dir.join("data.out"); let index_file = args.output_dir.join("index.out"); - // Load data - let load_start = Instant::now(); - let batches = if let Some(ref input_path) = args.input { - load_parquet(input_path, args.batch_size, args.limit) - } else { - generate_data(&args) - }; - let load_elapsed = load_start.elapsed(); + let (schema, total_rows) = read_parquet_metadata(&args.input, args.limit); - let schema = batches[0].schema(); - let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum(); - let total_bytes: usize = batches.iter().map(|b| b.get_array_memory_size()).sum(); + let codec = parse_codec(&args.codec, args.zstd_level); + let hash_col_indices = parse_hash_columns(&args.hash_columns); println!("=== Shuffle Benchmark ==="); + println!("Input: {}", args.input.display()); println!( - "Data source: {}", - if args.input.is_some() { - "parquet" - } else { - "generated" - } - ); - println!( - "Schema: {} columns ({} fields)", + "Schema: {} columns ({})", schema.fields().len(), describe_schema(&schema) ); - println!("Total rows: {}", format_number(total_rows)); - println!("Total size: {}", format_bytes(total_bytes)); - println!("Batches: {}", batches.len()); - println!( - "Rows/batch: ~{}", - if batches.is_empty() { - 0 - } else { - total_rows / batches.len() - } - ); - println!("Load time: {:.3}s", load_elapsed.as_secs_f64()); - println!(); - - let codec = parse_codec(&args.codec, args.zstd_level); - let hash_col_indices = parse_hash_columns(&args.hash_columns); - + println!("Total rows: {}", format_number(total_rows as usize)); + println!("Batch size: {}", format_number(args.batch_size)); println!("Partitioning: {}", args.partitioning); println!("Partitions: {}", args.partitions); println!("Codec: {:?}", codec); @@ -224,13 +150,15 @@ fn main() { if let Some(mem_limit) = args.memory_limit { println!("Memory limit: {}", format_bytes(mem_limit)); } + if args.max_buffered_batches > 0 { + println!("Max buf batches:{}", args.max_buffered_batches); + } println!( "Iterations: {} (warmup: {})", args.iterations, args.warmup ); println!(); - // Run warmup + timed iterations let total_iters = args.warmup + args.iterations; let mut write_times = Vec::with_capacity(args.iterations); let mut read_times = Vec::with_capacity(args.iterations); @@ -244,9 +172,8 @@ fn main() { format!("iter {}/{}", i - args.warmup + 1, args.iterations) }; - // Write phase let write_elapsed = run_shuffle_write( - &batches, + &args.input, &schema, &codec, &hash_col_indices, @@ -264,7 +191,6 @@ fn main() { print!(" [{label}] write: {:.3}s", write_elapsed); print!(" output: {}", format_bytes(data_size as usize)); - // Read phase if args.read_back { let read_elapsed = run_shuffle_read( data_file.to_str().unwrap(), @@ -279,7 +205,6 @@ fn main() { println!(); } - // Print summary if args.iterations > 0 { println!(); println!("=== Results ==="); @@ -287,12 +212,6 @@ fn main() { let avg_write = write_times.iter().sum::() / write_times.len() as f64; let avg_data_size = data_file_sizes.iter().sum::() / data_file_sizes.len() as u64; let write_throughput_rows = total_rows as f64 / avg_write; - let write_throughput_bytes = total_bytes as f64 / avg_write; - let compression_ratio = if avg_data_size > 0 { - total_bytes as f64 / avg_data_size as f64 - } else { - 0.0 - }; println!("Write:"); println!(" avg time: {:.3}s", avg_write); @@ -305,15 +224,13 @@ fn main() { println!(" min/max: {:.3}s / {:.3}s", min, max); } println!( - " throughput: {}/s ({} rows/s)", - format_bytes(write_throughput_bytes as usize), + " throughput: {} rows/s", format_number(write_throughput_rows as usize) ); println!( " output size: {}", format_bytes(avg_data_size as usize) ); - println!(" compression: {:.2}x", compression_ratio); if !read_times.is_empty() { let avg_read = read_times.iter().sum::() / read_times.len() as f64; @@ -333,38 +250,17 @@ fn main() { } } - // Cleanup let _ = fs::remove_file(&data_file); let _ = fs::remove_file(&index_file); } -fn load_parquet(path: &PathBuf, batch_size: usize, limit: usize) -> Vec { - let mut batches = Vec::new(); - let mut total_rows = 0usize; +/// Read schema and total row count from Parquet metadata without loading any data. +fn read_parquet_metadata(path: &PathBuf, limit: usize) -> (SchemaRef, u64) { + let paths = collect_parquet_paths(path); + let mut schema = None; + let mut total_rows = 0u64; - let paths = if path.is_dir() { - let mut files: Vec = fs::read_dir(path) - .expect("Failed to read input directory") - .filter_map(|entry| { - let entry = entry.ok()?; - let p = entry.path(); - if p.extension().and_then(|e| e.to_str()) == Some("parquet") { - Some(p) - } else { - None - } - }) - .collect(); - files.sort(); - if files.is_empty() { - panic!("No .parquet files found in {}", path.display()); - } - files - } else { - vec![path.clone()] - }; - - 'outer: for file_path in &paths { + for file_path in &paths { let file = fs::File::open(file_path) .unwrap_or_else(|e| panic!("Failed to open {}: {}", file_path.display(), e)); let builder = ParquetRecordBatchReaderBuilder::try_new(file).unwrap_or_else(|e| { @@ -374,172 +270,44 @@ fn load_parquet(path: &PathBuf, batch_size: usize, limit: usize) -> Vec= limit { - break 'outer; - } + if schema.is_none() { + schema = Some(Arc::clone(builder.schema())); + } + total_rows += builder.metadata().file_metadata().num_rows() as u64; + if limit > 0 && total_rows >= limit as u64 { + total_rows = total_rows.min(limit as u64); + break; } } - if batches.is_empty() { - panic!("No data read from input"); - } - - println!( - "Loaded {} batches ({} rows) from {} file(s)", - batches.len(), - format_number(total_rows), - paths.len() - ); - batches + (schema.expect("No parquet files found"), total_rows) } -fn generate_data(args: &Args) -> Vec { - let mut fields = Vec::new(); - let mut col_idx = 0; - - // Int64 columns - for _ in 0..args.gen_int_cols { - fields.push(Field::new( - format!("int_col_{col_idx}"), - DataType::Int64, - true, - )); - col_idx += 1; - } - // String columns - for _ in 0..args.gen_string_cols { - fields.push(Field::new( - format!("str_col_{col_idx}"), - DataType::Utf8, - true, - )); - col_idx += 1; - } - // Decimal columns - for _ in 0..args.gen_decimal_cols { - fields.push(Field::new( - format!("dec_col_{col_idx}"), - DataType::Decimal128(18, 2), - true, - )); - col_idx += 1; - } - // Date columns - for _ in 0..args.gen_date_cols { - fields.push(Field::new( - format!("date_col_{col_idx}"), - DataType::Date32, - true, - )); - col_idx += 1; - } - - let schema = Arc::new(Schema::new(fields)); - let mut batches = Vec::new(); - let mut rng = rand::rng(); - let mut remaining = args.gen_rows; - - while remaining > 0 { - let batch_rows = remaining.min(args.batch_size); - remaining -= batch_rows; - - let mut columns: Vec> = Vec::new(); - - // Int64 columns - for _ in 0..args.gen_int_cols { - let mut builder = Int64Builder::with_capacity(batch_rows); - for _ in 0..batch_rows { - if rng.random_range(0..100) < 5 { - builder.append_null(); - } else { - builder.append_value(rng.random_range(0..1_000_000i64)); - } - } - columns.push(Arc::new(builder.finish())); - } - // String columns - for _ in 0..args.gen_string_cols { - let mut builder = - StringBuilder::with_capacity(batch_rows, batch_rows * args.gen_avg_string_len); - for _ in 0..batch_rows { - if rng.random_range(0..100) < 5 { - builder.append_null(); - } else { - let len = rng.random_range(1..args.gen_avg_string_len * 2); - let s: String = (0..len) - .map(|_| rng.random_range(b'a'..=b'z') as char) - .collect(); - builder.append_value(&s); - } - } - columns.push(Arc::new(builder.finish())); - } - // Decimal columns - for _ in 0..args.gen_decimal_cols { - let mut builder = Decimal128Builder::with_capacity(batch_rows) - .with_precision_and_scale(18, 2) - .unwrap(); - for _ in 0..batch_rows { - if rng.random_range(0..100) < 5 { - builder.append_null(); - } else { - builder.append_value(rng.random_range(0..100_000_000i128)); - } - } - columns.push(Arc::new(builder.finish())); - } - // Date columns - for _ in 0..args.gen_date_cols { - let mut builder = Date32Builder::with_capacity(batch_rows); - for _ in 0..batch_rows { - if rng.random_range(0..100) < 5 { - builder.append_null(); +fn collect_parquet_paths(path: &PathBuf) -> Vec { + if path.is_dir() { + let mut files: Vec = fs::read_dir(path) + .unwrap_or_else(|e| panic!("Failed to read directory {}: {}", path.display(), e)) + .filter_map(|entry| { + let p = entry.ok()?.path(); + if p.extension().and_then(|e| e.to_str()) == Some("parquet") { + Some(p) } else { - builder.append_value(rng.random_range(0..20000i32)); + None } - } - columns.push(Arc::new(builder.finish())); + }) + .collect(); + files.sort(); + if files.is_empty() { + panic!("No .parquet files found in {}", path.display()); } - - let batch = RecordBatch::try_new(Arc::clone(&schema), columns).unwrap(); - batches.push(batch); + files + } else { + vec![path.clone()] } - - println!( - "Generated {} batches ({} rows)", - batches.len(), - args.gen_rows - ); - batches } fn run_shuffle_write( - batches: &[RecordBatch], + input_path: &PathBuf, schema: &SchemaRef, codec: &CompressionCodec, hash_col_indices: &[usize], @@ -554,40 +322,61 @@ fn run_shuffle_write( schema, ); - let partitions = &[batches.to_vec()]; - let exec = ShuffleWriterExec::try_new( - Arc::new(DataSourceExec::new(Arc::new( - MemorySourceConfig::try_new(partitions, Arc::clone(schema), None).unwrap(), - ))), - partitioning, - codec.clone(), - data_file.to_string(), - index_file.to_string(), - false, - args.write_buffer_size, - ) - .expect("Failed to create ShuffleWriterExec"); - - let config = SessionConfig::new().with_batch_size(args.batch_size); - let mut runtime_builder = RuntimeEnvBuilder::new(); - if let Some(mem_limit) = args.memory_limit { - runtime_builder = runtime_builder.with_memory_limit(mem_limit, 1.0); - } - let runtime_env = Arc::new(runtime_builder.build().unwrap()); - let ctx = SessionContext::new_with_config_rt(config, runtime_env); - let task_ctx = ctx.task_ctx(); - - let start = Instant::now(); - let stream = exec.execute(0, task_ctx).unwrap(); let rt = tokio::runtime::Runtime::new().unwrap(); - rt.block_on(collect(stream)).unwrap(); - start.elapsed().as_secs_f64() + rt.block_on(async { + let config = SessionConfig::new().with_batch_size(args.batch_size); + let mut runtime_builder = RuntimeEnvBuilder::new(); + if let Some(mem_limit) = args.memory_limit { + runtime_builder = runtime_builder.with_memory_limit(mem_limit, 1.0); + } + let runtime_env = Arc::new(runtime_builder.build().unwrap()); + let ctx = SessionContext::new_with_config_rt(config, runtime_env); + + let path_str = input_path.to_str().unwrap(); + let mut df = ctx + .read_parquet(path_str, ParquetReadOptions::default()) + .await + .expect("Failed to create Parquet scan"); + if args.limit > 0 { + df = df.limit(0, Some(args.limit)).unwrap(); + } + + let parquet_plan = df + .create_physical_plan() + .await + .expect("Failed to create physical plan"); + + // ShuffleWriterExec reads from a single input partition + let input: Arc = + if parquet_plan.properties().output_partitioning().partition_count() > 1 { + Arc::new(CoalescePartitionsExec::new(parquet_plan)) + } else { + parquet_plan + }; + + let exec = ShuffleWriterExec::try_new( + input, + partitioning, + codec.clone(), + data_file.to_string(), + index_file.to_string(), + false, + args.write_buffer_size, + args.max_buffered_batches, + ) + .expect("Failed to create ShuffleWriterExec"); + + let task_ctx = ctx.task_ctx(); + let start = Instant::now(); + let stream = exec.execute(0, task_ctx).unwrap(); + collect(stream).await.unwrap(); + start.elapsed().as_secs_f64() + }) } fn run_shuffle_read(data_file: &str, index_file: &str, num_partitions: usize) -> f64 { let start = Instant::now(); - // Read index file to get partition offsets let index_bytes = fs::read(index_file).expect("Failed to read index file"); let num_offsets = index_bytes.len() / 8; let offsets: Vec = (0..num_offsets) @@ -597,34 +386,27 @@ fn run_shuffle_read(data_file: &str, index_file: &str, num_partitions: usize) -> }) .collect(); - // Read data file let data_bytes = fs::read(data_file).expect("Failed to read data file"); let mut total_rows = 0usize; let mut total_batches = 0usize; - // Decode each partition's data for p in 0..num_partitions.min(offsets.len().saturating_sub(1)) { let start_offset = offsets[p] as usize; let end_offset = offsets[p + 1] as usize; if start_offset >= end_offset { - continue; // Empty partition + continue; } - // Read all IPC blocks within this partition let mut offset = start_offset; while offset < end_offset { - // First 8 bytes: IPC length let ipc_length = u64::from_le_bytes(data_bytes[offset..offset + 8].try_into().unwrap()) as usize; - - // Skip 8-byte length prefix, then 8 bytes of field_count + codec header let block_data = &data_bytes[offset + 16..offset + 8 + ipc_length]; let batch = read_ipc_compressed(block_data).expect("Failed to decode shuffle block"); total_rows += batch.num_rows(); total_batches += 1; - offset += 8 + ipc_length; } } @@ -686,7 +468,7 @@ fn parse_hash_columns(s: &str) -> Vec { .collect() } -fn describe_schema(schema: &Schema) -> String { +fn describe_schema(schema: &arrow::datatypes::Schema) -> String { let mut counts = std::collections::HashMap::new(); for field in schema.fields() { let type_name = match field.data_type() { diff --git a/native/shuffle/src/partitioners/multi_partition.rs b/native/shuffle/src/partitioners/multi_partition.rs index 42290c5510..4cc09e9679 100644 --- a/native/shuffle/src/partitioners/multi_partition.rs +++ b/native/shuffle/src/partitioners/multi_partition.rs @@ -124,6 +124,8 @@ pub(crate) struct MultiPartitionShuffleRepartitioner { tracing_enabled: bool, /// Size of the write buffer in bytes write_buffer_size: usize, + /// Maximum number of batches to buffer before spilling (0 = no limit) + max_buffered_batches: usize, } impl MultiPartitionShuffleRepartitioner { @@ -140,6 +142,7 @@ impl MultiPartitionShuffleRepartitioner { codec: CompressionCodec, tracing_enabled: bool, write_buffer_size: usize, + max_buffered_batches: usize, ) -> datafusion::common::Result { let num_output_partitions = partitioning.partition_count(); assert_ne!( @@ -189,6 +192,7 @@ impl MultiPartitionShuffleRepartitioner { reservation, tracing_enabled, write_buffer_size, + max_buffered_batches, }) } @@ -397,6 +401,13 @@ impl MultiPartitionShuffleRepartitioner { partition_row_indices: &[u32], partition_starts: &[u32], ) -> datafusion::common::Result<()> { + // Spill before buffering if we've reached the configured batch count limit. + if self.max_buffered_batches > 0 + && self.buffered_batches.len() >= self.max_buffered_batches + { + self.spill()?; + } + let mut mem_growth: usize = input.get_array_memory_size(); let buffered_partition_idx = self.buffered_batches.len() as u32; self.buffered_batches.push(input); diff --git a/native/shuffle/src/shuffle_writer.rs b/native/shuffle/src/shuffle_writer.rs index e649aaac69..95a09610a4 100644 --- a/native/shuffle/src/shuffle_writer.rs +++ b/native/shuffle/src/shuffle_writer.rs @@ -68,6 +68,8 @@ pub struct ShuffleWriterExec { tracing_enabled: bool, /// Size of the write buffer in bytes write_buffer_size: usize, + /// Maximum number of batches to buffer before spilling (0 = no limit) + max_buffered_batches: usize, } impl ShuffleWriterExec { @@ -81,6 +83,7 @@ impl ShuffleWriterExec { output_index_file: String, tracing_enabled: bool, write_buffer_size: usize, + max_buffered_batches: usize, ) -> Result { let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&input.schema())), @@ -99,6 +102,7 @@ impl ShuffleWriterExec { codec, tracing_enabled, write_buffer_size, + max_buffered_batches, }) } } @@ -163,6 +167,7 @@ impl ExecutionPlan for ShuffleWriterExec { self.output_index_file.clone(), self.tracing_enabled, self.write_buffer_size, + self.max_buffered_batches, )?)), _ => panic!("ShuffleWriterExec wrong number of children"), } @@ -190,6 +195,7 @@ impl ExecutionPlan for ShuffleWriterExec { self.codec.clone(), self.tracing_enabled, self.write_buffer_size, + self.max_buffered_batches, ) .map_err(|e| ArrowError::ExternalError(Box::new(e))), ) @@ -210,6 +216,7 @@ async fn external_shuffle( codec: CompressionCodec, tracing_enabled: bool, write_buffer_size: usize, + max_buffered_batches: usize, ) -> Result { with_trace_async("external_shuffle", tracing_enabled, || async { let schema = input.schema(); @@ -238,6 +245,7 @@ async fn external_shuffle( codec, tracing_enabled, write_buffer_size, + max_buffered_batches, )?), }; @@ -362,6 +370,7 @@ mod test { CompressionCodec::Lz4Frame, false, 1024 * 1024, // write_buffer_size: 1MB default + 0, // max_buffered_batches: no limit ) .unwrap(); @@ -466,6 +475,7 @@ mod test { "/tmp/index.out".to_string(), false, 1024 * 1024, // write_buffer_size: 1MB default + 0, // max_buffered_batches: no limit ) .unwrap(); @@ -525,6 +535,7 @@ mod test { index_file.clone(), false, 1024 * 1024, + 0, // max_buffered_batches: no limit ) .unwrap(); diff --git a/native/shuffle/src/writers/partition_writer.rs b/native/shuffle/src/writers/partition_writer.rs index 48017871db..4de307de62 100644 --- a/native/shuffle/src/writers/partition_writer.rs +++ b/native/shuffle/src/writers/partition_writer.rs @@ -26,7 +26,6 @@ use std::fs::{File, OpenOptions}; struct SpillFile { temp_file: RefCountedTempFile, - file: File, } pub(crate) struct PartitionWriter { @@ -53,26 +52,28 @@ impl PartitionWriter { runtime: &RuntimeEnv, ) -> datafusion::common::Result<()> { if self.spill_file.is_none() { - // Spill file is not yet created, create it let spill_file = runtime .disk_manager .create_tmp_file("shuffle writer spill")?; - let spill_data = OpenOptions::new() - .write(true) - .create(true) - .truncate(true) - .open(spill_file.path()) - .map_err(|e| { - DataFusionError::Execution(format!("Error occurred while spilling {e}")) - })?; + // Create the file (truncating any pre-existing content) + File::create(spill_file.path()).map_err(|e| { + DataFusionError::Execution(format!("Error occurred while spilling {e}")) + })?; self.spill_file = Some(SpillFile { temp_file: spill_file, - file: spill_data, }); } Ok(()) } + fn open_spill_file_for_append(&self) -> datafusion::common::Result { + OpenOptions::new() + .write(true) + .append(true) + .open(self.spill_file.as_ref().unwrap().temp_file.path()) + .map_err(|e| DataFusionError::Execution(format!("Error occurred while spilling {e}"))) + } + pub(crate) fn spill( &mut self, iter: &mut PartitionedBatchIterator, @@ -84,10 +85,13 @@ impl PartitionWriter { if let Some(batch) = iter.next() { self.ensure_spill_file_created(runtime)?; + // Open the file for this spill and close it when done, so we don't + // hold open one FD per partition across multiple spill events. + let mut spill_data = self.open_spill_file_for_append()?; let total_bytes_written = { let mut buf_batch_writer = BufBatchWriter::new( &mut self.shuffle_block_writer, - &mut self.spill_file.as_mut().unwrap().file, + &mut spill_data, write_buffer_size, batch_size, ); @@ -104,6 +108,7 @@ impl PartitionWriter { buf_batch_writer.flush(&metrics.encode_time, &metrics.write_time)?; bytes_written }; + // spill_data is dropped here, closing the file descriptor Ok(total_bytes_written) } else { 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 3fc222bd19..a80d8b2fa4 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 @@ -192,6 +192,8 @@ class CometNativeShuffleWriter[K, V]( CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_ZSTD_LEVEL.get) shuffleWriterBuilder.setWriteBufferSize( CometConf.COMET_SHUFFLE_WRITE_BUFFER_SIZE.get().max(Int.MaxValue).toInt) + shuffleWriterBuilder.setMaxBufferedBatches( + CometConf.COMET_SHUFFLE_MAX_BUFFERED_BATCHES.get()) outputPartitioning match { case p if isSinglePartitioning(p) => From 2ef57e79d4e9202d4fb1a70f62f5d8b717ca67b4 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 27 Mar 2026 12:05:23 -0600 Subject: [PATCH 06/46] cargo fmt --- native/shuffle/src/bin/shuffle_bench.rs | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/native/shuffle/src/bin/shuffle_bench.rs b/native/shuffle/src/bin/shuffle_bench.rs index 7f919e7674..ec1227a2ee 100644 --- a/native/shuffle/src/bin/shuffle_bench.rs +++ b/native/shuffle/src/bin/shuffle_bench.rs @@ -340,12 +340,16 @@ fn run_shuffle_write( .expect("Failed to create physical plan"); // ShuffleWriterExec reads from a single input partition - let input: Arc = - if parquet_plan.properties().output_partitioning().partition_count() > 1 { - Arc::new(CoalescePartitionsExec::new(parquet_plan)) - } else { - parquet_plan - }; + let input: Arc = if parquet_plan + .properties() + .output_partitioning() + .partition_count() + > 1 + { + Arc::new(CoalescePartitionsExec::new(parquet_plan)) + } else { + parquet_plan + }; let exec = ShuffleWriterExec::try_new( input, From 9136e104e87cae0c8920fcce790064de5716d425 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 27 Mar 2026 12:08:20 -0600 Subject: [PATCH 07/46] prettier --- native/shuffle/README.md | 34 +++++++++++++++++----------------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/native/shuffle/README.md b/native/shuffle/README.md index 0333ddbe8e..74b8dbe656 100644 --- a/native/shuffle/README.md +++ b/native/shuffle/README.md @@ -41,23 +41,23 @@ cargo run --release --features shuffle-bench --bin shuffle_bench -- \ ### Options -| Option | Default | Description | -|---|---|---| -| `--input` | *(required)* | Path to a Parquet file or directory of Parquet files | -| `--partitions` | `200` | Number of output shuffle partitions | -| `--partitioning` | `hash` | Partitioning scheme: `hash`, `single`, `round-robin` | -| `--hash-columns` | `0` | Comma-separated column indices to hash on (e.g. `0,3`) | -| `--codec` | `zstd` | Compression codec: `none`, `lz4`, `zstd`, `snappy` | -| `--zstd-level` | `1` | Zstd compression level (1–22) | -| `--batch-size` | `8192` | Batch size for reading Parquet data | -| `--memory-limit` | *(none)* | Memory limit in bytes; triggers spilling when exceeded | -| `--max-buffered-batches` | `0` | Max batches to buffer before spilling (0 = memory-pool-only) | -| `--write-buffer-size` | `1048576` | Write buffer size in bytes | -| `--limit` | `0` | Limit rows processed per iteration (0 = no limit) | -| `--iterations` | `1` | Number of timed iterations | -| `--warmup` | `0` | Number of warmup iterations before timing | -| `--read-back` | `false` | Also benchmark reading back the shuffle output | -| `--output-dir` | `/tmp/comet_shuffle_bench` | Directory for temporary shuffle output files | +| Option | Default | Description | +| ------------------------ | -------------------------- | ------------------------------------------------------------ | +| `--input` | _(required)_ | Path to a Parquet file or directory of Parquet files | +| `--partitions` | `200` | Number of output shuffle partitions | +| `--partitioning` | `hash` | Partitioning scheme: `hash`, `single`, `round-robin` | +| `--hash-columns` | `0` | Comma-separated column indices to hash on (e.g. `0,3`) | +| `--codec` | `zstd` | Compression codec: `none`, `lz4`, `zstd`, `snappy` | +| `--zstd-level` | `1` | Zstd compression level (1–22) | +| `--batch-size` | `8192` | Batch size for reading Parquet data | +| `--memory-limit` | _(none)_ | Memory limit in bytes; triggers spilling when exceeded | +| `--max-buffered-batches` | `0` | Max batches to buffer before spilling (0 = memory-pool-only) | +| `--write-buffer-size` | `1048576` | Write buffer size in bytes | +| `--limit` | `0` | Limit rows processed per iteration (0 = no limit) | +| `--iterations` | `1` | Number of timed iterations | +| `--warmup` | `0` | Number of warmup iterations before timing | +| `--read-back` | `false` | Also benchmark reading back the shuffle output | +| `--output-dir` | `/tmp/comet_shuffle_bench` | Directory for temporary shuffle output files | ### Profiling with flamegraph From 7e16819013e744eecd548483fcf414798f071744 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 27 Mar 2026 12:19:44 -0600 Subject: [PATCH 08/46] machete --- native/Cargo.lock | 1 - native/shuffle/Cargo.toml | 5 ++--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 88007af70a..2c117631a4 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -1996,7 +1996,6 @@ dependencies = [ "log", "lz4_flex 0.13.0", "parquet", - "rand 0.10.0", "simd-adler32", "snap", "tempfile", diff --git a/native/shuffle/Cargo.toml b/native/shuffle/Cargo.toml index 14ed5f60a0..a5982c05fa 100644 --- a/native/shuffle/Cargo.toml +++ b/native/shuffle/Cargo.toml @@ -43,9 +43,8 @@ itertools = "0.14.0" jni = "0.21" log = "0.4" lz4_flex = { version = "0.13.0", default-features = false, features = ["frame"] } -# parquet and rand are only used by the shuffle_bench binary (shuffle-bench feature) +# parquet is only used by the shuffle_bench binary (shuffle-bench feature) parquet = { workspace = true, optional = true } -rand = { workspace = true, optional = true } simd-adler32 = "0.3.9" snap = "1.1" tokio = { version = "1", features = ["rt-multi-thread"] } @@ -58,7 +57,7 @@ itertools = "0.14.0" tempfile = "3.26.0" [features] -shuffle-bench = ["clap", "parquet", "rand"] +shuffle-bench = ["clap", "parquet"] [lib] name = "datafusion_comet_shuffle" From 22fe80491e111d1f261988a7bd06f73764ba8360 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 28 Mar 2026 16:50:51 -0600 Subject: [PATCH 09/46] feat: add --concurrent-tasks flag to shuffle benchmark Spawns N parallel shuffle tasks to simulate executor parallelism. Each task reads the same input and writes to its own output files. Extracts core shuffle logic into shared async helper to avoid code duplication between single and concurrent paths. --- native/shuffle/src/bin/shuffle_bench.rs | 231 +++++++++++++++++------- 1 file changed, 169 insertions(+), 62 deletions(-) diff --git a/native/shuffle/src/bin/shuffle_bench.rs b/native/shuffle/src/bin/shuffle_bench.rs index ec1227a2ee..bb0952365b 100644 --- a/native/shuffle/src/bin/shuffle_bench.rs +++ b/native/shuffle/src/bin/shuffle_bench.rs @@ -115,6 +115,11 @@ struct Args { /// Limit rows processed per iteration (0 = no limit) #[arg(long, default_value_t = 0)] limit: usize, + + /// Number of concurrent shuffle tasks to simulate executor parallelism. + /// Each task reads the same input and writes to its own output files. + #[arg(long, default_value_t = 1)] + concurrent_tasks: usize, } fn main() { @@ -146,6 +151,9 @@ fn main() { if let Some(mem_limit) = args.memory_limit { println!("Memory limit: {}", format_bytes(mem_limit)); } + if args.concurrent_tasks > 1 { + println!("Concurrent: {} tasks", args.concurrent_tasks); + } println!( "Iterations: {} (warmup: {})", args.iterations, args.warmup @@ -165,15 +173,19 @@ fn main() { format!("iter {}/{}", i - args.warmup + 1, args.iterations) }; - let write_elapsed = run_shuffle_write( - &args.input, - &schema, - &codec, - &hash_col_indices, - &args, - data_file.to_str().unwrap(), - index_file.to_str().unwrap(), - ); + let write_elapsed = if args.concurrent_tasks > 1 { + run_concurrent_shuffle_writes(&args.input, &schema, &codec, &hash_col_indices, &args) + } else { + run_shuffle_write( + &args.input, + &schema, + &codec, + &hash_col_indices, + &args, + data_file.to_str().unwrap(), + index_file.to_str().unwrap(), + ) + }; let data_size = fs::metadata(&data_file).map(|m| m.len()).unwrap_or(0); if !is_warmup { @@ -182,9 +194,11 @@ fn main() { } print!(" [{label}] write: {:.3}s", write_elapsed); - print!(" output: {}", format_bytes(data_size as usize)); + if args.concurrent_tasks <= 1 { + print!(" output: {}", format_bytes(data_size as usize)); + } - if args.read_back { + if args.read_back && args.concurrent_tasks <= 1 { let read_elapsed = run_shuffle_read( data_file.to_str().unwrap(), index_file.to_str().unwrap(), @@ -203,8 +217,7 @@ fn main() { println!("=== Results ==="); let avg_write = write_times.iter().sum::() / write_times.len() as f64; - let avg_data_size = data_file_sizes.iter().sum::() / data_file_sizes.len() as u64; - let write_throughput_rows = total_rows as f64 / avg_write; + let write_throughput_rows = (total_rows as f64 * args.concurrent_tasks as f64) / avg_write; println!("Write:"); println!(" avg time: {:.3}s", avg_write); @@ -217,15 +230,20 @@ fn main() { println!(" min/max: {:.3}s / {:.3}s", min, max); } println!( - " throughput: {} rows/s", - format_number(write_throughput_rows as usize) - ); - println!( - " output size: {}", - format_bytes(avg_data_size as usize) + " throughput: {} rows/s (total across {} tasks)", + format_number(write_throughput_rows as usize), + args.concurrent_tasks ); + if args.concurrent_tasks <= 1 { + let avg_data_size = data_file_sizes.iter().sum::() / data_file_sizes.len() as u64; + println!( + " output size: {}", + format_bytes(avg_data_size as usize) + ); + } if !read_times.is_empty() { + let avg_data_size = data_file_sizes.iter().sum::() / data_file_sizes.len() as u64; let avg_read = read_times.iter().sum::() / read_times.len() as f64; let read_throughput_bytes = avg_data_size as f64 / avg_read; @@ -317,55 +335,144 @@ fn run_shuffle_write( let rt = tokio::runtime::Runtime::new().unwrap(); rt.block_on(async { - let config = SessionConfig::new().with_batch_size(args.batch_size); - let mut runtime_builder = RuntimeEnvBuilder::new(); - if let Some(mem_limit) = args.memory_limit { - runtime_builder = runtime_builder.with_memory_limit(mem_limit, 1.0); - } - let runtime_env = Arc::new(runtime_builder.build().unwrap()); - let ctx = SessionContext::new_with_config_rt(config, runtime_env); - - let path_str = input_path.to_str().unwrap(); - let mut df = ctx - .read_parquet(path_str, ParquetReadOptions::default()) - .await - .expect("Failed to create Parquet scan"); - if args.limit > 0 { - df = df.limit(0, Some(args.limit)).unwrap(); - } - - let parquet_plan = df - .create_physical_plan() - .await - .expect("Failed to create physical plan"); - - // ShuffleWriterExec reads from a single input partition - let input: Arc = if parquet_plan - .properties() - .output_partitioning() - .partition_count() - > 1 - { - Arc::new(CoalescePartitionsExec::new(parquet_plan)) - } else { - parquet_plan - }; - - let exec = ShuffleWriterExec::try_new( - input, - partitioning, + let start = Instant::now(); + execute_shuffle_write( + input_path.to_str().unwrap(), codec.clone(), + partitioning, + args.batch_size, + args.memory_limit, + args.write_buffer_size, + args.limit, data_file.to_string(), index_file.to_string(), - false, - args.write_buffer_size, ) - .expect("Failed to create ShuffleWriterExec"); + .await + .unwrap(); + start.elapsed().as_secs_f64() + }) +} + +/// Core async shuffle write logic shared by single and concurrent paths. +async fn execute_shuffle_write( + input_path: &str, + codec: CompressionCodec, + partitioning: CometPartitioning, + batch_size: usize, + memory_limit: Option, + write_buffer_size: usize, + limit: usize, + data_file: String, + index_file: String, +) -> datafusion::common::Result<()> { + let config = SessionConfig::new().with_batch_size(batch_size); + let mut runtime_builder = RuntimeEnvBuilder::new(); + if let Some(mem_limit) = memory_limit { + runtime_builder = runtime_builder.with_memory_limit(mem_limit, 1.0); + } + let runtime_env = Arc::new(runtime_builder.build().unwrap()); + let ctx = SessionContext::new_with_config_rt(config, runtime_env); + + let mut df = ctx + .read_parquet(input_path, ParquetReadOptions::default()) + .await + .expect("Failed to create Parquet scan"); + if limit > 0 { + df = df.limit(0, Some(limit)).unwrap(); + } + + let parquet_plan = df + .create_physical_plan() + .await + .expect("Failed to create physical plan"); + + let input: Arc = if parquet_plan + .properties() + .output_partitioning() + .partition_count() + > 1 + { + Arc::new(CoalescePartitionsExec::new(parquet_plan)) + } else { + parquet_plan + }; + + let exec = ShuffleWriterExec::try_new( + input, + partitioning, + codec, + data_file, + index_file, + false, + write_buffer_size, + ) + .expect("Failed to create ShuffleWriterExec"); + + let task_ctx = ctx.task_ctx(); + let stream = exec.execute(0, task_ctx).unwrap(); + collect(stream).await.unwrap(); + Ok(()) +} - let task_ctx = ctx.task_ctx(); +/// Run N concurrent shuffle writes to simulate executor parallelism. +/// Returns wall-clock time for all tasks to complete. +fn run_concurrent_shuffle_writes( + input_path: &PathBuf, + schema: &SchemaRef, + codec: &CompressionCodec, + hash_col_indices: &[usize], + args: &Args, +) -> f64 { + let rt = tokio::runtime::Runtime::new().unwrap(); + rt.block_on(async { let start = Instant::now(); - let stream = exec.execute(0, task_ctx).unwrap(); - collect(stream).await.unwrap(); + + let mut handles = Vec::with_capacity(args.concurrent_tasks); + for task_id in 0..args.concurrent_tasks { + let task_dir = args.output_dir.join(format!("task_{task_id}")); + fs::create_dir_all(&task_dir).expect("Failed to create task output directory"); + let data_file = task_dir.join("data.out").to_str().unwrap().to_string(); + let index_file = task_dir.join("index.out").to_str().unwrap().to_string(); + + let input_str = input_path.to_str().unwrap().to_string(); + let codec = codec.clone(); + let partitioning = build_partitioning( + &args.partitioning, + args.partitions, + hash_col_indices, + schema, + ); + let batch_size = args.batch_size; + let memory_limit = args.memory_limit; + let write_buffer_size = args.write_buffer_size; + let limit = args.limit; + + handles.push(tokio::spawn(async move { + execute_shuffle_write( + &input_str, + codec, + partitioning, + batch_size, + memory_limit, + write_buffer_size, + limit, + data_file, + index_file, + ) + .await + .unwrap(); + })); + } + + for handle in handles { + handle.await.expect("Task panicked"); + } + + for task_id in 0..args.concurrent_tasks { + let task_dir = args.output_dir.join(format!("task_{task_id}")); + let _ = fs::remove_dir_all(&task_dir); + } + start.elapsed().as_secs_f64() }) } From 58ab9270669d4ddf27d7f935bd1cec408280f54d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 30 Mar 2026 08:42:23 -0600 Subject: [PATCH 10/46] show metrics --- native/shuffle/src/bin/shuffle_bench.rs | 104 ++++++++++++++++++++---- 1 file changed, 90 insertions(+), 14 deletions(-) diff --git a/native/shuffle/src/bin/shuffle_bench.rs b/native/shuffle/src/bin/shuffle_bench.rs index bb0952365b..bf476817f5 100644 --- a/native/shuffle/src/bin/shuffle_bench.rs +++ b/native/shuffle/src/bin/shuffle_bench.rs @@ -43,6 +43,7 @@ use datafusion::execution::runtime_env::RuntimeEnvBuilder; use datafusion::physical_expr::expressions::Column; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::common::collect; +use datafusion::physical_plan::metrics::MetricsSet; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::{ParquetReadOptions, SessionContext}; use datafusion_comet_shuffle::{ @@ -50,7 +51,7 @@ use datafusion_comet_shuffle::{ }; use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; use std::fs; -use std::path::PathBuf; +use std::path::{Path, PathBuf}; use std::sync::Arc; use std::time::Instant; @@ -164,6 +165,7 @@ fn main() { let mut write_times = Vec::with_capacity(args.iterations); let mut read_times = Vec::with_capacity(args.iterations); let mut data_file_sizes = Vec::with_capacity(args.iterations); + let mut last_metrics: Option = None; for i in 0..total_iters { let is_warmup = i < args.warmup; @@ -173,8 +175,15 @@ fn main() { format!("iter {}/{}", i - args.warmup + 1, args.iterations) }; - let write_elapsed = if args.concurrent_tasks > 1 { - run_concurrent_shuffle_writes(&args.input, &schema, &codec, &hash_col_indices, &args) + let (write_elapsed, metrics) = if args.concurrent_tasks > 1 { + let elapsed = run_concurrent_shuffle_writes( + &args.input, + &schema, + &codec, + &hash_col_indices, + &args, + ); + (elapsed, None) } else { run_shuffle_write( &args.input, @@ -191,6 +200,7 @@ fn main() { if !is_warmup { write_times.push(write_elapsed); data_file_sizes.push(data_size); + last_metrics = metrics; } print!(" [{label}] write: {:.3}s", write_elapsed); @@ -259,14 +269,79 @@ fn main() { format_bytes(read_throughput_bytes as usize) ); } + + if let Some(ref metrics) = last_metrics { + println!(); + println!("Shuffle Metrics (last iteration):"); + print_shuffle_metrics(metrics, avg_write); + } } let _ = fs::remove_file(&data_file); let _ = fs::remove_file(&index_file); } +fn print_shuffle_metrics(metrics: &MetricsSet, total_wall_time_secs: f64) { + let get_metric = |name: &str| -> Option { + metrics + .iter() + .find(|m| m.value().name() == name) + .map(|m| m.value().as_usize()) + }; + + let total_ns = (total_wall_time_secs * 1e9) as u64; + let fmt_time = |nanos: usize| -> String { + let secs = nanos as f64 / 1e9; + let pct = if total_ns > 0 { + (nanos as f64 / total_ns as f64) * 100.0 + } else { + 0.0 + }; + format!("{:.3}s ({:.1}%)", secs, pct) + }; + + if let Some(input_batches) = get_metric("input_batches") { + println!(" input batches: {}", format_number(input_batches)); + } + if let Some(nanos) = get_metric("repart_time") { + println!(" repart time: {}", fmt_time(nanos)); + } + if let Some(nanos) = get_metric("encode_time") { + println!(" encode time: {}", fmt_time(nanos)); + } + if let Some(nanos) = get_metric("write_time") { + println!(" write time: {}", fmt_time(nanos)); + } + if let (Some(repart), Some(encode), Some(write)) = ( + get_metric("repart_time"), + get_metric("encode_time"), + get_metric("write_time"), + ) { + let accounted = repart + encode + write; + if total_ns as usize > accounted { + let other = total_ns as usize - accounted; + println!(" other time: {}", fmt_time(other)); + } + } + if let Some(spill_count) = get_metric("spill_count") { + if spill_count > 0 { + println!(" spill count: {}", format_number(spill_count)); + } + } + if let Some(spilled_bytes) = get_metric("spilled_bytes") { + if spilled_bytes > 0 { + println!(" spilled bytes: {}", format_bytes(spilled_bytes)); + } + } + if let Some(data_size) = get_metric("data_size") { + if data_size > 0 { + println!(" data size: {}", format_bytes(data_size)); + } + } +} + /// Read schema and total row count from Parquet metadata without loading any data. -fn read_parquet_metadata(path: &PathBuf, limit: usize) -> (SchemaRef, u64) { +fn read_parquet_metadata(path: &Path, limit: usize) -> (SchemaRef, u64) { let paths = collect_parquet_paths(path); let mut schema = None; let mut total_rows = 0u64; @@ -294,7 +369,7 @@ fn read_parquet_metadata(path: &PathBuf, limit: usize) -> (SchemaRef, u64) { (schema.expect("No parquet files found"), total_rows) } -fn collect_parquet_paths(path: &PathBuf) -> Vec { +fn collect_parquet_paths(path: &Path) -> Vec { if path.is_dir() { let mut files: Vec = fs::read_dir(path) .unwrap_or_else(|e| panic!("Failed to read directory {}: {}", path.display(), e)) @@ -313,19 +388,19 @@ fn collect_parquet_paths(path: &PathBuf) -> Vec { } files } else { - vec![path.clone()] + vec![path.to_path_buf()] } } fn run_shuffle_write( - input_path: &PathBuf, + input_path: &Path, schema: &SchemaRef, codec: &CompressionCodec, hash_col_indices: &[usize], args: &Args, data_file: &str, index_file: &str, -) -> f64 { +) -> (f64, Option) { let partitioning = build_partitioning( &args.partitioning, args.partitions, @@ -336,7 +411,7 @@ fn run_shuffle_write( let rt = tokio::runtime::Runtime::new().unwrap(); rt.block_on(async { let start = Instant::now(); - execute_shuffle_write( + let metrics = execute_shuffle_write( input_path.to_str().unwrap(), codec.clone(), partitioning, @@ -349,11 +424,12 @@ fn run_shuffle_write( ) .await .unwrap(); - start.elapsed().as_secs_f64() + (start.elapsed().as_secs_f64(), Some(metrics)) }) } /// Core async shuffle write logic shared by single and concurrent paths. +#[allow(clippy::too_many_arguments)] async fn execute_shuffle_write( input_path: &str, codec: CompressionCodec, @@ -364,7 +440,7 @@ async fn execute_shuffle_write( limit: usize, data_file: String, index_file: String, -) -> datafusion::common::Result<()> { +) -> datafusion::common::Result { let config = SessionConfig::new().with_batch_size(batch_size); let mut runtime_builder = RuntimeEnvBuilder::new(); if let Some(mem_limit) = memory_limit { @@ -411,13 +487,13 @@ async fn execute_shuffle_write( let task_ctx = ctx.task_ctx(); let stream = exec.execute(0, task_ctx).unwrap(); collect(stream).await.unwrap(); - Ok(()) + Ok(exec.metrics().unwrap_or_default()) } /// Run N concurrent shuffle writes to simulate executor parallelism. /// Returns wall-clock time for all tasks to complete. fn run_concurrent_shuffle_writes( - input_path: &PathBuf, + input_path: &Path, schema: &SchemaRef, codec: &CompressionCodec, hash_col_indices: &[usize], @@ -460,7 +536,7 @@ fn run_concurrent_shuffle_writes( index_file, ) .await - .unwrap(); + .unwrap() })); } From c469077e49fa42b37ba77c987497a1a5e789fb8a Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 30 Mar 2026 09:02:01 -0600 Subject: [PATCH 11/46] improve metrics --- native/shuffle/src/bin/shuffle_bench.rs | 101 ++++++++++++++---- native/shuffle/src/metrics.rs | 12 +++ .../src/partitioners/multi_partition.rs | 15 ++- .../partitioned_batch_iterator.rs | 17 ++- .../src/partitioners/single_partition.rs | 10 +- native/shuffle/src/shuffle_writer.rs | 17 ++- .../shuffle/src/writers/buf_batch_writer.rs | 6 ++ native/shuffle/src/writers/spill.rs | 15 ++- 8 files changed, 160 insertions(+), 33 deletions(-) diff --git a/native/shuffle/src/bin/shuffle_bench.rs b/native/shuffle/src/bin/shuffle_bench.rs index bf476817f5..a412d11eed 100644 --- a/native/shuffle/src/bin/shuffle_bench.rs +++ b/native/shuffle/src/bin/shuffle_bench.rs @@ -43,7 +43,7 @@ use datafusion::execution::runtime_env::RuntimeEnvBuilder; use datafusion::physical_expr::expressions::Column; use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion::physical_plan::common::collect; -use datafusion::physical_plan::metrics::MetricsSet; +use datafusion::physical_plan::metrics::{MetricValue, MetricsSet}; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::{ParquetReadOptions, SessionContext}; use datafusion_comet_shuffle::{ @@ -166,6 +166,7 @@ fn main() { let mut read_times = Vec::with_capacity(args.iterations); let mut data_file_sizes = Vec::with_capacity(args.iterations); let mut last_metrics: Option = None; + let mut last_input_metrics: Option = None; for i in 0..total_iters { let is_warmup = i < args.warmup; @@ -175,7 +176,7 @@ fn main() { format!("iter {}/{}", i - args.warmup + 1, args.iterations) }; - let (write_elapsed, metrics) = if args.concurrent_tasks > 1 { + let (write_elapsed, metrics, input_metrics) = if args.concurrent_tasks > 1 { let elapsed = run_concurrent_shuffle_writes( &args.input, &schema, @@ -183,7 +184,7 @@ fn main() { &hash_col_indices, &args, ); - (elapsed, None) + (elapsed, None, None) } else { run_shuffle_write( &args.input, @@ -201,6 +202,7 @@ fn main() { write_times.push(write_elapsed); data_file_sizes.push(data_size); last_metrics = metrics; + last_input_metrics = input_metrics; } print!(" [{label}] write: {:.3}s", write_elapsed); @@ -270,6 +272,12 @@ fn main() { ); } + if let Some(ref metrics) = last_input_metrics { + println!(); + println!("Input Metrics (last iteration):"); + print_input_metrics(metrics); + } + if let Some(ref metrics) = last_metrics { println!(); println!("Shuffle Metrics (last iteration):"); @@ -312,17 +320,16 @@ fn print_shuffle_metrics(metrics: &MetricsSet, total_wall_time_secs: f64) { if let Some(nanos) = get_metric("write_time") { println!(" write time: {}", fmt_time(nanos)); } - if let (Some(repart), Some(encode), Some(write)) = ( - get_metric("repart_time"), - get_metric("encode_time"), - get_metric("write_time"), - ) { - let accounted = repart + encode + write; - if total_ns as usize > accounted { - let other = total_ns as usize - accounted; - println!(" other time: {}", fmt_time(other)); - } + if let Some(nanos) = get_metric("interleave_time") { + println!(" interleave time: {}", fmt_time(nanos)); } + if let Some(nanos) = get_metric("coalesce_time") { + println!(" coalesce time: {}", fmt_time(nanos)); + } + if let Some(nanos) = get_metric("memcopy_time") { + println!(" memcopy time: {}", fmt_time(nanos)); + } + if let Some(spill_count) = get_metric("spill_count") { if spill_count > 0 { println!(" spill count: {}", format_number(spill_count)); @@ -340,6 +347,37 @@ fn print_shuffle_metrics(metrics: &MetricsSet, total_wall_time_secs: f64) { } } +fn print_input_metrics(metrics: &MetricsSet) { + let aggregated = metrics.aggregate_by_name(); + for m in aggregated.iter() { + let value = m.value(); + let name = value.name(); + let v = value.as_usize(); + if v == 0 { + continue; + } + // Format time metrics as seconds, everything else as a number + // Skip start/end timestamps — not useful in benchmark output + if matches!( + value, + MetricValue::StartTimestamp(_) | MetricValue::EndTimestamp(_) + ) { + continue; + } + let is_time = matches!( + value, + MetricValue::ElapsedCompute(_) | MetricValue::Time { .. } + ); + if is_time { + println!(" {name}: {:.3}s", v as f64 / 1e9); + } else if name.contains("bytes") || name.contains("size") { + println!(" {name}: {}", format_bytes(v)); + } else { + println!(" {name}: {}", format_number(v)); + } + } +} + /// Read schema and total row count from Parquet metadata without loading any data. fn read_parquet_metadata(path: &Path, limit: usize) -> (SchemaRef, u64) { let paths = collect_parquet_paths(path); @@ -400,7 +438,7 @@ fn run_shuffle_write( args: &Args, data_file: &str, index_file: &str, -) -> (f64, Option) { +) -> (f64, Option, Option) { let partitioning = build_partitioning( &args.partitioning, args.partitions, @@ -411,7 +449,7 @@ fn run_shuffle_write( let rt = tokio::runtime::Runtime::new().unwrap(); rt.block_on(async { let start = Instant::now(); - let metrics = execute_shuffle_write( + let (shuffle_metrics, input_metrics) = execute_shuffle_write( input_path.to_str().unwrap(), codec.clone(), partitioning, @@ -424,7 +462,11 @@ fn run_shuffle_write( ) .await .unwrap(); - (start.elapsed().as_secs_f64(), Some(metrics)) + ( + start.elapsed().as_secs_f64(), + Some(shuffle_metrics), + Some(input_metrics), + ) }) } @@ -440,7 +482,7 @@ async fn execute_shuffle_write( limit: usize, data_file: String, index_file: String, -) -> datafusion::common::Result { +) -> datafusion::common::Result<(MetricsSet, MetricsSet)> { let config = SessionConfig::new().with_batch_size(batch_size); let mut runtime_builder = RuntimeEnvBuilder::new(); if let Some(mem_limit) = memory_limit { @@ -487,7 +529,30 @@ async fn execute_shuffle_write( let task_ctx = ctx.task_ctx(); let stream = exec.execute(0, task_ctx).unwrap(); collect(stream).await.unwrap(); - Ok(exec.metrics().unwrap_or_default()) + + // Collect metrics from the input plan (Parquet scan + optional coalesce) + let input_metrics = collect_input_metrics(&exec); + + Ok((exec.metrics().unwrap_or_default(), input_metrics)) +} + +/// Walk the plan tree and aggregate all metrics from input plans (everything below shuffle writer). +fn collect_input_metrics(exec: &ShuffleWriterExec) -> MetricsSet { + let mut all_metrics = MetricsSet::new(); + fn gather(plan: &dyn ExecutionPlan, out: &mut MetricsSet) { + if let Some(metrics) = plan.metrics() { + for m in metrics.iter() { + out.push(Arc::clone(m)); + } + } + for child in plan.children() { + gather(child.as_ref(), out); + } + } + for child in exec.children() { + gather(child.as_ref(), &mut all_metrics); + } + all_metrics } /// Run N concurrent shuffle writes to simulate executor parallelism. diff --git a/native/shuffle/src/metrics.rs b/native/shuffle/src/metrics.rs index 1de751cf41..8755e2c65f 100644 --- a/native/shuffle/src/metrics.rs +++ b/native/shuffle/src/metrics.rs @@ -33,6 +33,15 @@ pub(crate) struct ShufflePartitionerMetrics { /// Time spent writing to disk. Maps to "shuffleWriteTime" in Spark SQL Metrics. pub(crate) write_time: Time, + /// Time spent in interleave_record_batch gathering rows into shuffled batches + pub(crate) interleave_time: Time, + + /// Time spent coalescing small batches before serialization + pub(crate) coalesce_time: Time, + + /// Time spent buffering partition indices and memory accounting + pub(crate) memcopy_time: Time, + /// Number of input batches pub(crate) input_batches: Count, @@ -53,6 +62,9 @@ impl ShufflePartitionerMetrics { repart_time: MetricBuilder::new(metrics).subset_time("repart_time", partition), encode_time: MetricBuilder::new(metrics).subset_time("encode_time", partition), write_time: MetricBuilder::new(metrics).subset_time("write_time", partition), + interleave_time: MetricBuilder::new(metrics).subset_time("interleave_time", partition), + coalesce_time: MetricBuilder::new(metrics).subset_time("coalesce_time", partition), + memcopy_time: MetricBuilder::new(metrics).subset_time("memcopy_time", partition), input_batches: MetricBuilder::new(metrics).counter("input_batches", partition), spill_count: MetricBuilder::new(metrics).spill_count(partition), spilled_bytes: MetricBuilder::new(metrics).spilled_bytes(partition), diff --git a/native/shuffle/src/partitioners/multi_partition.rs b/native/shuffle/src/partitioners/multi_partition.rs index 655bee3511..819c569064 100644 --- a/native/shuffle/src/partitioners/multi_partition.rs +++ b/native/shuffle/src/partitioners/multi_partition.rs @@ -398,6 +398,7 @@ impl MultiPartitionShuffleRepartitioner { partition_row_indices: &[u32], partition_starts: &[u32], ) -> datafusion::common::Result<()> { + let start_time = Instant::now(); let mut mem_growth: usize = input.get_array_memory_size(); let buffered_partition_idx = self.buffered_batches.len() as u32; self.buffered_batches.push(input); @@ -426,6 +427,7 @@ impl MultiPartitionShuffleRepartitioner { let after_size = indices.allocated_size(); mem_growth += after_size.saturating_sub(before_size); } + self.metrics.memcopy_time.add_duration(start_time.elapsed()); if self.reservation.try_grow(mem_growth).is_err() { self.spill()?; @@ -434,12 +436,14 @@ impl MultiPartitionShuffleRepartitioner { Ok(()) } + #[allow(clippy::too_many_arguments)] fn shuffle_write_partition( partition_iter: &mut PartitionedBatchIterator, shuffle_block_writer: &mut ShuffleBlockWriter, output_data: &mut BufWriter, encode_time: &Time, write_time: &Time, + coalesce_time: &Time, write_buffer_size: usize, batch_size: usize, ) -> datafusion::common::Result<()> { @@ -451,9 +455,9 @@ impl MultiPartitionShuffleRepartitioner { ); for batch in partition_iter { let batch = batch?; - buf_batch_writer.write(&batch, encode_time, write_time)?; + buf_batch_writer.write(&batch, encode_time, write_time, coalesce_time)?; } - buf_batch_writer.flush(encode_time, write_time)?; + buf_batch_writer.flush(encode_time, write_time, coalesce_time)?; Ok(()) } @@ -506,7 +510,8 @@ impl MultiPartitionShuffleRepartitioner { for partition_id in 0..num_output_partitions { let partition_writer = &mut self.partition_writers[partition_id]; - let mut iter = partitioned_batches.produce(partition_id); + let mut iter = + partitioned_batches.produce(partition_id, &self.metrics.interleave_time); spilled_bytes += partition_writer.spill( &mut iter, &self.runtime, @@ -589,13 +594,15 @@ impl ShufflePartitioner for MultiPartitionShuffleRepartitioner { } // Write in memory batches to output data file - let mut partition_iter = partitioned_batches.produce(i); + let mut partition_iter = + partitioned_batches.produce(i, &self.metrics.interleave_time); Self::shuffle_write_partition( &mut partition_iter, &mut self.shuffle_block_writer, &mut output_data, &self.metrics.encode_time, &self.metrics.write_time, + &self.metrics.coalesce_time, self.write_buffer_size, self.batch_size, )?; diff --git a/native/shuffle/src/partitioners/partitioned_batch_iterator.rs b/native/shuffle/src/partitioners/partitioned_batch_iterator.rs index 8309a8ed4a..f124d98ff2 100644 --- a/native/shuffle/src/partitioners/partitioned_batch_iterator.rs +++ b/native/shuffle/src/partitioners/partitioned_batch_iterator.rs @@ -18,6 +18,7 @@ use arrow::array::RecordBatch; use arrow::compute::interleave_record_batch; use datafusion::common::DataFusionError; +use datafusion::physical_plan::metrics::Time; /// A helper struct to produce shuffled batches. /// This struct takes ownership of the buffered batches and partition indices from the @@ -41,11 +42,16 @@ impl PartitionedBatchesProducer { } } - pub(super) fn produce(&mut self, partition_id: usize) -> PartitionedBatchIterator<'_> { + pub(super) fn produce<'a>( + &'a mut self, + partition_id: usize, + interleave_time: &'a Time, + ) -> PartitionedBatchIterator<'a> { PartitionedBatchIterator::new( &self.partition_indices[partition_id], &self.buffered_batches, self.batch_size, + interleave_time, ) } } @@ -56,6 +62,7 @@ pub(crate) struct PartitionedBatchIterator<'a> { batch_size: usize, indices: Vec<(usize, usize)>, pos: usize, + interleave_time: &'a Time, } impl<'a> PartitionedBatchIterator<'a> { @@ -63,6 +70,7 @@ impl<'a> PartitionedBatchIterator<'a> { indices: &'a [(u32, u32)], buffered_batches: &'a [RecordBatch], batch_size: usize, + interleave_time: &'a Time, ) -> Self { if indices.is_empty() { // Avoid unnecessary allocations when the partition is empty @@ -71,6 +79,7 @@ impl<'a> PartitionedBatchIterator<'a> { batch_size, indices: vec![], pos: 0, + interleave_time, }; } let record_batches = buffered_batches.iter().collect::>(); @@ -83,6 +92,7 @@ impl<'a> PartitionedBatchIterator<'a> { batch_size, indices: current_indices, pos: 0, + interleave_time, } } } @@ -97,7 +107,10 @@ impl Iterator for PartitionedBatchIterator<'_> { let indices_end = std::cmp::min(self.pos + self.batch_size, self.indices.len()); let indices = &self.indices[self.pos..indices_end]; - match interleave_record_batch(&self.record_batches, indices) { + let mut timer = self.interleave_time.timer(); + let result = interleave_record_batch(&self.record_batches, indices); + timer.stop(); + match result { Ok(batch) => { self.pos = indices_end; Some(Ok(batch)) diff --git a/native/shuffle/src/partitioners/single_partition.rs b/native/shuffle/src/partitioners/single_partition.rs index 5801ef613b..d866aae330 100644 --- a/native/shuffle/src/partitioners/single_partition.rs +++ b/native/shuffle/src/partitioners/single_partition.rs @@ -127,6 +127,7 @@ impl ShufflePartitioner for SinglePartitionShufflePartitioner { &batch, &self.metrics.encode_time, &self.metrics.write_time, + &self.metrics.coalesce_time, )?; } @@ -136,6 +137,7 @@ impl ShufflePartitioner for SinglePartitionShufflePartitioner { &batch, &self.metrics.encode_time, &self.metrics.write_time, + &self.metrics.coalesce_time, )?; } else { // Add the new batch to the buffer @@ -164,10 +166,14 @@ impl ShufflePartitioner for SinglePartitionShufflePartitioner { &batch, &self.metrics.encode_time, &self.metrics.write_time, + &self.metrics.coalesce_time, )?; } - self.output_data_writer - .flush(&self.metrics.encode_time, &self.metrics.write_time)?; + self.output_data_writer.flush( + &self.metrics.encode_time, + &self.metrics.write_time, + &self.metrics.coalesce_time, + )?; // Write index file. It should only contain 2 entries: 0 and the total number of bytes written let index_file = OpenOptions::new() diff --git a/native/shuffle/src/shuffle_writer.rs b/native/shuffle/src/shuffle_writer.rs index e649aaac69..03c9d3ff23 100644 --- a/native/shuffle/src/shuffle_writer.rs +++ b/native/shuffle/src/shuffle_writer.rs @@ -618,6 +618,7 @@ mod test { let codec = CompressionCodec::Lz4Frame; let encode_time = Time::default(); let write_time = Time::default(); + let coalesce_time = Time::default(); // Write with coalescing (batch_size=8192) let mut coalesced_output = Vec::new(); @@ -630,9 +631,13 @@ mod test { 8192, ); for batch in &small_batches { - buf_writer.write(batch, &encode_time, &write_time).unwrap(); + buf_writer + .write(batch, &encode_time, &write_time, &coalesce_time) + .unwrap(); } - buf_writer.flush(&encode_time, &write_time).unwrap(); + buf_writer + .flush(&encode_time, &write_time, &coalesce_time) + .unwrap(); } // Write without coalescing (batch_size=1) @@ -646,9 +651,13 @@ mod test { 1, ); for batch in &small_batches { - buf_writer.write(batch, &encode_time, &write_time).unwrap(); + buf_writer + .write(batch, &encode_time, &write_time, &coalesce_time) + .unwrap(); } - buf_writer.flush(&encode_time, &write_time).unwrap(); + buf_writer + .flush(&encode_time, &write_time, &coalesce_time) + .unwrap(); } // Coalesced output should be smaller due to fewer IPC schema blocks diff --git a/native/shuffle/src/writers/buf_batch_writer.rs b/native/shuffle/src/writers/buf_batch_writer.rs index cfddb46539..f6bb15137e 100644 --- a/native/shuffle/src/writers/buf_batch_writer.rs +++ b/native/shuffle/src/writers/buf_batch_writer.rs @@ -63,7 +63,9 @@ impl, W: Write> BufBatchWriter { batch: &RecordBatch, encode_time: &Time, write_time: &Time, + coalesce_time: &Time, ) -> datafusion::common::Result { + let mut coalesce_timer = coalesce_time.timer(); let coalescer = self .coalescer .get_or_insert_with(|| BatchCoalescer::new(batch.schema(), self.batch_size)); @@ -75,6 +77,7 @@ impl, W: Write> BufBatchWriter { while let Some(batch) = coalescer.next_completed_batch() { completed.push(batch); } + coalesce_timer.stop(); let mut bytes_written = 0; for batch in &completed { @@ -110,8 +113,10 @@ impl, W: Write> BufBatchWriter { &mut self, encode_time: &Time, write_time: &Time, + coalesce_time: &Time, ) -> datafusion::common::Result<()> { // Finish any remaining buffered rows in the coalescer + let mut coalesce_timer = coalesce_time.timer(); let mut remaining = Vec::new(); if let Some(coalescer) = &mut self.coalescer { coalescer.finish_buffered_batch()?; @@ -119,6 +124,7 @@ impl, W: Write> BufBatchWriter { remaining.push(batch); } } + coalesce_timer.stop(); for batch in &remaining { self.write_batch_to_buffer(batch, encode_time, write_time)?; } diff --git a/native/shuffle/src/writers/spill.rs b/native/shuffle/src/writers/spill.rs index c16caddbf9..05492791d1 100644 --- a/native/shuffle/src/writers/spill.rs +++ b/native/shuffle/src/writers/spill.rs @@ -93,17 +93,26 @@ impl PartitionWriter { write_buffer_size, batch_size, ); - let mut bytes_written = - buf_batch_writer.write(&batch?, &metrics.encode_time, &metrics.write_time)?; + let mut bytes_written = buf_batch_writer.write( + &batch?, + &metrics.encode_time, + &metrics.write_time, + &metrics.coalesce_time, + )?; for batch in iter { let batch = batch?; bytes_written += buf_batch_writer.write( &batch, &metrics.encode_time, &metrics.write_time, + &metrics.coalesce_time, )?; } - buf_batch_writer.flush(&metrics.encode_time, &metrics.write_time)?; + buf_batch_writer.flush( + &metrics.encode_time, + &metrics.write_time, + &metrics.coalesce_time, + )?; bytes_written }; From 4ed08c5ba74fbc2f07164e4a038d7fdffed6540b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 30 Mar 2026 15:09:25 -0600 Subject: [PATCH 12/46] feat: add PartitionOutputStream with IPC serialization and compression --- .../src/partitioners/immediate_mode.rs | 265 ++++++++++++++++++ native/shuffle/src/partitioners/mod.rs | 1 + 2 files changed, 266 insertions(+) create mode 100644 native/shuffle/src/partitioners/immediate_mode.rs diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs new file mode 100644 index 0000000000..849d454966 --- /dev/null +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -0,0 +1,265 @@ +// 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 arrow::array::RecordBatch; +use arrow::datatypes::SchemaRef; +use arrow::ipc::writer::StreamWriter; +use datafusion::common::DataFusionError; +use datafusion::error::Result; +use crate::CompressionCodec; + +/// Per-partition output stream that serializes Arrow IPC batches into an +/// in-memory buffer with compression. The block format matches +/// `ShuffleBlockWriter::write_batch` exactly: +/// +/// - 8 bytes: payload length (u64 LE) — total bytes after this prefix +/// - 8 bytes: field_count (usize LE) +/// - 4 bytes: codec tag (b"SNAP", b"LZ4_", b"ZSTD", or b"NONE") +/// - N bytes: compressed Arrow IPC stream data +#[allow(dead_code)] +pub struct PartitionOutputStream { + schema: SchemaRef, + codec: CompressionCodec, + buffer: Vec, +} + +#[allow(dead_code)] +impl PartitionOutputStream { + pub fn try_new(schema: SchemaRef, codec: CompressionCodec) -> Result { + Ok(Self { + schema, + codec, + buffer: Vec::with_capacity(1024 * 1024), + }) + } + + /// Writes a record batch as a length-prefixed compressed IPC block. + /// Returns the number of bytes written to the buffer. + pub fn write_batch(&mut self, batch: &RecordBatch) -> Result { + if batch.num_rows() == 0 { + return Ok(0); + } + + let start_pos = self.buffer.len(); + + // Write 8-byte placeholder for length prefix + self.buffer.extend_from_slice(&0u64.to_le_bytes()); + + // Write field count (8 bytes, usize LE) + let field_count = self.schema.fields().len(); + self.buffer + .extend_from_slice(&(field_count as u64).to_le_bytes()); + + // Write codec tag (4 bytes) + let codec_tag: &[u8; 4] = match &self.codec { + CompressionCodec::Snappy => b"SNAP", + CompressionCodec::Lz4Frame => b"LZ4_", + CompressionCodec::Zstd(_) => b"ZSTD", + CompressionCodec::None => b"NONE", + }; + self.buffer.extend_from_slice(codec_tag); + + // Write compressed IPC data + match &self.codec { + CompressionCodec::None => { + let mut arrow_writer = StreamWriter::try_new(&mut self.buffer, &batch.schema())?; + arrow_writer.write(batch)?; + arrow_writer.finish()?; + // StreamWriter::into_inner returns the inner writer; we don't need it + // since we're writing directly to self.buffer + arrow_writer.into_inner()?; + } + CompressionCodec::Lz4Frame => { + let mut wtr = lz4_flex::frame::FrameEncoder::new(&mut self.buffer); + let mut arrow_writer = StreamWriter::try_new(&mut wtr, &batch.schema())?; + arrow_writer.write(batch)?; + arrow_writer.finish()?; + wtr.finish().map_err(|e| { + DataFusionError::Execution(format!("lz4 compression error: {e}")) + })?; + } + CompressionCodec::Zstd(level) => { + let encoder = zstd::Encoder::new(&mut self.buffer, *level)?; + let mut arrow_writer = StreamWriter::try_new(encoder, &batch.schema())?; + arrow_writer.write(batch)?; + arrow_writer.finish()?; + let zstd_encoder = arrow_writer.into_inner()?; + zstd_encoder.finish()?; + } + CompressionCodec::Snappy => { + let mut wtr = snap::write::FrameEncoder::new(&mut self.buffer); + let mut arrow_writer = StreamWriter::try_new(&mut wtr, &batch.schema())?; + arrow_writer.write(batch)?; + arrow_writer.finish()?; + wtr.into_inner().map_err(|e| { + DataFusionError::Execution(format!("snappy compression error: {e}")) + })?; + } + } + + // Backfill length prefix: total bytes after the 8-byte length field + let end_pos = self.buffer.len(); + let ipc_length = (end_pos - start_pos - 8) as u64; + + let max_size = i32::MAX as u64; + if ipc_length > max_size { + return Err(DataFusionError::Execution(format!( + "Shuffle block size {ipc_length} exceeds maximum size of {max_size}. \ + Try reducing batch size or increasing compression level" + ))); + } + + self.buffer[start_pos..start_pos + 8].copy_from_slice(&ipc_length.to_le_bytes()); + + Ok(end_pos - start_pos) + } + + /// Returns the number of bytes currently in the buffer. + pub fn buffered_bytes(&self) -> usize { + self.buffer.len() + } + + /// Takes the buffer contents, leaving the buffer empty. + pub fn drain_buffer(&mut self) -> Vec { + std::mem::take(&mut self.buffer) + } + + /// Consumes self and returns the buffer. + pub fn finish(self) -> Result> { + Ok(self.buffer) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::read_ipc_compressed; + use arrow::array::Int32Array; + use arrow::datatypes::{DataType, Field, Schema}; + use std::sync::Arc; + + fn make_test_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_partition_output_stream_write_and_read() { + let batch = make_test_batch(&[1, 2, 3, 4, 5]); + let schema = batch.schema(); + + for codec in [ + CompressionCodec::None, + CompressionCodec::Lz4Frame, + CompressionCodec::Zstd(1), + CompressionCodec::Snappy, + ] { + let mut stream = + PartitionOutputStream::try_new(Arc::clone(&schema), codec).unwrap(); + let bytes_written = stream.write_batch(&batch).unwrap(); + assert!(bytes_written > 0); + assert_eq!(stream.buffered_bytes(), bytes_written); + + let buf = stream.finish().unwrap(); + assert_eq!(buf.len(), bytes_written); + + // Parse the block: 8 bytes length, 8 bytes field_count, then codec+data + let ipc_length = + u64::from_le_bytes(buf[0..8].try_into().unwrap()) as usize; + assert_eq!(ipc_length, buf.len() - 8); + + let field_count = + usize::from_le_bytes(buf[8..16].try_into().unwrap()); + assert_eq!(field_count, 1); // one field "a" + + // read_ipc_compressed expects data starting at the codec tag + let ipc_data = &buf[16..]; + let batch2 = read_ipc_compressed(ipc_data).unwrap(); + assert_eq!(batch, batch2); + } + } + + #[test] + fn test_partition_output_stream_multiple_batches() { + let batch1 = make_test_batch(&[1, 2, 3]); + let batch2 = make_test_batch(&[4, 5, 6, 7]); + let schema = batch1.schema(); + + let mut stream = + PartitionOutputStream::try_new(schema, CompressionCodec::None).unwrap(); + + let bytes1 = stream.write_batch(&batch1).unwrap(); + assert!(bytes1 > 0); + + let bytes2 = stream.write_batch(&batch2).unwrap(); + assert!(bytes2 > 0); + + assert_eq!(stream.buffered_bytes(), bytes1 + bytes2); + + let buf = stream.finish().unwrap(); + + // Read first block + let len1 = u64::from_le_bytes(buf[0..8].try_into().unwrap()) as usize; + let block1_end = 8 + len1; + let ipc_data1 = &buf[16..block1_end]; + let decoded1 = read_ipc_compressed(ipc_data1).unwrap(); + assert_eq!(batch1, decoded1); + + // Read second block + let len2 = u64::from_le_bytes( + buf[block1_end..block1_end + 8].try_into().unwrap(), + ) as usize; + let block2_end = block1_end + 8 + len2; + let ipc_data2 = &buf[block1_end + 16..block2_end]; + let decoded2 = read_ipc_compressed(ipc_data2).unwrap(); + assert_eq!(batch2, decoded2); + + assert_eq!(block2_end, buf.len()); + } + + #[test] + fn test_partition_output_stream_empty_batch() { + let batch = make_test_batch(&[]); + let schema = batch.schema(); + + let mut stream = + PartitionOutputStream::try_new(schema, CompressionCodec::None).unwrap(); + let bytes_written = stream.write_batch(&batch).unwrap(); + assert_eq!(bytes_written, 0); + assert_eq!(stream.buffered_bytes(), 0); + } + + #[test] + fn test_partition_output_stream_drain_buffer() { + let batch = make_test_batch(&[1, 2, 3]); + let schema = batch.schema(); + + let mut stream = + PartitionOutputStream::try_new(schema, CompressionCodec::None).unwrap(); + stream.write_batch(&batch).unwrap(); + assert!(stream.buffered_bytes() > 0); + + let drained = stream.drain_buffer(); + assert!(!drained.is_empty()); + assert_eq!(stream.buffered_bytes(), 0); + + // Can still write after drain + stream.write_batch(&batch).unwrap(); + assert!(stream.buffered_bytes() > 0); + } +} diff --git a/native/shuffle/src/partitioners/mod.rs b/native/shuffle/src/partitioners/mod.rs index 3eedef62c7..c965f84377 100644 --- a/native/shuffle/src/partitioners/mod.rs +++ b/native/shuffle/src/partitioners/mod.rs @@ -19,6 +19,7 @@ mod multi_partition; mod partitioned_batch_iterator; mod single_partition; mod traits; +mod immediate_mode; pub(crate) use multi_partition::MultiPartitionShuffleRepartitioner; pub(crate) use partitioned_batch_iterator::PartitionedBatchIterator; From b0d4ab21f0715702f706892388e636c054c18b1a Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 30 Mar 2026 15:21:14 -0600 Subject: [PATCH 13/46] feat: implement ImmediateModePartitioner with partition eval, row routing, and spilling --- .../src/partitioners/immediate_mode.rs | 530 +++++++++++++++++- 1 file changed, 520 insertions(+), 10 deletions(-) diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index 849d454966..bd9f2b9eb6 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -15,12 +15,21 @@ // specific language governing permissions and limitations // under the License. -use arrow::array::RecordBatch; +use crate::metrics::ShufflePartitionerMetrics; +use crate::partitioners::ShufflePartitioner; +use crate::{comet_partitioning, CometPartitioning, CompressionCodec}; +use arrow::array::{ArrayRef, RecordBatch}; +use arrow::compute::interleave_record_batch; use arrow::datatypes::SchemaRef; use arrow::ipc::writer::StreamWriter; -use datafusion::common::DataFusionError; -use datafusion::error::Result; -use crate::CompressionCodec; +use datafusion::common::{DataFusionError, Result}; +use datafusion::execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use datafusion::execution::runtime_env::RuntimeEnv; +use datafusion_comet_spark_expr::murmur3::create_murmur3_hashes; +use std::fs::{File, OpenOptions}; +use std::io::{BufReader, BufWriter, Seek, Write}; +use std::sync::Arc; +use tokio::time::Instant; /// Per-partition output stream that serializes Arrow IPC batches into an /// in-memory buffer with compression. The block format matches @@ -31,7 +40,7 @@ use crate::CompressionCodec; /// - 4 bytes: codec tag (b"SNAP", b"LZ4_", b"ZSTD", or b"NONE") /// - N bytes: compressed Arrow IPC stream data #[allow(dead_code)] -pub struct PartitionOutputStream { +pub(crate) struct PartitionOutputStream { schema: SchemaRef, codec: CompressionCodec, buffer: Vec, @@ -39,7 +48,7 @@ pub struct PartitionOutputStream { #[allow(dead_code)] impl PartitionOutputStream { - pub fn try_new(schema: SchemaRef, codec: CompressionCodec) -> Result { + pub(crate) fn try_new(schema: SchemaRef, codec: CompressionCodec) -> Result { Ok(Self { schema, codec, @@ -49,7 +58,7 @@ impl PartitionOutputStream { /// Writes a record batch as a length-prefixed compressed IPC block. /// Returns the number of bytes written to the buffer. - pub fn write_batch(&mut self, batch: &RecordBatch) -> Result { + fn write_batch(&mut self, batch: &RecordBatch) -> Result { if batch.num_rows() == 0 { return Ok(0); } @@ -129,27 +138,382 @@ impl PartitionOutputStream { } /// Returns the number of bytes currently in the buffer. - pub fn buffered_bytes(&self) -> usize { + fn buffered_bytes(&self) -> usize { self.buffer.len() } /// Takes the buffer contents, leaving the buffer empty. - pub fn drain_buffer(&mut self) -> Vec { + fn drain_buffer(&mut self) -> Vec { std::mem::take(&mut self.buffer) } /// Consumes self and returns the buffer. - pub fn finish(self) -> Result> { + fn finish(self) -> Result> { Ok(self.buffer) } } +#[allow(dead_code)] +struct SpillFile { + _temp_file: datafusion::execution::disk_manager::RefCountedTempFile, + file: File, +} + +/// A partitioner that immediately writes IPC blocks per partition as batches arrive, +/// rather than buffering all data until shuffle_write. Supports spilling per-partition +/// buffers to disk under memory pressure. +#[allow(dead_code)] +pub(crate) struct ImmediateModePartitioner { + output_data_file: String, + output_index_file: String, + streams: Vec, + spill_files: Vec>, + partitioning: CometPartitioning, + runtime: Arc, + reservation: MemoryReservation, + metrics: ShufflePartitionerMetrics, + hashes_buf: Vec, + partition_ids: Vec, + batch_size: usize, +} + +#[allow(dead_code)] +impl ImmediateModePartitioner { + #[allow(clippy::too_many_arguments)] + pub(crate) fn try_new( + partition: usize, + output_data_file: String, + output_index_file: String, + schema: SchemaRef, + partitioning: CometPartitioning, + metrics: ShufflePartitionerMetrics, + runtime: Arc, + batch_size: usize, + codec: CompressionCodec, + ) -> Result { + let num_output_partitions = partitioning.partition_count(); + + let streams = (0..num_output_partitions) + .map(|_| PartitionOutputStream::try_new(Arc::clone(&schema), codec.clone())) + .collect::>>()?; + + let spill_files: Vec> = + (0..num_output_partitions).map(|_| None).collect(); + + let hashes_buf = match &partitioning { + CometPartitioning::Hash(_, _) | CometPartitioning::RoundRobin(_, _) => { + vec![0u32; batch_size] + } + _ => vec![], + }; + + let reservation = + MemoryConsumer::new(format!("ImmediateModePartitioner[{partition}]")) + .with_can_spill(true) + .register(&runtime.memory_pool); + + Ok(Self { + output_data_file, + output_index_file, + streams, + spill_files, + partitioning, + runtime, + reservation, + metrics, + hashes_buf, + partition_ids: vec![0u32; batch_size], + batch_size, + }) + } + + /// Compute partition IDs for each row in the batch, storing results in + /// `self.partition_ids`. Returns the number of output partitions. + fn compute_partition_ids(&mut self, batch: &RecordBatch) -> Result { + let num_rows = batch.num_rows(); + match &self.partitioning { + CometPartitioning::Hash(exprs, num_output_partitions) => { + let num_output_partitions = *num_output_partitions; + let arrays = exprs + .iter() + .map(|expr| expr.evaluate(batch)?.into_array(num_rows)) + .collect::>>()?; + + let hashes_buf = &mut self.hashes_buf[..num_rows]; + hashes_buf.fill(42_u32); + create_murmur3_hashes(&arrays, hashes_buf)?; + + let partition_ids = &mut self.partition_ids[..num_rows]; + for (idx, hash) in hashes_buf.iter().enumerate() { + partition_ids[idx] = + comet_partitioning::pmod(*hash, num_output_partitions) as u32; + } + Ok(num_output_partitions) + } + CometPartitioning::RoundRobin(num_output_partitions, max_hash_columns) => { + let num_output_partitions = *num_output_partitions; + let max_hash_columns = *max_hash_columns; + let num_columns_to_hash = if max_hash_columns == 0 { + batch.num_columns() + } else { + max_hash_columns.min(batch.num_columns()) + }; + let columns_to_hash: Vec = (0..num_columns_to_hash) + .map(|i| Arc::clone(batch.column(i))) + .collect(); + + let hashes_buf = &mut self.hashes_buf[..num_rows]; + hashes_buf.fill(42_u32); + create_murmur3_hashes(&columns_to_hash, hashes_buf)?; + + let partition_ids = &mut self.partition_ids[..num_rows]; + for (idx, hash) in hashes_buf.iter().enumerate() { + partition_ids[idx] = + comet_partitioning::pmod(*hash, num_output_partitions) as u32; + } + Ok(num_output_partitions) + } + CometPartitioning::RangePartitioning( + lex_ordering, + num_output_partitions, + row_converter, + bounds, + ) => { + let num_output_partitions = *num_output_partitions; + let arrays = lex_ordering + .iter() + .map(|expr| expr.expr.evaluate(batch)?.into_array(num_rows)) + .collect::>>()?; + + let row_batch = row_converter.convert_columns(arrays.as_slice())?; + let partition_ids = &mut self.partition_ids[..num_rows]; + for (row_idx, row) in row_batch.iter().enumerate() { + partition_ids[row_idx] = bounds + .as_slice() + .partition_point(|bound| bound.row() <= row) + as u32; + } + Ok(num_output_partitions) + } + other => Err(DataFusionError::NotImplemented(format!( + "Unsupported shuffle partitioning scheme {other:?}" + ))), + } + } + + /// Route rows to their partition streams using interleave, then write IPC blocks. + /// Returns total bytes written across all partitions. + fn write_partitioned_rows(&mut self, batch: &RecordBatch) -> Result { + let num_partitions = self.streams.len(); + let num_rows = batch.num_rows(); + + // Build per-partition row indices + let mut partition_row_indices: Vec> = + vec![Vec::new(); num_partitions]; + for row_idx in 0..num_rows { + let pid = self.partition_ids[row_idx] as usize; + partition_row_indices[pid].push((0, row_idx)); + } + + let batch_refs = [batch]; + let mut total_bytes = 0; + + let mut interleave_timer = self.metrics.interleave_time.timer(); + for (pid, indices) in partition_row_indices.iter().enumerate() { + if indices.is_empty() { + continue; + } + let sub_batch = interleave_record_batch(&batch_refs, indices) + .map_err(|e| DataFusionError::ArrowError(Box::from(e), None))?; + interleave_timer.stop(); + + let mut encode_timer = self.metrics.encode_time.timer(); + let bytes = self.streams[pid].write_batch(&sub_batch)?; + encode_timer.stop(); + + total_bytes += bytes; + interleave_timer = self.metrics.interleave_time.timer(); + } + interleave_timer.stop(); + + Ok(total_bytes) + } + + /// Spill all partition buffers to per-partition temp files. + fn spill_all(&mut self) -> Result<()> { + let mut spilled_bytes = 0usize; + + for pid in 0..self.streams.len() { + let buf = self.streams[pid].drain_buffer(); + if buf.is_empty() { + continue; + } + + // Create spill file on first spill for this partition + if self.spill_files[pid].is_none() { + let temp_file = self + .runtime + .disk_manager + .create_tmp_file(&format!("imm_shuffle_p{pid}"))?; + let path = temp_file.path().to_owned(); + let file = OpenOptions::new() + .append(true) + .open(&path) + .map_err(|e| { + DataFusionError::Execution(format!( + "Failed to open spill file: {e}" + )) + })?; + self.spill_files[pid] = Some(SpillFile { + _temp_file: temp_file, + file, + }); + } + + if let Some(spill) = &mut self.spill_files[pid] { + spill.file.write_all(&buf).map_err(|e| { + DataFusionError::Execution(format!("Failed to write spill: {e}")) + })?; + spilled_bytes += buf.len(); + } + } + + // Flush all spill files so data is visible when re-opened for reading in shuffle_write + for spill in self.spill_files.iter_mut().flatten() { + spill.file.flush()?; + } + + self.reservation.free(); + if spilled_bytes > 0 { + self.metrics.spill_count.add(1); + self.metrics.spilled_bytes.add(spilled_bytes); + } + + Ok(()) + } +} + +#[async_trait::async_trait] +impl ShufflePartitioner for ImmediateModePartitioner { + async fn insert_batch(&mut self, batch: RecordBatch) -> Result<()> { + if batch.num_rows() == 0 { + return Ok(()); + } + + let start_time = Instant::now(); + + self.metrics.data_size.add(batch.get_array_memory_size()); + self.metrics.baseline.record_output(batch.num_rows()); + + let mut start = 0; + while start < batch.num_rows() { + let end = (start + self.batch_size).min(batch.num_rows()); + let chunk = batch.slice(start, end - start); + + let repart_start = Instant::now(); + self.compute_partition_ids(&chunk)?; + self.metrics + .repart_time + .add_duration(repart_start.elapsed()); + + let bytes_written = self.write_partitioned_rows(&chunk)?; + + if self.reservation.try_grow(bytes_written).is_err() { + self.spill_all()?; + } + + start = end; + } + + self.metrics.input_batches.add(1); + self.metrics + .baseline + .elapsed_compute() + .add_duration(start_time.elapsed()); + + Ok(()) + } + + fn shuffle_write(&mut self) -> Result<()> { + let start_time = Instant::now(); + let num_output_partitions = self.streams.len(); + let mut offsets = vec![0i64; num_output_partitions + 1]; + + let data_file = OpenOptions::new() + .write(true) + .create(true) + .truncate(true) + .open(&self.output_data_file) + .map_err(|e| { + DataFusionError::Execution(format!("shuffle write error: {e:?}")) + })?; + let mut output_data = BufWriter::new(data_file); + + #[allow(clippy::needless_range_loop)] + for pid in 0..num_output_partitions { + offsets[pid] = output_data.stream_position()? as i64; + + // Copy spill file contents if any + if let Some(spill) = &self.spill_files[pid] { + let path = spill._temp_file.path().to_owned(); + let spill_reader = File::open(&path).map_err(|e| { + DataFusionError::Execution(format!( + "Failed to open spill file for reading: {e}" + )) + })?; + let mut reader = BufReader::new(spill_reader); + let mut write_timer = self.metrics.write_time.timer(); + std::io::copy(&mut reader, &mut output_data)?; + write_timer.stop(); + } + + // Write remaining in-memory buffer + let buf = self.streams[pid].drain_buffer(); + if !buf.is_empty() { + let mut write_timer = self.metrics.write_time.timer(); + output_data.write_all(&buf)?; + write_timer.stop(); + } + } + + let mut write_timer = self.metrics.write_time.timer(); + output_data.flush()?; + write_timer.stop(); + + // Record final offset + offsets[num_output_partitions] = output_data.stream_position()? as i64; + + // Write index file + let mut write_timer = self.metrics.write_time.timer(); + let mut output_index = BufWriter::new( + File::create(&self.output_index_file).map_err(|e| { + DataFusionError::Execution(format!("shuffle write error: {e:?}")) + })?, + ); + for offset in &offsets { + output_index.write_all(&offset.to_le_bytes())?; + } + output_index.flush()?; + write_timer.stop(); + + self.metrics + .baseline + .elapsed_compute() + .add_duration(start_time.elapsed()); + + Ok(()) + } +} + #[cfg(test)] mod tests { use super::*; use crate::read_ipc_compressed; use arrow::array::Int32Array; use arrow::datatypes::{DataType, Field, Schema}; + use datafusion::execution::memory_pool::GreedyMemoryPool; + use datafusion::execution::runtime_env::RuntimeEnvBuilder; + use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; use std::sync::Arc; fn make_test_batch(values: &[i32]) -> RecordBatch { @@ -262,4 +626,150 @@ mod tests { stream.write_batch(&batch).unwrap(); assert!(stream.buffered_bytes() > 0); } + + fn make_hash_partitioning(col_name: &str, num_partitions: usize) -> CometPartitioning { + use datafusion::physical_expr::expressions::Column; + let expr: Arc = + Arc::new(Column::new(col_name, 0)); + CometPartitioning::Hash(vec![expr], num_partitions) + } + + #[tokio::test] + async fn test_immediate_mode_partitioner_hash() { + let batch = make_test_batch(&[1, 2, 3, 4, 5, 6, 7, 8]); + let schema = batch.schema(); + let dir = tempfile::tempdir().unwrap(); + let data_path = dir.path().join("data").to_str().unwrap().to_string(); + let index_path = dir.path().join("index").to_str().unwrap().to_string(); + + let metrics = ShufflePartitionerMetrics::new(&ExecutionPlanMetricsSet::new(), 0); + let runtime = Arc::new(RuntimeEnvBuilder::new().build().unwrap()); + + let mut partitioner = ImmediateModePartitioner::try_new( + 0, + data_path, + index_path, + schema, + make_hash_partitioning("a", 4), + metrics, + runtime, + 8192, + CompressionCodec::None, + ) + .unwrap(); + + partitioner.insert_batch(batch).await.unwrap(); + + // Verify some bytes are buffered across partitions + let total_buffered: usize = partitioner.streams.iter().map(|s| s.buffered_bytes()).sum(); + assert!(total_buffered > 0, "Expected some buffered bytes"); + } + + #[tokio::test] + async fn test_immediate_mode_shuffle_write() { + let batch1 = make_test_batch(&[1, 2, 3, 4, 5, 6]); + let batch2 = make_test_batch(&[7, 8, 9, 10, 11, 12]); + let schema = batch1.schema(); + let dir = tempfile::tempdir().unwrap(); + let data_path = dir.path().join("data").to_str().unwrap().to_string(); + let index_path = dir.path().join("index").to_str().unwrap().to_string(); + + let num_partitions = 3; + let metrics = ShufflePartitionerMetrics::new(&ExecutionPlanMetricsSet::new(), 0); + let runtime = Arc::new(RuntimeEnvBuilder::new().build().unwrap()); + + let mut partitioner = ImmediateModePartitioner::try_new( + 0, + data_path.clone(), + index_path.clone(), + schema, + make_hash_partitioning("a", num_partitions), + metrics, + runtime, + 8192, + CompressionCodec::None, + ) + .unwrap(); + + partitioner.insert_batch(batch1).await.unwrap(); + partitioner.insert_batch(batch2).await.unwrap(); + partitioner.shuffle_write().unwrap(); + + // Verify index file has (num_partitions + 1) * 8 bytes + let index_data = std::fs::read(&index_path).unwrap(); + assert_eq!(index_data.len(), (num_partitions + 1) * 8); + + // First offset should be 0 + let first_offset = i64::from_le_bytes(index_data[0..8].try_into().unwrap()); + assert_eq!(first_offset, 0); + + // Last offset should equal data file size + let data_file_size = std::fs::metadata(&data_path).unwrap().len(); + let last_offset = i64::from_le_bytes( + index_data[num_partitions * 8..(num_partitions + 1) * 8] + .try_into() + .unwrap(), + ); + assert_eq!(last_offset as u64, data_file_size); + + // Data file should be non-empty + assert!(data_file_size > 0); + } + + #[tokio::test] + async fn test_immediate_mode_spill() { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + let dir = tempfile::tempdir().unwrap(); + let data_path = dir.path().join("data").to_str().unwrap().to_string(); + let index_path = dir.path().join("index").to_str().unwrap().to_string(); + + let num_partitions = 2; + let metrics = ShufflePartitionerMetrics::new(&ExecutionPlanMetricsSet::new(), 0); + + // Use a tiny memory pool to force spilling + let runtime = Arc::new( + RuntimeEnvBuilder::new() + .with_memory_pool(Arc::new(GreedyMemoryPool::new(256))) + .build() + .unwrap(), + ); + + let mut partitioner = ImmediateModePartitioner::try_new( + 0, + data_path.clone(), + index_path.clone(), + Arc::clone(&schema), + make_hash_partitioning("a", num_partitions), + metrics, + runtime, + 8192, + CompressionCodec::None, + ) + .unwrap(); + + // Insert enough data to exceed the tiny memory pool and trigger spills + for i in 0..10 { + let values: Vec = ((i * 10)..((i + 1) * 10)).collect(); + let batch = make_test_batch(&values); + partitioner.insert_batch(batch).await.unwrap(); + } + + partitioner.shuffle_write().unwrap(); + + // Verify output is valid + let index_data = std::fs::read(&index_path).unwrap(); + assert_eq!(index_data.len(), (num_partitions + 1) * 8); + + let first_offset = i64::from_le_bytes(index_data[0..8].try_into().unwrap()); + assert_eq!(first_offset, 0); + + let data_file_size = std::fs::metadata(&data_path).unwrap().len(); + let last_offset = i64::from_le_bytes( + index_data[num_partitions * 8..(num_partitions + 1) * 8] + .try_into() + .unwrap(), + ); + assert_eq!(last_offset as u64, data_file_size); + assert!(data_file_size > 0); + } } From 56574d58377b1ec5adb21536571ff972f8668758 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 30 Mar 2026 15:26:14 -0600 Subject: [PATCH 14/46] feat: export ImmediateModePartitioner and document factory wiring --- native/shuffle/src/partitioners/mod.rs | 2 ++ native/shuffle/src/shuffle_writer.rs | 12 ++++++++++++ 2 files changed, 14 insertions(+) diff --git a/native/shuffle/src/partitioners/mod.rs b/native/shuffle/src/partitioners/mod.rs index c965f84377..300127160f 100644 --- a/native/shuffle/src/partitioners/mod.rs +++ b/native/shuffle/src/partitioners/mod.rs @@ -21,6 +21,8 @@ mod single_partition; mod traits; mod immediate_mode; +#[allow(unused_imports)] +pub(crate) use immediate_mode::ImmediateModePartitioner; pub(crate) use multi_partition::MultiPartitionShuffleRepartitioner; pub(crate) use partitioned_batch_iterator::PartitionedBatchIterator; pub(crate) use single_partition::SinglePartitionShufflePartitioner; diff --git a/native/shuffle/src/shuffle_writer.rs b/native/shuffle/src/shuffle_writer.rs index 03c9d3ff23..510d2ae909 100644 --- a/native/shuffle/src/shuffle_writer.rs +++ b/native/shuffle/src/shuffle_writer.rs @@ -226,6 +226,18 @@ async fn external_shuffle( write_buffer_size, )?) } + // To use ImmediateModePartitioner instead of MultiPartitionShuffleRepartitioner: + // _ => Box::new(ImmediateModePartitioner::try_new( + // partition, + // output_data_file, + // output_index_file, + // Arc::clone(&schema), + // partitioning, + // metrics, + // context.runtime_env(), + // context.session_config().batch_size(), + // codec, + // )?), _ => Box::new(MultiPartitionShuffleRepartitioner::try_new( partition, output_data_file, From cdbbdee04b5fd6b30ebb4394573704d4db40f5b6 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 30 Mar 2026 15:26:47 -0600 Subject: [PATCH 15/46] test: verify ImmediateModePartitioner block format compatibility with read_ipc_compressed --- .../src/partitioners/immediate_mode.rs | 97 +++++++++++++++++++ 1 file changed, 97 insertions(+) diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index bd9f2b9eb6..92a20ea6c4 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -772,4 +772,101 @@ mod tests { assert_eq!(last_offset as u64, data_file_size); assert!(data_file_size > 0); } + + #[tokio::test] + async fn test_block_format_compatible_with_read_ipc_compressed() { + let batch = make_test_batch(&[1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); + let schema = batch.schema(); + let dir = tempfile::tempdir().unwrap(); + let data_path = dir.path().join("data").to_str().unwrap().to_string(); + let index_path = dir.path().join("index").to_str().unwrap().to_string(); + + let num_partitions = 2; + let metrics = ShufflePartitionerMetrics::new(&ExecutionPlanMetricsSet::new(), 0); + let runtime = Arc::new(RuntimeEnvBuilder::new().build().unwrap()); + + let mut partitioner = ImmediateModePartitioner::try_new( + 0, + data_path.clone(), + index_path.clone(), + Arc::clone(&schema), + make_hash_partitioning("a", num_partitions), + metrics, + runtime, + 8192, + CompressionCodec::Lz4Frame, + ) + .unwrap(); + + partitioner.insert_batch(batch).await.unwrap(); + partitioner.shuffle_write().unwrap(); + + // Read index file to get partition offsets + let index_data = std::fs::read(&index_path).unwrap(); + let mut offsets = Vec::new(); + for i in 0..=num_partitions { + let offset = + i64::from_le_bytes(index_data[i * 8..(i + 1) * 8].try_into().unwrap()); + offsets.push(offset as usize); + } + + // Read entire data file + let data = std::fs::read(&data_path).unwrap(); + + let mut total_rows = 0; + for pid in 0..num_partitions { + let partition_start = offsets[pid]; + let partition_end = offsets[pid + 1]; + if partition_start == partition_end { + continue; + } + + // Parse blocks within this partition's byte range + let mut pos = partition_start; + while pos < partition_end { + // Read 8-byte length prefix + let payload_len = u64::from_le_bytes( + data[pos..pos + 8].try_into().unwrap(), + ) as usize; + assert!(payload_len > 0, "Block payload length should be > 0"); + + // Skip 8-byte field_count + let field_count = u64::from_le_bytes( + data[pos + 8..pos + 16].try_into().unwrap(), + ) as usize; + assert_eq!(field_count, 1, "Expected 1 field"); + + // Pass codec tag + IPC data to read_ipc_compressed + let block_end = pos + 8 + payload_len; + let ipc_data = &data[pos + 16..block_end]; + let decoded = read_ipc_compressed(ipc_data).unwrap(); + + assert_eq!(decoded.num_columns(), 1); + assert!(decoded.num_rows() > 0); + + // Verify values are valid Int32 + let col = decoded + .column(0) + .as_any() + .downcast_ref::() + .expect("Expected Int32Array"); + for i in 0..col.len() { + let v = col.value(i); + assert!( + (1..=10).contains(&v), + "Value {v} not in expected range 1..=10" + ); + } + + total_rows += decoded.num_rows(); + pos = block_end; + } + assert_eq!( + pos, partition_end, + "Block parsing should consume exactly the partition's bytes" + ); + } + + assert_eq!(total_rows, 10, "Total decoded rows should match input"); + } } From 4bcb0143d4f37241afb9aa9e8589cab5f1709b82 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 30 Mar 2026 15:33:43 -0600 Subject: [PATCH 16/46] feat: add spark.comet.exec.shuffle.partitionerMode config Add CometConf config to choose between 'immediate' and 'buffered' shuffle partitioner modes. The config flows through protobuf to the native shuffle writer factory, which selects ImmediateModePartitioner or MultiPartitionShuffleRepartitioner accordingly. Defaults to 'immediate' mode. --- .../scala/org/apache/comet/CometConf.scala | 12 ++++++ native/core/src/execution/planner.rs | 1 + native/proto/src/proto/operator.proto | 4 ++ native/shuffle/benches/shuffle_writer.rs | 1 + native/shuffle/src/bin/shuffle_bench.rs | 1 + .../src/partitioners/immediate_mode.rs | 7 +--- native/shuffle/src/partitioners/mod.rs | 1 - native/shuffle/src/shuffle_writer.rs | 37 ++++++++++++------- .../shuffle/CometNativeShuffleWriter.scala | 2 + 9 files changed, 47 insertions(+), 19 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index bfe90181ff..2d5a715ad1 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -523,6 +523,18 @@ object CometConf extends ShimCometConf { "Should not be larger than batch size `spark.comet.batchSize`") .createWithDefault(8192) + val COMET_SHUFFLE_PARTITIONER_MODE: ConfigEntry[String] = + conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.partitionerMode") + .category(CATEGORY_SHUFFLE) + .doc( + "The partitioner mode used by the native shuffle writer. " + + "'immediate' writes partitioned IPC blocks immediately as batches arrive, " + + "reducing memory usage. 'buffered' buffers all rows before writing, which may " + + "improve performance for small datasets but uses more memory.") + .stringConf + .checkValues(Set("immediate", "buffered")) + .createWithDefault("immediate") + val COMET_SHUFFLE_WRITE_BUFFER_SIZE: ConfigEntry[Long] = conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.writeBufferSize") .category(CATEGORY_SHUFFLE) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 5af31fcc22..838099d85a 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1360,6 +1360,7 @@ impl PhysicalPlanner { writer.output_index_file.clone(), writer.tracing_enabled, write_buffer_size, + writer.immediate_mode, )?); Ok(( diff --git a/native/proto/src/proto/operator.proto b/native/proto/src/proto/operator.proto index 344b9f0f21..5726484c0f 100644 --- a/native/proto/src/proto/operator.proto +++ b/native/proto/src/proto/operator.proto @@ -294,6 +294,10 @@ message ShuffleWriter { // Size of the write buffer in bytes used when writing shuffle data to disk. // Larger values may improve write performance but use more memory. int32 write_buffer_size = 8; + // Whether to use immediate mode partitioner. When true, partitioned IPC blocks + // are written immediately as batches arrive. When false, rows are buffered + // before writing (the original behavior). + bool immediate_mode = 9; } message ParquetWriter { diff --git a/native/shuffle/benches/shuffle_writer.rs b/native/shuffle/benches/shuffle_writer.rs index 27abd919fa..873e872adf 100644 --- a/native/shuffle/benches/shuffle_writer.rs +++ b/native/shuffle/benches/shuffle_writer.rs @@ -153,6 +153,7 @@ fn create_shuffle_writer_exec( "/tmp/index.out".to_string(), false, 1024 * 1024, + false, // immediate_mode ) .unwrap() } diff --git a/native/shuffle/src/bin/shuffle_bench.rs b/native/shuffle/src/bin/shuffle_bench.rs index a412d11eed..174989f1c0 100644 --- a/native/shuffle/src/bin/shuffle_bench.rs +++ b/native/shuffle/src/bin/shuffle_bench.rs @@ -523,6 +523,7 @@ async fn execute_shuffle_write( index_file, false, write_buffer_size, + false, // immediate_mode ) .expect("Failed to create ShuffleWriterExec"); diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index 92a20ea6c4..899885d4db 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -39,14 +39,12 @@ use tokio::time::Instant; /// - 8 bytes: field_count (usize LE) /// - 4 bytes: codec tag (b"SNAP", b"LZ4_", b"ZSTD", or b"NONE") /// - N bytes: compressed Arrow IPC stream data -#[allow(dead_code)] pub(crate) struct PartitionOutputStream { schema: SchemaRef, codec: CompressionCodec, buffer: Vec, } -#[allow(dead_code)] impl PartitionOutputStream { pub(crate) fn try_new(schema: SchemaRef, codec: CompressionCodec) -> Result { Ok(Self { @@ -138,6 +136,7 @@ impl PartitionOutputStream { } /// Returns the number of bytes currently in the buffer. + #[cfg(test)] fn buffered_bytes(&self) -> usize { self.buffer.len() } @@ -148,12 +147,12 @@ impl PartitionOutputStream { } /// Consumes self and returns the buffer. + #[cfg(test)] fn finish(self) -> Result> { Ok(self.buffer) } } -#[allow(dead_code)] struct SpillFile { _temp_file: datafusion::execution::disk_manager::RefCountedTempFile, file: File, @@ -162,7 +161,6 @@ struct SpillFile { /// A partitioner that immediately writes IPC blocks per partition as batches arrive, /// rather than buffering all data until shuffle_write. Supports spilling per-partition /// buffers to disk under memory pressure. -#[allow(dead_code)] pub(crate) struct ImmediateModePartitioner { output_data_file: String, output_index_file: String, @@ -177,7 +175,6 @@ pub(crate) struct ImmediateModePartitioner { batch_size: usize, } -#[allow(dead_code)] impl ImmediateModePartitioner { #[allow(clippy::too_many_arguments)] pub(crate) fn try_new( diff --git a/native/shuffle/src/partitioners/mod.rs b/native/shuffle/src/partitioners/mod.rs index 300127160f..18b46813d3 100644 --- a/native/shuffle/src/partitioners/mod.rs +++ b/native/shuffle/src/partitioners/mod.rs @@ -21,7 +21,6 @@ mod single_partition; mod traits; mod immediate_mode; -#[allow(unused_imports)] pub(crate) use immediate_mode::ImmediateModePartitioner; pub(crate) use multi_partition::MultiPartitionShuffleRepartitioner; pub(crate) use partitioned_batch_iterator::PartitionedBatchIterator; diff --git a/native/shuffle/src/shuffle_writer.rs b/native/shuffle/src/shuffle_writer.rs index 510d2ae909..c6602217a1 100644 --- a/native/shuffle/src/shuffle_writer.rs +++ b/native/shuffle/src/shuffle_writer.rs @@ -19,7 +19,8 @@ use crate::metrics::ShufflePartitionerMetrics; use crate::partitioners::{ - MultiPartitionShuffleRepartitioner, ShufflePartitioner, SinglePartitionShufflePartitioner, + ImmediateModePartitioner, MultiPartitionShuffleRepartitioner, ShufflePartitioner, + SinglePartitionShufflePartitioner, }; use crate::{CometPartitioning, CompressionCodec}; use async_trait::async_trait; @@ -68,6 +69,8 @@ pub struct ShuffleWriterExec { tracing_enabled: bool, /// Size of the write buffer in bytes write_buffer_size: usize, + /// When true, use ImmediateModePartitioner instead of MultiPartitionShuffleRepartitioner + immediate_mode: bool, } impl ShuffleWriterExec { @@ -81,6 +84,7 @@ impl ShuffleWriterExec { output_index_file: String, tracing_enabled: bool, write_buffer_size: usize, + immediate_mode: bool, ) -> Result { let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&input.schema())), @@ -99,6 +103,7 @@ impl ShuffleWriterExec { codec, tracing_enabled, write_buffer_size, + immediate_mode, }) } } @@ -163,6 +168,7 @@ impl ExecutionPlan for ShuffleWriterExec { self.output_index_file.clone(), self.tracing_enabled, self.write_buffer_size, + self.immediate_mode, )?)), _ => panic!("ShuffleWriterExec wrong number of children"), } @@ -190,6 +196,7 @@ impl ExecutionPlan for ShuffleWriterExec { self.codec.clone(), self.tracing_enabled, self.write_buffer_size, + self.immediate_mode, ) .map_err(|e| ArrowError::ExternalError(Box::new(e))), ) @@ -210,6 +217,7 @@ async fn external_shuffle( codec: CompressionCodec, tracing_enabled: bool, write_buffer_size: usize, + immediate_mode: bool, ) -> Result { with_trace_async("external_shuffle", tracing_enabled, || async { let schema = input.schema(); @@ -226,18 +234,19 @@ async fn external_shuffle( write_buffer_size, )?) } - // To use ImmediateModePartitioner instead of MultiPartitionShuffleRepartitioner: - // _ => Box::new(ImmediateModePartitioner::try_new( - // partition, - // output_data_file, - // output_index_file, - // Arc::clone(&schema), - // partitioning, - // metrics, - // context.runtime_env(), - // context.session_config().batch_size(), - // codec, - // )?), + _ if immediate_mode => { + Box::new(ImmediateModePartitioner::try_new( + partition, + output_data_file, + output_index_file, + Arc::clone(&schema), + partitioning, + metrics, + context.runtime_env(), + context.session_config().batch_size(), + codec, + )?) + } _ => Box::new(MultiPartitionShuffleRepartitioner::try_new( partition, output_data_file, @@ -478,6 +487,7 @@ mod test { "/tmp/index.out".to_string(), false, 1024 * 1024, // write_buffer_size: 1MB default + false, // immediate_mode ) .unwrap(); @@ -537,6 +547,7 @@ mod test { index_file.clone(), false, 1024 * 1024, + false, // immediate_mode ) .unwrap(); 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 3fc222bd19..96c140300b 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 @@ -192,6 +192,8 @@ class CometNativeShuffleWriter[K, V]( CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_ZSTD_LEVEL.get) shuffleWriterBuilder.setWriteBufferSize( CometConf.COMET_SHUFFLE_WRITE_BUFFER_SIZE.get().max(Int.MaxValue).toInt) + shuffleWriterBuilder.setImmediateMode( + CometConf.COMET_SHUFFLE_PARTITIONER_MODE.get() == "immediate") outputPartitioning match { case p if isSinglePartitioning(p) => From f2e459e1fe7b439b1aeabb4c4c1b6f83f581b54c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 30 Mar 2026 15:41:11 -0600 Subject: [PATCH 17/46] feat: add batch coalescing to PartitionOutputStream --- .../src/partitioners/immediate_mode.rs | 134 ++++++++++++------ 1 file changed, 91 insertions(+), 43 deletions(-) diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index 899885d4db..96a84c1c6d 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -19,6 +19,7 @@ use crate::metrics::ShufflePartitionerMetrics; use crate::partitioners::ShufflePartitioner; use crate::{comet_partitioning, CometPartitioning, CompressionCodec}; use arrow::array::{ArrayRef, RecordBatch}; +use arrow::compute::kernels::coalesce::BatchCoalescer; use arrow::compute::interleave_record_batch; use arrow::datatypes::SchemaRef; use arrow::ipc::writer::StreamWriter; @@ -43,24 +44,70 @@ pub(crate) struct PartitionOutputStream { schema: SchemaRef, codec: CompressionCodec, buffer: Vec, + /// Coalesces small batches into target_batch_size before serialization, + /// reducing per-block IPC schema overhead. Lazily initialized on first write. + coalescer: Option, + batch_size: usize, } impl PartitionOutputStream { - pub(crate) fn try_new(schema: SchemaRef, codec: CompressionCodec) -> Result { + pub(crate) fn try_new( + schema: SchemaRef, + codec: CompressionCodec, + batch_size: usize, + ) -> Result { Ok(Self { schema, codec, buffer: Vec::with_capacity(1024 * 1024), + coalescer: None, + batch_size, }) } - /// Writes a record batch as a length-prefixed compressed IPC block. - /// Returns the number of bytes written to the buffer. + /// Push a batch into the coalescer, serializing any completed batches as + /// length-prefixed compressed IPC blocks. Returns total bytes written. fn write_batch(&mut self, batch: &RecordBatch) -> Result { if batch.num_rows() == 0 { return Ok(0); } + let coalescer = self + .coalescer + .get_or_insert_with(|| BatchCoalescer::new(batch.schema(), self.batch_size)); + coalescer.push_batch(batch.clone())?; + + let mut completed = Vec::new(); + while let Some(batch) = coalescer.next_completed_batch() { + completed.push(batch); + } + + let mut total_bytes = 0; + for batch in &completed { + total_bytes += self.write_ipc_block(batch)?; + } + Ok(total_bytes) + } + + /// Flush any remaining rows in the coalescer as a final IPC block. + /// Must be called before `drain_buffer` or `finish` to avoid losing data. + fn flush(&mut self) -> Result { + let mut total_bytes = 0; + if let Some(coalescer) = &mut self.coalescer { + coalescer.finish_buffered_batch()?; + let mut remaining = Vec::new(); + while let Some(batch) = coalescer.next_completed_batch() { + remaining.push(batch); + } + for batch in &remaining { + total_bytes += self.write_ipc_block(batch)?; + } + } + Ok(total_bytes) + } + + /// Serialize a single record batch as a length-prefixed compressed IPC block. + fn write_ipc_block(&mut self, batch: &RecordBatch) -> Result { let start_pos = self.buffer.len(); // Write 8-byte placeholder for length prefix @@ -191,7 +238,7 @@ impl ImmediateModePartitioner { let num_output_partitions = partitioning.partition_count(); let streams = (0..num_output_partitions) - .map(|_| PartitionOutputStream::try_new(Arc::clone(&schema), codec.clone())) + .map(|_| PartitionOutputStream::try_new(Arc::clone(&schema), codec.clone(), batch_size)) .collect::>>()?; let spill_files: Vec> = @@ -341,6 +388,8 @@ impl ImmediateModePartitioner { let mut spilled_bytes = 0usize; for pid in 0..self.streams.len() { + // Flush coalescer so buffered rows are serialized before draining + self.streams[pid].flush()?; let buf = self.streams[pid].drain_buffer(); if buf.is_empty() { continue; @@ -464,7 +513,8 @@ impl ShufflePartitioner for ImmediateModePartitioner { write_timer.stop(); } - // Write remaining in-memory buffer + // Flush coalescer and write remaining in-memory buffer + self.streams[pid].flush()?; let buf = self.streams[pid].drain_buffer(); if !buf.is_empty() { let mut write_timer = self.metrics.write_time.timer(); @@ -530,67 +580,58 @@ mod tests { CompressionCodec::Zstd(1), CompressionCodec::Snappy, ] { + // Use batch_size=1 to force immediate serialization (no coalescing) let mut stream = - PartitionOutputStream::try_new(Arc::clone(&schema), codec).unwrap(); - let bytes_written = stream.write_batch(&batch).unwrap(); - assert!(bytes_written > 0); - assert_eq!(stream.buffered_bytes(), bytes_written); + PartitionOutputStream::try_new(Arc::clone(&schema), codec, 1).unwrap(); + stream.write_batch(&batch).unwrap(); + stream.flush().unwrap(); let buf = stream.finish().unwrap(); - assert_eq!(buf.len(), bytes_written); + assert!(!buf.is_empty()); - // Parse the block: 8 bytes length, 8 bytes field_count, then codec+data + // Parse the first block: 8 bytes length, 8 bytes field_count, then codec+data let ipc_length = u64::from_le_bytes(buf[0..8].try_into().unwrap()) as usize; - assert_eq!(ipc_length, buf.len() - 8); + assert!(ipc_length > 0); let field_count = usize::from_le_bytes(buf[8..16].try_into().unwrap()); assert_eq!(field_count, 1); // one field "a" // read_ipc_compressed expects data starting at the codec tag - let ipc_data = &buf[16..]; + let block_end = 8 + ipc_length; + let ipc_data = &buf[16..block_end]; let batch2 = read_ipc_compressed(ipc_data).unwrap(); - assert_eq!(batch, batch2); + assert!(batch2.num_rows() > 0); } } #[test] - fn test_partition_output_stream_multiple_batches() { + fn test_partition_output_stream_coalesces_small_batches() { let batch1 = make_test_batch(&[1, 2, 3]); let batch2 = make_test_batch(&[4, 5, 6, 7]); let schema = batch1.schema(); + // batch_size=10 means both batches (3+4=7 rows) fit in one coalesced block let mut stream = - PartitionOutputStream::try_new(schema, CompressionCodec::None).unwrap(); - - let bytes1 = stream.write_batch(&batch1).unwrap(); - assert!(bytes1 > 0); + PartitionOutputStream::try_new(schema, CompressionCodec::None, 10).unwrap(); - let bytes2 = stream.write_batch(&batch2).unwrap(); - assert!(bytes2 > 0); + // Small batches sit in coalescer, no IPC block written yet + stream.write_batch(&batch1).unwrap(); + assert_eq!(stream.buffered_bytes(), 0); - assert_eq!(stream.buffered_bytes(), bytes1 + bytes2); + stream.write_batch(&batch2).unwrap(); + assert_eq!(stream.buffered_bytes(), 0); + // Flush produces one coalesced block with all 7 rows + stream.flush().unwrap(); let buf = stream.finish().unwrap(); + assert!(!buf.is_empty()); - // Read first block - let len1 = u64::from_le_bytes(buf[0..8].try_into().unwrap()) as usize; - let block1_end = 8 + len1; - let ipc_data1 = &buf[16..block1_end]; - let decoded1 = read_ipc_compressed(ipc_data1).unwrap(); - assert_eq!(batch1, decoded1); - - // Read second block - let len2 = u64::from_le_bytes( - buf[block1_end..block1_end + 8].try_into().unwrap(), - ) as usize; - let block2_end = block1_end + 8 + len2; - let ipc_data2 = &buf[block1_end + 16..block2_end]; - let decoded2 = read_ipc_compressed(ipc_data2).unwrap(); - assert_eq!(batch2, decoded2); - - assert_eq!(block2_end, buf.len()); + let ipc_length = u64::from_le_bytes(buf[0..8].try_into().unwrap()) as usize; + let ipc_data = &buf[16..8 + ipc_length]; + let decoded = read_ipc_compressed(ipc_data).unwrap(); + assert_eq!(decoded.num_rows(), 7); } #[test] @@ -599,20 +640,23 @@ mod tests { let schema = batch.schema(); let mut stream = - PartitionOutputStream::try_new(schema, CompressionCodec::None).unwrap(); + PartitionOutputStream::try_new(schema, CompressionCodec::None, 8192).unwrap(); let bytes_written = stream.write_batch(&batch).unwrap(); assert_eq!(bytes_written, 0); assert_eq!(stream.buffered_bytes(), 0); } #[test] - fn test_partition_output_stream_drain_buffer() { + fn test_partition_output_stream_drain_after_flush() { let batch = make_test_batch(&[1, 2, 3]); let schema = batch.schema(); let mut stream = - PartitionOutputStream::try_new(schema, CompressionCodec::None).unwrap(); + PartitionOutputStream::try_new(schema, CompressionCodec::None, 8192).unwrap(); stream.write_batch(&batch).unwrap(); + + // Flush coalescer then drain + stream.flush().unwrap(); assert!(stream.buffered_bytes() > 0); let drained = stream.drain_buffer(); @@ -621,6 +665,7 @@ mod tests { // Can still write after drain stream.write_batch(&batch).unwrap(); + stream.flush().unwrap(); assert!(stream.buffered_bytes() > 0); } @@ -657,7 +702,10 @@ mod tests { partitioner.insert_batch(batch).await.unwrap(); - // Verify some bytes are buffered across partitions + // Flush coalesced data so it appears in the byte buffer + for stream in &mut partitioner.streams { + stream.flush().unwrap(); + } let total_buffered: usize = partitioner.streams.iter().map(|s| s.buffered_bytes()).sum(); assert!(total_buffered > 0, "Expected some buffered bytes"); } From 9ccdda794674ede179d9776b672a1302302fbf07 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 30 Mar 2026 15:42:26 -0600 Subject: [PATCH 18/46] feat: add --mode flag to shuffle benchmark for immediate vs buffered --- native/shuffle/src/bin/shuffle_bench.rs | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/native/shuffle/src/bin/shuffle_bench.rs b/native/shuffle/src/bin/shuffle_bench.rs index 174989f1c0..21228af87c 100644 --- a/native/shuffle/src/bin/shuffle_bench.rs +++ b/native/shuffle/src/bin/shuffle_bench.rs @@ -121,6 +121,11 @@ struct Args { /// Each task reads the same input and writes to its own output files. #[arg(long, default_value_t = 1)] concurrent_tasks: usize, + + /// Shuffle mode: 'immediate' writes IPC blocks per batch as they arrive, + /// 'buffered' buffers all rows before writing (original behavior). + #[arg(long, default_value = "immediate")] + mode: String, } fn main() { @@ -148,6 +153,7 @@ fn main() { println!("Partitioning: {}", args.partitioning); println!("Partitions: {}", args.partitions); println!("Codec: {:?}", codec); + println!("Mode: {}", args.mode); println!("Hash columns: {:?}", hash_col_indices); if let Some(mem_limit) = args.memory_limit { println!("Memory limit: {}", format_bytes(mem_limit)); @@ -459,6 +465,7 @@ fn run_shuffle_write( args.limit, data_file.to_string(), index_file.to_string(), + args.mode == "immediate", ) .await .unwrap(); @@ -482,6 +489,7 @@ async fn execute_shuffle_write( limit: usize, data_file: String, index_file: String, + immediate_mode: bool, ) -> datafusion::common::Result<(MetricsSet, MetricsSet)> { let config = SessionConfig::new().with_batch_size(batch_size); let mut runtime_builder = RuntimeEnvBuilder::new(); @@ -523,7 +531,7 @@ async fn execute_shuffle_write( index_file, false, write_buffer_size, - false, // immediate_mode + immediate_mode, ) .expect("Failed to create ShuffleWriterExec"); @@ -588,6 +596,7 @@ fn run_concurrent_shuffle_writes( let memory_limit = args.memory_limit; let write_buffer_size = args.write_buffer_size; let limit = args.limit; + let immediate_mode = args.mode == "immediate"; handles.push(tokio::spawn(async move { execute_shuffle_write( @@ -600,6 +609,7 @@ fn run_concurrent_shuffle_writes( limit, data_file, index_file, + immediate_mode, ) .await .unwrap() From cf56e395a590f50c93d833ad282fcea30781f172 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 30 Mar 2026 15:52:12 -0600 Subject: [PATCH 19/46] feat: add upfront memory reservation for coalescer buffers --- .../src/partitioners/immediate_mode.rs | 61 ++++++++++++++++++- 1 file changed, 58 insertions(+), 3 deletions(-) diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index 96a84c1c6d..bdbce57db2 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -21,7 +21,7 @@ use crate::{comet_partitioning, CometPartitioning, CompressionCodec}; use arrow::array::{ArrayRef, RecordBatch}; use arrow::compute::kernels::coalesce::BatchCoalescer; use arrow::compute::interleave_record_batch; -use arrow::datatypes::SchemaRef; +use arrow::datatypes::{DataType, SchemaRef}; use arrow::ipc::writer::StreamWriter; use datafusion::common::{DataFusionError, Result}; use datafusion::execution::memory_pool::{MemoryConsumer, MemoryReservation}; @@ -32,6 +32,42 @@ use std::io::{BufReader, BufWriter, Seek, Write}; use std::sync::Arc; use tokio::time::Instant; +/// Estimate the in-memory byte cost per row for a given schema. +/// Uses the primitive width for fixed-size types, and a heuristic for +/// variable-length types (strings, binary, nested). +fn estimate_row_bytes(schema: &SchemaRef) -> usize { + let mut total = 0; + for field in schema.fields() { + total += estimate_field_bytes(field.data_type()); + // 1 bit for null bitmap, round up + total += 1; + } + total.max(8) // minimum 8 bytes per row +} + +fn estimate_field_bytes(dt: &DataType) -> usize { + match dt { + DataType::Boolean => 1, + DataType::Null => 0, + dt if dt.primitive_width().is_some() => dt.primitive_width().unwrap(), + // Variable-length types: use a reasonable default estimate. + // 32 bytes covers typical short strings/binary. + DataType::Utf8 | DataType::Binary => 32 + 4, // data + 4-byte offset + DataType::LargeUtf8 | DataType::LargeBinary => 32 + 8, + DataType::Utf8View | DataType::BinaryView => 32 + 16, // data + 16-byte view + DataType::List(f) | DataType::LargeList(f) => { + 4 + estimate_field_bytes(f.data_type()) + } + DataType::Struct(fields) => fields + .iter() + .map(|f| estimate_field_bytes(f.data_type()) + 1) + .sum(), + DataType::Dictionary(key_dt, _) => estimate_field_bytes(key_dt), + // Fallback for other types + _ => 32, + } +} + /// Per-partition output stream that serializes Arrow IPC batches into an /// in-memory buffer with compression. The block format matches /// `ShuffleBlockWriter::write_batch` exactly: @@ -220,6 +256,10 @@ pub(crate) struct ImmediateModePartitioner { hashes_buf: Vec, partition_ids: Vec, batch_size: usize, + /// Fixed upfront reservation for coalescer buffers across all partitions. + /// Each coalescer holds at most batch_size rows; this budget is estimated + /// from the schema and reserved once to avoid incremental drift. + coalescer_budget: usize, } impl ImmediateModePartitioner { @@ -251,11 +291,20 @@ impl ImmediateModePartitioner { _ => vec![], }; - let reservation = + let mut reservation = MemoryConsumer::new(format!("ImmediateModePartitioner[{partition}]")) .with_can_spill(true) .register(&runtime.memory_pool); + // Reserve memory upfront for coalescer buffers across all partitions. + // Each coalescer holds at most batch_size rows. We estimate per-row bytes + // from the schema so this budget is fixed and doesn't drift. + let estimated_row_bytes = estimate_row_bytes(&schema); + let coalescer_budget = estimated_row_bytes * batch_size * num_output_partitions; + // Use try_grow — if the pool can't accommodate the budget, we proceed + // without the reservation (spilling will be more aggressive). + let _ = reservation.try_grow(coalescer_budget); + Ok(Self { output_data_file, output_index_file, @@ -268,6 +317,7 @@ impl ImmediateModePartitioner { hashes_buf, partition_ids: vec![0u32; batch_size], batch_size, + coalescer_budget, }) } @@ -429,7 +479,12 @@ impl ImmediateModePartitioner { spill.file.flush()?; } - self.reservation.free(); + // Shrink reservation to just the coalescer budget (IPC bytes were spilled, + // coalescer is empty after flush but will be refilled by future batches). + let target = self.coalescer_budget; + if self.reservation.size() > target { + self.reservation.shrink(self.reservation.size() - target); + } if spilled_bytes > 0 { self.metrics.spill_count.add(1); self.metrics.spilled_bytes.add(spilled_bytes); From f9139d756ffcc886067a0bbfd47fd1e3250a4937 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 30 Mar 2026 16:34:14 -0600 Subject: [PATCH 20/46] format --- .../src/partitioners/immediate_mode.rs | 58 +++++++------------ native/shuffle/src/partitioners/mod.rs | 2 +- native/shuffle/src/shuffle_writer.rs | 24 ++++---- 3 files changed, 32 insertions(+), 52 deletions(-) diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index bdbce57db2..0ced1b77a4 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -19,8 +19,8 @@ use crate::metrics::ShufflePartitionerMetrics; use crate::partitioners::ShufflePartitioner; use crate::{comet_partitioning, CometPartitioning, CompressionCodec}; use arrow::array::{ArrayRef, RecordBatch}; -use arrow::compute::kernels::coalesce::BatchCoalescer; use arrow::compute::interleave_record_batch; +use arrow::compute::kernels::coalesce::BatchCoalescer; use arrow::datatypes::{DataType, SchemaRef}; use arrow::ipc::writer::StreamWriter; use datafusion::common::{DataFusionError, Result}; @@ -55,9 +55,7 @@ fn estimate_field_bytes(dt: &DataType) -> usize { DataType::Utf8 | DataType::Binary => 32 + 4, // data + 4-byte offset DataType::LargeUtf8 | DataType::LargeBinary => 32 + 8, DataType::Utf8View | DataType::BinaryView => 32 + 16, // data + 16-byte view - DataType::List(f) | DataType::LargeList(f) => { - 4 + estimate_field_bytes(f.data_type()) - } + DataType::List(f) | DataType::LargeList(f) => 4 + estimate_field_bytes(f.data_type()), DataType::Struct(fields) => fields .iter() .map(|f| estimate_field_bytes(f.data_type()) + 1) @@ -291,10 +289,9 @@ impl ImmediateModePartitioner { _ => vec![], }; - let mut reservation = - MemoryConsumer::new(format!("ImmediateModePartitioner[{partition}]")) - .with_can_spill(true) - .register(&runtime.memory_pool); + let mut reservation = MemoryConsumer::new(format!("ImmediateModePartitioner[{partition}]")) + .with_can_spill(true) + .register(&runtime.memory_pool); // Reserve memory upfront for coalescer buffers across all partitions. // Each coalescer holds at most batch_size rows. We estimate per-row bytes @@ -402,8 +399,7 @@ impl ImmediateModePartitioner { let num_rows = batch.num_rows(); // Build per-partition row indices - let mut partition_row_indices: Vec> = - vec![Vec::new(); num_partitions]; + let mut partition_row_indices: Vec> = vec![Vec::new(); num_partitions]; for row_idx in 0..num_rows { let pid = self.partition_ids[row_idx] as usize; partition_row_indices[pid].push((0, row_idx)); @@ -452,14 +448,9 @@ impl ImmediateModePartitioner { .disk_manager .create_tmp_file(&format!("imm_shuffle_p{pid}"))?; let path = temp_file.path().to_owned(); - let file = OpenOptions::new() - .append(true) - .open(&path) - .map_err(|e| { - DataFusionError::Execution(format!( - "Failed to open spill file: {e}" - )) - })?; + let file = OpenOptions::new().append(true).open(&path).map_err(|e| { + DataFusionError::Execution(format!("Failed to open spill file: {e}")) + })?; self.spill_files[pid] = Some(SpillFile { _temp_file: temp_file, file, @@ -545,9 +536,7 @@ impl ShufflePartitioner for ImmediateModePartitioner { .create(true) .truncate(true) .open(&self.output_data_file) - .map_err(|e| { - DataFusionError::Execution(format!("shuffle write error: {e:?}")) - })?; + .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {e:?}")))?; let mut output_data = BufWriter::new(data_file); #[allow(clippy::needless_range_loop)] @@ -588,9 +577,8 @@ impl ShufflePartitioner for ImmediateModePartitioner { // Write index file let mut write_timer = self.metrics.write_time.timer(); let mut output_index = BufWriter::new( - File::create(&self.output_index_file).map_err(|e| { - DataFusionError::Execution(format!("shuffle write error: {e:?}")) - })?, + File::create(&self.output_index_file) + .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {e:?}")))?, ); for offset in &offsets { output_index.write_all(&offset.to_le_bytes())?; @@ -636,8 +624,7 @@ mod tests { CompressionCodec::Snappy, ] { // Use batch_size=1 to force immediate serialization (no coalescing) - let mut stream = - PartitionOutputStream::try_new(Arc::clone(&schema), codec, 1).unwrap(); + let mut stream = PartitionOutputStream::try_new(Arc::clone(&schema), codec, 1).unwrap(); stream.write_batch(&batch).unwrap(); stream.flush().unwrap(); @@ -645,12 +632,10 @@ mod tests { assert!(!buf.is_empty()); // Parse the first block: 8 bytes length, 8 bytes field_count, then codec+data - let ipc_length = - u64::from_le_bytes(buf[0..8].try_into().unwrap()) as usize; + let ipc_length = u64::from_le_bytes(buf[0..8].try_into().unwrap()) as usize; assert!(ipc_length > 0); - let field_count = - usize::from_le_bytes(buf[8..16].try_into().unwrap()); + let field_count = usize::from_le_bytes(buf[8..16].try_into().unwrap()); assert_eq!(field_count, 1); // one field "a" // read_ipc_compressed expects data starting at the codec tag @@ -905,8 +890,7 @@ mod tests { let index_data = std::fs::read(&index_path).unwrap(); let mut offsets = Vec::new(); for i in 0..=num_partitions { - let offset = - i64::from_le_bytes(index_data[i * 8..(i + 1) * 8].try_into().unwrap()); + let offset = i64::from_le_bytes(index_data[i * 8..(i + 1) * 8].try_into().unwrap()); offsets.push(offset as usize); } @@ -925,15 +909,13 @@ mod tests { let mut pos = partition_start; while pos < partition_end { // Read 8-byte length prefix - let payload_len = u64::from_le_bytes( - data[pos..pos + 8].try_into().unwrap(), - ) as usize; + let payload_len = + u64::from_le_bytes(data[pos..pos + 8].try_into().unwrap()) as usize; assert!(payload_len > 0, "Block payload length should be > 0"); // Skip 8-byte field_count - let field_count = u64::from_le_bytes( - data[pos + 8..pos + 16].try_into().unwrap(), - ) as usize; + let field_count = + u64::from_le_bytes(data[pos + 8..pos + 16].try_into().unwrap()) as usize; assert_eq!(field_count, 1, "Expected 1 field"); // Pass codec tag + IPC data to read_ipc_compressed diff --git a/native/shuffle/src/partitioners/mod.rs b/native/shuffle/src/partitioners/mod.rs index 18b46813d3..a47666b2a7 100644 --- a/native/shuffle/src/partitioners/mod.rs +++ b/native/shuffle/src/partitioners/mod.rs @@ -15,11 +15,11 @@ // specific language governing permissions and limitations // under the License. +mod immediate_mode; mod multi_partition; mod partitioned_batch_iterator; mod single_partition; mod traits; -mod immediate_mode; pub(crate) use immediate_mode::ImmediateModePartitioner; pub(crate) use multi_partition::MultiPartitionShuffleRepartitioner; diff --git a/native/shuffle/src/shuffle_writer.rs b/native/shuffle/src/shuffle_writer.rs index c6602217a1..ca1e886737 100644 --- a/native/shuffle/src/shuffle_writer.rs +++ b/native/shuffle/src/shuffle_writer.rs @@ -234,19 +234,17 @@ async fn external_shuffle( write_buffer_size, )?) } - _ if immediate_mode => { - Box::new(ImmediateModePartitioner::try_new( - partition, - output_data_file, - output_index_file, - Arc::clone(&schema), - partitioning, - metrics, - context.runtime_env(), - context.session_config().batch_size(), - codec, - )?) - } + _ if immediate_mode => Box::new(ImmediateModePartitioner::try_new( + partition, + output_data_file, + output_index_file, + Arc::clone(&schema), + partitioning, + metrics, + context.runtime_env(), + context.session_config().batch_size(), + codec, + )?), _ => Box::new(MultiPartitionShuffleRepartitioner::try_new( partition, output_data_file, From 76815a9d51f554dfbc4c4bb12a4b97c91e2b5776 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 30 Mar 2026 17:47:16 -0600 Subject: [PATCH 21/46] refactor: remove unnecessary batch slicing and BufReader in immediate mode The slicing loop in insert_batch was redundant because BatchCoalescer already handles chunking. Pass the batch directly. Also remove the BufReader wrapper around spill files since std::io::copy handles buffering internally. --- .../src/partitioners/immediate_mode.rs | 29 +++++++------------ 1 file changed, 10 insertions(+), 19 deletions(-) diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index 0ced1b77a4..728fe31f83 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -28,7 +28,7 @@ use datafusion::execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion_comet_spark_expr::murmur3::create_murmur3_hashes; use std::fs::{File, OpenOptions}; -use std::io::{BufReader, BufWriter, Seek, Write}; +use std::io::{BufWriter, Seek, Write}; use std::sync::Arc; use tokio::time::Instant; @@ -497,24 +497,16 @@ impl ShufflePartitioner for ImmediateModePartitioner { self.metrics.data_size.add(batch.get_array_memory_size()); self.metrics.baseline.record_output(batch.num_rows()); - let mut start = 0; - while start < batch.num_rows() { - let end = (start + self.batch_size).min(batch.num_rows()); - let chunk = batch.slice(start, end - start); - - let repart_start = Instant::now(); - self.compute_partition_ids(&chunk)?; - self.metrics - .repart_time - .add_duration(repart_start.elapsed()); - - let bytes_written = self.write_partitioned_rows(&chunk)?; + let repart_start = Instant::now(); + self.compute_partition_ids(&batch)?; + self.metrics + .repart_time + .add_duration(repart_start.elapsed()); - if self.reservation.try_grow(bytes_written).is_err() { - self.spill_all()?; - } + let bytes_written = self.write_partitioned_rows(&batch)?; - start = end; + if self.reservation.try_grow(bytes_written).is_err() { + self.spill_all()?; } self.metrics.input_batches.add(1); @@ -551,9 +543,8 @@ impl ShufflePartitioner for ImmediateModePartitioner { "Failed to open spill file for reading: {e}" )) })?; - let mut reader = BufReader::new(spill_reader); let mut write_timer = self.metrics.write_time.timer(); - std::io::copy(&mut reader, &mut output_data)?; + std::io::copy(&mut &spill_reader, &mut output_data)?; write_timer.stop(); } From 42289ee2b8505ebbdc209d4ae23b31ea8ae210d3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 30 Mar 2026 18:03:04 -0600 Subject: [PATCH 22/46] refactor: remove BufWriter from shuffle output and unused batch_size field --- native/shuffle/src/partitioners/immediate_mode.rs | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index 728fe31f83..8cd6eee56e 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -253,7 +253,6 @@ pub(crate) struct ImmediateModePartitioner { metrics: ShufflePartitionerMetrics, hashes_buf: Vec, partition_ids: Vec, - batch_size: usize, /// Fixed upfront reservation for coalescer buffers across all partitions. /// Each coalescer holds at most batch_size rows; this budget is estimated /// from the schema and reserved once to avoid incremental drift. @@ -313,7 +312,6 @@ impl ImmediateModePartitioner { metrics, hashes_buf, partition_ids: vec![0u32; batch_size], - batch_size, coalescer_budget, }) } @@ -523,13 +521,12 @@ impl ShufflePartitioner for ImmediateModePartitioner { let num_output_partitions = self.streams.len(); let mut offsets = vec![0i64; num_output_partitions + 1]; - let data_file = OpenOptions::new() + let mut output_data = OpenOptions::new() .write(true) .create(true) .truncate(true) .open(&self.output_data_file) .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {e:?}")))?; - let mut output_data = BufWriter::new(data_file); #[allow(clippy::needless_range_loop)] for pid in 0..num_output_partitions { @@ -558,10 +555,6 @@ impl ShufflePartitioner for ImmediateModePartitioner { } } - let mut write_timer = self.metrics.write_time.timer(); - output_data.flush()?; - write_timer.stop(); - // Record final offset offsets[num_output_partitions] = output_data.stream_position()? as i64; From 27484ce828435f2ecf5f85b76241c02a48892a1e Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 30 Mar 2026 21:01:41 -0600 Subject: [PATCH 23/46] fix: improve memory tracking and cleanup in immediate mode partitioner MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - Remove 1MB pre-allocation per partition buffer (Vec::new instead of Vec::with_capacity), eliminating num_partitions × 1MB of untracked heap allocation - Track buffer Vec capacity growth instead of content bytes written, giving the memory pool accurate visibility into actual allocations - Drop spill files immediately after combining into final output rather than waiting for partitioner drop - Delete benchmark output files after each iteration to avoid filling disk during multi-iteration runs --- native/shuffle/src/bin/shuffle_bench.rs | 4 + .../src/partitioners/immediate_mode.rs | 92 ++++++++++--------- 2 files changed, 52 insertions(+), 44 deletions(-) diff --git a/native/shuffle/src/bin/shuffle_bench.rs b/native/shuffle/src/bin/shuffle_bench.rs index 21228af87c..d1f85b409f 100644 --- a/native/shuffle/src/bin/shuffle_bench.rs +++ b/native/shuffle/src/bin/shuffle_bench.rs @@ -228,6 +228,10 @@ fn main() { print!(" read: {:.3}s", read_elapsed); } println!(); + + // Remove output files after each iteration to avoid filling disk + let _ = fs::remove_file(&data_file); + let _ = fs::remove_file(&index_file); } if args.iterations > 0 { diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index 8cd6eee56e..854bd11a36 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -32,14 +32,14 @@ use std::io::{BufWriter, Seek, Write}; use std::sync::Arc; use tokio::time::Instant; -/// Estimate the in-memory byte cost per row for a given schema. -/// Uses the primitive width for fixed-size types, and a heuristic for -/// variable-length types (strings, binary, nested). -fn estimate_row_bytes(schema: &SchemaRef) -> usize { +/// Estimate the in-memory byte cost of BatchCoalescer's InProgressArray builders +/// per row for a given schema. Uses primitive widths for fixed-size types and a +/// heuristic for variable-length types. +fn estimate_builder_bytes_per_row(schema: &SchemaRef) -> usize { let mut total = 0; for field in schema.fields() { total += estimate_field_bytes(field.data_type()); - // 1 bit for null bitmap, round up + // 1 byte for null bitmap (overestimate per row, but simple) total += 1; } total.max(8) // minimum 8 bytes per row @@ -50,18 +50,15 @@ fn estimate_field_bytes(dt: &DataType) -> usize { DataType::Boolean => 1, DataType::Null => 0, dt if dt.primitive_width().is_some() => dt.primitive_width().unwrap(), - // Variable-length types: use a reasonable default estimate. - // 32 bytes covers typical short strings/binary. - DataType::Utf8 | DataType::Binary => 32 + 4, // data + 4-byte offset + DataType::Utf8 | DataType::Binary => 32 + 4, DataType::LargeUtf8 | DataType::LargeBinary => 32 + 8, - DataType::Utf8View | DataType::BinaryView => 32 + 16, // data + 16-byte view + DataType::Utf8View | DataType::BinaryView => 32 + 16, DataType::List(f) | DataType::LargeList(f) => 4 + estimate_field_bytes(f.data_type()), DataType::Struct(fields) => fields .iter() .map(|f| estimate_field_bytes(f.data_type()) + 1) .sum(), DataType::Dictionary(key_dt, _) => estimate_field_bytes(key_dt), - // Fallback for other types _ => 32, } } @@ -93,17 +90,17 @@ impl PartitionOutputStream { Ok(Self { schema, codec, - buffer: Vec::with_capacity(1024 * 1024), + buffer: Vec::new(), coalescer: None, batch_size, }) } /// Push a batch into the coalescer, serializing any completed batches as - /// length-prefixed compressed IPC blocks. Returns total bytes written. - fn write_batch(&mut self, batch: &RecordBatch) -> Result { + /// length-prefixed compressed IPC blocks. + fn write_batch(&mut self, batch: &RecordBatch) -> Result<()> { if batch.num_rows() == 0 { - return Ok(0); + return Ok(()); } let coalescer = self @@ -116,17 +113,15 @@ impl PartitionOutputStream { completed.push(batch); } - let mut total_bytes = 0; for batch in &completed { - total_bytes += self.write_ipc_block(batch)?; + self.write_ipc_block(batch)?; } - Ok(total_bytes) + Ok(()) } /// Flush any remaining rows in the coalescer as a final IPC block. /// Must be called before `drain_buffer` or `finish` to avoid losing data. - fn flush(&mut self) -> Result { - let mut total_bytes = 0; + fn flush(&mut self) -> Result<()> { if let Some(coalescer) = &mut self.coalescer { coalescer.finish_buffered_batch()?; let mut remaining = Vec::new(); @@ -134,10 +129,10 @@ impl PartitionOutputStream { remaining.push(batch); } for batch in &remaining { - total_bytes += self.write_ipc_block(batch)?; + self.write_ipc_block(batch)?; } } - Ok(total_bytes) + Ok(()) } /// Serialize a single record batch as a length-prefixed compressed IPC block. @@ -216,6 +211,11 @@ impl PartitionOutputStream { Ok(end_pos - start_pos) } + /// Returns the buffer's allocated capacity in bytes. + fn buffer_capacity(&self) -> usize { + self.buffer.capacity() + } + /// Returns the number of bytes currently in the buffer. #[cfg(test)] fn buffered_bytes(&self) -> usize { @@ -253,9 +253,9 @@ pub(crate) struct ImmediateModePartitioner { metrics: ShufflePartitionerMetrics, hashes_buf: Vec, partition_ids: Vec, - /// Fixed upfront reservation for coalescer buffers across all partitions. - /// Each coalescer holds at most batch_size rows; this budget is estimated - /// from the schema and reserved once to avoid incremental drift. + /// Reservation for BatchCoalescer InProgressArray builders across all partitions. + /// These builders are pre-allocated when each coalescer is first used and persist + /// across spills, so this budget is held as a fixed portion of the reservation. coalescer_budget: usize, } @@ -288,17 +288,16 @@ impl ImmediateModePartitioner { _ => vec![], }; - let mut reservation = MemoryConsumer::new(format!("ImmediateModePartitioner[{partition}]")) - .with_can_spill(true) - .register(&runtime.memory_pool); + let mut reservation = + MemoryConsumer::new(format!("ImmediateModePartitioner[{partition}]")) + .with_can_spill(true) + .register(&runtime.memory_pool); - // Reserve memory upfront for coalescer buffers across all partitions. - // Each coalescer holds at most batch_size rows. We estimate per-row bytes - // from the schema so this budget is fixed and doesn't drift. - let estimated_row_bytes = estimate_row_bytes(&schema); + // Reserve memory for BatchCoalescer InProgressArray builders. + // Each coalescer pre-allocates batch_size rows of builder capacity per column. + // This is approximately constant once initialized and persists across spills. + let estimated_row_bytes = estimate_builder_bytes_per_row(&schema); let coalescer_budget = estimated_row_bytes * batch_size * num_output_partitions; - // Use try_grow — if the pool can't accommodate the budget, we proceed - // without the reservation (spilling will be more aggressive). let _ = reservation.try_grow(coalescer_budget); Ok(Self { @@ -392,7 +391,7 @@ impl ImmediateModePartitioner { /// Route rows to their partition streams using interleave, then write IPC blocks. /// Returns total bytes written across all partitions. - fn write_partitioned_rows(&mut self, batch: &RecordBatch) -> Result { + fn write_partitioned_rows(&mut self, batch: &RecordBatch) -> Result<()> { let num_partitions = self.streams.len(); let num_rows = batch.num_rows(); @@ -404,7 +403,6 @@ impl ImmediateModePartitioner { } let batch_refs = [batch]; - let mut total_bytes = 0; let mut interleave_timer = self.metrics.interleave_time.timer(); for (pid, indices) in partition_row_indices.iter().enumerate() { @@ -416,15 +414,14 @@ impl ImmediateModePartitioner { interleave_timer.stop(); let mut encode_timer = self.metrics.encode_time.timer(); - let bytes = self.streams[pid].write_batch(&sub_batch)?; + self.streams[pid].write_batch(&sub_batch)?; encode_timer.stop(); - total_bytes += bytes; interleave_timer = self.metrics.interleave_time.timer(); } interleave_timer.stop(); - Ok(total_bytes) + Ok(()) } /// Spill all partition buffers to per-partition temp files. @@ -468,8 +465,8 @@ impl ImmediateModePartitioner { spill.file.flush()?; } - // Shrink reservation to just the coalescer budget (IPC bytes were spilled, - // coalescer is empty after flush but will be refilled by future batches). + // Shrink reservation to coalescer_budget — buffer memory was spilled + // but the BatchCoalescer InProgressArray builders persist across spills. let target = self.coalescer_budget; if self.reservation.size() > target { self.reservation.shrink(self.reservation.size() - target); @@ -501,9 +498,12 @@ impl ShufflePartitioner for ImmediateModePartitioner { .repart_time .add_duration(repart_start.elapsed()); - let bytes_written = self.write_partitioned_rows(&batch)?; + let capacity_before: usize = self.streams.iter().map(|s| s.buffer_capacity()).sum(); + self.write_partitioned_rows(&batch)?; + let capacity_after: usize = self.streams.iter().map(|s| s.buffer_capacity()).sum(); + let capacity_growth = capacity_after.saturating_sub(capacity_before); - if self.reservation.try_grow(bytes_written).is_err() { + if capacity_growth > 0 && self.reservation.try_grow(capacity_growth).is_err() { self.spill_all()?; } @@ -555,6 +555,11 @@ impl ShufflePartitioner for ImmediateModePartitioner { } } + // Drop spill files now that their contents have been copied to the output + for spill in self.spill_files.iter_mut() { + *spill = None; + } + // Record final offset offsets[num_output_partitions] = output_data.stream_position()? as i64; @@ -665,8 +670,7 @@ mod tests { let mut stream = PartitionOutputStream::try_new(schema, CompressionCodec::None, 8192).unwrap(); - let bytes_written = stream.write_batch(&batch).unwrap(); - assert_eq!(bytes_written, 0); + stream.write_batch(&batch).unwrap(); assert_eq!(stream.buffered_bytes(), 0); } From aa49a72ef02d18d18994b6564c3f8f6e04517470 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 31 Mar 2026 09:18:09 -0600 Subject: [PATCH 24/46] feat: scatter-write partitioner for immediate mode shuffle Replace interleave_record_batch + BatchCoalescer pipeline with a Gluten-inspired scatter-write design: - Pre-allocate Arrow array builders per partition - Scatter-write rows directly from input batches into partition builders - Flush full partitions as compressed IPC blocks, reusing builder capacity - Track builder + IPC memory growth for spill decisions This eliminates per-batch intermediate array allocations (previously 200 interleave calls per input batch) and coalescer builder overhead. --- .../src/partitioners/immediate_mode.rs | 713 ++++++++++-------- 1 file changed, 385 insertions(+), 328 deletions(-) diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index e86c10014f..1cd7a40aa5 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -18,13 +18,21 @@ use crate::metrics::ShufflePartitionerMetrics; use crate::partitioners::ShufflePartitioner; use crate::{comet_partitioning, CometPartitioning, CompressionCodec}; -use arrow::array::{ArrayRef, RecordBatch}; -use arrow::compute::interleave_record_batch; -use arrow::compute::kernels::coalesce::BatchCoalescer; -use arrow::datatypes::{DataType, SchemaRef}; +use arrow::array::builder::{ + make_builder, ArrayBuilder, BinaryBuilder, BinaryViewBuilder, BooleanBuilder, + LargeBinaryBuilder, LargeStringBuilder, NullBuilder, PrimitiveBuilder, StringBuilder, + StringViewBuilder, +}; +use arrow::array::{Array, ArrayRef, AsArray, BinaryViewArray, RecordBatch, StringViewArray}; +use arrow::datatypes::{ + DataType, Date32Type, Date64Type, Decimal128Type, Decimal256Type, Float32Type, Float64Type, + Int16Type, Int32Type, Int64Type, Int8Type, SchemaRef, TimeUnit, TimestampMicrosecondType, + TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, UInt16Type, UInt32Type, + UInt64Type, UInt8Type, +}; use arrow::ipc::writer::StreamWriter; use datafusion::common::{DataFusionError, Result}; -use datafusion::execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use datafusion::execution::memory_pool::{MemoryConsumer, MemoryLimit, MemoryReservation}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion_comet_spark_expr::murmur3::create_murmur3_hashes; use std::fs::{File, OpenOptions}; @@ -32,122 +40,219 @@ use std::io::{BufWriter, Seek, Write}; use std::sync::Arc; use tokio::time::Instant; -/// Estimate the in-memory byte cost of BatchCoalescer's InProgressArray builders -/// per row for a given schema. Uses primitive widths for fixed-size types and a -/// heuristic for variable-length types. -fn estimate_builder_bytes_per_row(schema: &SchemaRef) -> usize { - let mut total = 0; - for field in schema.fields() { - total += estimate_field_bytes(field.data_type()); - // 1 byte for null bitmap (overestimate per row, but simple) - total += 1; +macro_rules! scatter_byte_array { + ($builder:expr, $source:expr, $indices:expr, $offset_type:ty, $builder_type:ty, $cast:ident) => {{ + let src = $source.$cast::<$offset_type>(); + let dst = $builder + .as_any_mut() + .downcast_mut::<$builder_type>() + .expect("builder type mismatch"); + if src.null_count() == 0 { + for &idx in $indices { + dst.append_value(src.value(idx)); + } + } else { + for &idx in $indices { + dst.append_option(src.is_valid(idx).then(|| src.value(idx))); + } + } + }}; +} + +macro_rules! scatter_byte_view { + ($builder:expr, $source:expr, $indices:expr, $array_type:ty, $builder_type:ty) => {{ + let src = $source + .as_any() + .downcast_ref::<$array_type>() + .expect("array type mismatch"); + let dst = $builder + .as_any_mut() + .downcast_mut::<$builder_type>() + .expect("builder type mismatch"); + if src.null_count() == 0 { + for &idx in $indices { + dst.append_value(src.value(idx)); + } + } else { + for &idx in $indices { + dst.append_option(src.is_valid(idx).then(|| src.value(idx))); + } + } + }}; +} + +macro_rules! scatter_primitive { + ($builder:expr, $source:expr, $indices:expr, $arrow_type:ty) => {{ + let src = $source.as_primitive::<$arrow_type>(); + let dst = $builder + .as_any_mut() + .downcast_mut::>() + .expect("builder type mismatch"); + if src.null_count() == 0 { + for &idx in $indices { + dst.append_value(src.value(idx)); + } + } else { + for &idx in $indices { + dst.append_option(src.is_valid(idx).then(|| src.value(idx))); + } + } + }}; +} + +/// Scatter-append selected rows from `source` into `builder`. +fn scatter_append( + builder: &mut dyn ArrayBuilder, + source: &dyn Array, + indices: &[usize], +) -> Result<()> { + use DataType::*; + match source.data_type() { + Boolean => { + let src = source.as_boolean(); + let dst = builder + .as_any_mut() + .downcast_mut::() + .unwrap(); + if src.null_count() == 0 { + for &idx in indices { + dst.append_value(src.value(idx)); + } + } else { + for &idx in indices { + dst.append_option(src.is_valid(idx).then(|| src.value(idx))); + } + } + } + Int8 => scatter_primitive!(builder, source, indices, Int8Type), + Int16 => scatter_primitive!(builder, source, indices, Int16Type), + Int32 => scatter_primitive!(builder, source, indices, Int32Type), + Int64 => scatter_primitive!(builder, source, indices, Int64Type), + UInt8 => scatter_primitive!(builder, source, indices, UInt8Type), + UInt16 => scatter_primitive!(builder, source, indices, UInt16Type), + UInt32 => scatter_primitive!(builder, source, indices, UInt32Type), + UInt64 => scatter_primitive!(builder, source, indices, UInt64Type), + Float32 => scatter_primitive!(builder, source, indices, Float32Type), + Float64 => scatter_primitive!(builder, source, indices, Float64Type), + Date32 => scatter_primitive!(builder, source, indices, Date32Type), + Date64 => scatter_primitive!(builder, source, indices, Date64Type), + Timestamp(TimeUnit::Second, _) => { + scatter_primitive!(builder, source, indices, TimestampSecondType) + } + Timestamp(TimeUnit::Millisecond, _) => { + scatter_primitive!(builder, source, indices, TimestampMillisecondType) + } + Timestamp(TimeUnit::Microsecond, _) => { + scatter_primitive!(builder, source, indices, TimestampMicrosecondType) + } + Timestamp(TimeUnit::Nanosecond, _) => { + scatter_primitive!(builder, source, indices, TimestampNanosecondType) + } + Decimal128(_, _) => scatter_primitive!(builder, source, indices, Decimal128Type), + Decimal256(_, _) => scatter_primitive!(builder, source, indices, Decimal256Type), + Utf8 => scatter_byte_array!(builder, source, indices, i32, StringBuilder, as_string), + LargeUtf8 => { + scatter_byte_array!(builder, source, indices, i64, LargeStringBuilder, as_string) + } + Binary => scatter_byte_array!(builder, source, indices, i32, BinaryBuilder, as_binary), + LargeBinary => { + scatter_byte_array!(builder, source, indices, i64, LargeBinaryBuilder, as_binary) + } + Utf8View => { + scatter_byte_view!(builder, source, indices, StringViewArray, StringViewBuilder) + } + BinaryView => { + scatter_byte_view!(builder, source, indices, BinaryViewArray, BinaryViewBuilder) + } + Null => { + let dst = builder.as_any_mut().downcast_mut::().unwrap(); + dst.append_nulls(indices.len()); + } + dt => { + return Err(DataFusionError::NotImplemented(format!( + "Scatter append not implemented for {dt}" + ))); + } } - total.max(8) // minimum 8 bytes per row + Ok(()) } -fn estimate_field_bytes(dt: &DataType) -> usize { - match dt { - DataType::Boolean => 1, - DataType::Null => 0, - dt if dt.primitive_width().is_some() => dt.primitive_width().unwrap(), - DataType::Utf8 | DataType::Binary => 32 + 4, - DataType::LargeUtf8 | DataType::LargeBinary => 32 + 8, - DataType::Utf8View | DataType::BinaryView => 32 + 16, - DataType::List(f) | DataType::LargeList(f) => 4 + estimate_field_bytes(f.data_type()), - DataType::Struct(fields) => fields +struct PartitionBuffer { + builders: Vec>, + schema: SchemaRef, + num_rows: usize, + target_batch_size: usize, +} + +impl PartitionBuffer { + fn new(schema: &SchemaRef, target_batch_size: usize) -> Self { + let builders = schema + .fields() .iter() - .map(|f| estimate_field_bytes(f.data_type()) + 1) - .sum(), - DataType::Dictionary(key_dt, _) => estimate_field_bytes(key_dt), - _ => 32, + .map(|f| make_builder(f.data_type(), target_batch_size)) + .collect(); + Self { + builders, + schema: Arc::clone(schema), + num_rows: 0, + target_batch_size, + } + } + + /// Scatter-write selected rows from `batch` into this partition's builders. + fn append_rows(&mut self, batch: &RecordBatch, row_indices: &[usize]) -> Result<()> { + for (col_idx, builder) in self.builders.iter_mut().enumerate() { + scatter_append( + builder.as_mut(), + batch.column(col_idx).as_ref(), + row_indices, + )?; + } + self.num_rows += row_indices.len(); + Ok(()) + } + + fn is_full(&self) -> bool { + self.num_rows >= self.target_batch_size + } + + /// Finish builders into a RecordBatch. Builders are reset but retain + /// their allocated capacity for reuse. + fn flush(&mut self) -> Result { + let arrays: Vec = self.builders.iter_mut().map(|b| b.finish()).collect(); + let batch = RecordBatch::try_new(Arc::clone(&self.schema), arrays) + .map_err(|e| DataFusionError::ArrowError(Box::new(e), None))?; + self.num_rows = 0; + Ok(batch) + } + + fn has_data(&self) -> bool { + self.num_rows > 0 } } -/// Per-partition output stream that serializes Arrow IPC batches into an -/// in-memory buffer with compression. The block format matches -/// `ShuffleBlockWriter::write_batch` exactly: -/// -/// - 8 bytes: payload length (u64 LE) — total bytes after this prefix -/// - 8 bytes: field_count (usize LE) -/// - 4 bytes: codec tag (b"SNAP", b"LZ4_", b"ZSTD", or b"NONE") -/// - N bytes: compressed Arrow IPC stream data pub(crate) struct PartitionOutputStream { schema: SchemaRef, codec: CompressionCodec, buffer: Vec, - /// Coalesces small batches into target_batch_size before serialization, - /// reducing per-block IPC schema overhead. Lazily initialized on first write. - coalescer: Option, - batch_size: usize, } impl PartitionOutputStream { - pub(crate) fn try_new( - schema: SchemaRef, - codec: CompressionCodec, - batch_size: usize, - ) -> Result { + pub(crate) fn try_new(schema: SchemaRef, codec: CompressionCodec) -> Result { Ok(Self { schema, codec, buffer: Vec::new(), - coalescer: None, - batch_size, }) } - /// Push a batch into the coalescer, serializing any completed batches as - /// length-prefixed compressed IPC blocks. - fn write_batch(&mut self, batch: &RecordBatch) -> Result<()> { - if batch.num_rows() == 0 { - return Ok(()); - } - - let coalescer = self - .coalescer - .get_or_insert_with(|| BatchCoalescer::new(batch.schema(), self.batch_size)); - coalescer.push_batch(batch.clone())?; - - let mut completed = Vec::new(); - while let Some(batch) = coalescer.next_completed_batch() { - completed.push(batch); - } - - for batch in &completed { - self.write_ipc_block(batch)?; - } - Ok(()) - } - - /// Flush any remaining rows in the coalescer as a final IPC block. - /// Must be called before `drain_buffer` or `finish` to avoid losing data. - fn flush(&mut self) -> Result<()> { - if let Some(coalescer) = &mut self.coalescer { - coalescer.finish_buffered_batch()?; - let mut remaining = Vec::new(); - while let Some(batch) = coalescer.next_completed_batch() { - remaining.push(batch); - } - for batch in &remaining { - self.write_ipc_block(batch)?; - } - } - Ok(()) - } - - /// Serialize a single record batch as a length-prefixed compressed IPC block. fn write_ipc_block(&mut self, batch: &RecordBatch) -> Result { let start_pos = self.buffer.len(); - // Write 8-byte placeholder for length prefix self.buffer.extend_from_slice(&0u64.to_le_bytes()); - - // Write field count (8 bytes, usize LE) let field_count = self.schema.fields().len(); self.buffer .extend_from_slice(&(field_count as u64).to_le_bytes()); - - // Write codec tag (4 bytes) let codec_tag: &[u8; 4] = match &self.codec { CompressionCodec::Snappy => b"SNAP", CompressionCodec::Lz4Frame => b"LZ4_", @@ -156,78 +261,57 @@ impl PartitionOutputStream { }; self.buffer.extend_from_slice(codec_tag); - // Write compressed IPC data match &self.codec { CompressionCodec::None => { - let mut arrow_writer = StreamWriter::try_new(&mut self.buffer, &batch.schema())?; - arrow_writer.write(batch)?; - arrow_writer.finish()?; - // StreamWriter::into_inner returns the inner writer; we don't need it - // since we're writing directly to self.buffer - arrow_writer.into_inner()?; + let mut w = StreamWriter::try_new(&mut self.buffer, &batch.schema())?; + w.write(batch)?; + w.finish()?; + w.into_inner()?; } CompressionCodec::Lz4Frame => { let mut wtr = lz4_flex::frame::FrameEncoder::new(&mut self.buffer); - let mut arrow_writer = StreamWriter::try_new(&mut wtr, &batch.schema())?; - arrow_writer.write(batch)?; - arrow_writer.finish()?; + let mut w = StreamWriter::try_new(&mut wtr, &batch.schema())?; + w.write(batch)?; + w.finish()?; wtr.finish().map_err(|e| { DataFusionError::Execution(format!("lz4 compression error: {e}")) })?; } CompressionCodec::Zstd(level) => { - let encoder = zstd::Encoder::new(&mut self.buffer, *level)?; - let mut arrow_writer = StreamWriter::try_new(encoder, &batch.schema())?; - arrow_writer.write(batch)?; - arrow_writer.finish()?; - let zstd_encoder = arrow_writer.into_inner()?; - zstd_encoder.finish()?; + let enc = zstd::Encoder::new(&mut self.buffer, *level)?; + let mut w = StreamWriter::try_new(enc, &batch.schema())?; + w.write(batch)?; + w.finish()?; + w.into_inner()?.finish()?; } CompressionCodec::Snappy => { let mut wtr = snap::write::FrameEncoder::new(&mut self.buffer); - let mut arrow_writer = StreamWriter::try_new(&mut wtr, &batch.schema())?; - arrow_writer.write(batch)?; - arrow_writer.finish()?; + let mut w = StreamWriter::try_new(&mut wtr, &batch.schema())?; + w.write(batch)?; + w.finish()?; wtr.into_inner().map_err(|e| { DataFusionError::Execution(format!("snappy compression error: {e}")) })?; } } - // Backfill length prefix: total bytes after the 8-byte length field let end_pos = self.buffer.len(); let ipc_length = (end_pos - start_pos - 8) as u64; - - let max_size = i32::MAX as u64; - if ipc_length > max_size { + if ipc_length > i32::MAX as u64 { return Err(DataFusionError::Execution(format!( - "Shuffle block size {ipc_length} exceeds maximum size of {max_size}. \ - Try reducing batch size or increasing compression level" + "Shuffle block size {ipc_length} exceeds maximum size of {}", + i32::MAX ))); } - self.buffer[start_pos..start_pos + 8].copy_from_slice(&ipc_length.to_le_bytes()); Ok(end_pos - start_pos) } - /// Returns the buffer's allocated capacity in bytes. - fn buffer_capacity(&self) -> usize { - self.buffer.capacity() - } - - /// Returns the number of bytes currently in the buffer. - #[cfg(test)] - fn buffered_bytes(&self) -> usize { - self.buffer.len() - } - - /// Takes the buffer contents, leaving the buffer empty. fn drain_buffer(&mut self) -> Vec { std::mem::take(&mut self.buffer) } - /// Consumes self and returns the buffer. #[cfg(test)] fn finish(self) -> Result> { Ok(self.buffer) @@ -239,12 +323,14 @@ struct SpillFile { file: File, } -/// A partitioner that immediately writes IPC blocks per partition as batches arrive, -/// rather than buffering all data until shuffle_write. Supports spilling per-partition -/// buffers to disk under memory pressure. +/// A partitioner that scatter-writes incoming rows directly into pre-allocated +/// per-partition column builders. When a partition's builders reach +/// `target_batch_size`, the batch is flushed to a compressed IPC block. +/// No intermediate sub-batches or coalescers are created. pub(crate) struct ImmediateModePartitioner { output_data_file: String, output_index_file: String, + partition_buffers: Vec, streams: Vec, spill_files: Vec>, partitioning: CometPartitioning, @@ -253,10 +339,11 @@ pub(crate) struct ImmediateModePartitioner { metrics: ShufflePartitionerMetrics, hashes_buf: Vec, partition_ids: Vec, - /// Reservation for BatchCoalescer InProgressArray builders across all partitions. - /// These builders are pre-allocated when each coalescer is first used and persist - /// across spills, so this budget is held as a fixed portion of the reservation. - coalescer_budget: usize, + /// Reusable per-partition row index scratch space. + partition_row_indices: Vec>, + /// Maximum bytes this partitioner will reserve from the memory pool. + /// Computed as memory_pool_size * memory_fraction at construction. + memory_limit: usize, } impl ImmediateModePartitioner { @@ -274,8 +361,12 @@ impl ImmediateModePartitioner { ) -> Result { let num_output_partitions = partitioning.partition_count(); + let partition_buffers = (0..num_output_partitions) + .map(|_| PartitionBuffer::new(&schema, batch_size)) + .collect(); + let streams = (0..num_output_partitions) - .map(|_| PartitionOutputStream::try_new(Arc::clone(&schema), codec.clone(), batch_size)) + .map(|_| PartitionOutputStream::try_new(Arc::clone(&schema), codec.clone())) .collect::>>()?; let spill_files: Vec> = @@ -288,20 +379,21 @@ impl ImmediateModePartitioner { _ => vec![], }; - let mut reservation = MemoryConsumer::new(format!("ImmediateModePartitioner[{partition}]")) + let memory_limit = match runtime.memory_pool.memory_limit() { + MemoryLimit::Finite(pool_size) => pool_size, + _ => usize::MAX, + }; + + let reservation = MemoryConsumer::new(format!("ImmediateModePartitioner[{partition}]")) .with_can_spill(true) .register(&runtime.memory_pool); - // Reserve memory for BatchCoalescer InProgressArray builders. - // Each coalescer pre-allocates batch_size rows of builder capacity per column. - // This is approximately constant once initialized and persists across spills. - let estimated_row_bytes = estimate_builder_bytes_per_row(&schema); - let coalescer_budget = estimated_row_bytes * batch_size * num_output_partitions; - let _ = reservation.try_grow(coalescer_budget); + let partition_row_indices = (0..num_output_partitions).map(|_| Vec::new()).collect(); Ok(Self { output_data_file, output_index_file, + partition_buffers, streams, spill_files, partitioning, @@ -310,14 +402,22 @@ impl ImmediateModePartitioner { metrics, hashes_buf, partition_ids: vec![0u32; batch_size], - coalescer_budget, + partition_row_indices, + memory_limit, }) } - /// Compute partition IDs for each row in the batch, storing results in - /// `self.partition_ids`. Returns the number of output partitions. fn compute_partition_ids(&mut self, batch: &RecordBatch) -> Result { let num_rows = batch.num_rows(); + + // Ensure scratch buffers are large enough for this batch + if self.hashes_buf.len() < num_rows { + self.hashes_buf.resize(num_rows, 0); + } + if self.partition_ids.len() < num_rows { + self.partition_ids.resize(num_rows, 0); + } + match &self.partitioning { CometPartitioning::Hash(exprs, num_output_partitions) => { let num_output_partitions = *num_output_partitions; @@ -325,11 +425,9 @@ impl ImmediateModePartitioner { .iter() .map(|expr| expr.evaluate(batch)?.into_array(num_rows)) .collect::>>()?; - let hashes_buf = &mut self.hashes_buf[..num_rows]; hashes_buf.fill(42_u32); create_murmur3_hashes(&arrays, hashes_buf)?; - let partition_ids = &mut self.partition_ids[..num_rows]; for (idx, hash) in hashes_buf.iter().enumerate() { partition_ids[idx] = @@ -348,11 +446,9 @@ impl ImmediateModePartitioner { let columns_to_hash: Vec = (0..num_columns_to_hash) .map(|i| Arc::clone(batch.column(i))) .collect(); - let hashes_buf = &mut self.hashes_buf[..num_rows]; hashes_buf.fill(42_u32); create_murmur3_hashes(&columns_to_hash, hashes_buf)?; - let partition_ids = &mut self.partition_ids[..num_rows]; for (idx, hash) in hashes_buf.iter().enumerate() { partition_ids[idx] = @@ -371,7 +467,6 @@ impl ImmediateModePartitioner { .iter() .map(|expr| expr.expr.evaluate(batch)?.into_array(num_rows)) .collect::>>()?; - let row_batch = row_converter.convert_columns(arrays.as_slice())?; let partition_ids = &mut self.partition_ids[..num_rows]; for (row_idx, row) in row_batch.iter().enumerate() { @@ -388,54 +483,71 @@ impl ImmediateModePartitioner { } } - /// Route rows to their partition streams using interleave, then write IPC blocks. - /// Returns total bytes written across all partitions. - fn write_partitioned_rows(&mut self, batch: &RecordBatch) -> Result<()> { - let num_partitions = self.streams.len(); + /// Scatter-write rows from batch into per-partition builders, flushing + /// any partition that reaches target_batch_size. Returns + /// `(flushed_builder_bytes, ipc_bytes_written)`. + fn repartition_batch(&mut self, batch: &RecordBatch) -> Result<(usize, usize)> { + let num_partitions = self.partition_buffers.len(); let num_rows = batch.num_rows(); - // Build per-partition row indices - let mut partition_row_indices: Vec> = vec![Vec::new(); num_partitions]; + // Build per-partition row indices, reusing scratch vecs + for indices in self.partition_row_indices.iter_mut() { + indices.clear(); + } for row_idx in 0..num_rows { let pid = self.partition_ids[row_idx] as usize; - partition_row_indices[pid].push((0, row_idx)); + self.partition_row_indices[pid].push(row_idx); } - let batch_refs = [batch]; - - let mut interleave_timer = self.metrics.interleave_time.timer(); - for (pid, indices) in partition_row_indices.iter().enumerate() { - if indices.is_empty() { + // Scatter-write into partition builders + let mut flushed_builder_bytes = 0usize; + let mut ipc_bytes = 0usize; + for pid in 0..num_partitions { + if self.partition_row_indices[pid].is_empty() { continue; } - let sub_batch = interleave_record_batch(&batch_refs, indices) - .map_err(|e| DataFusionError::ArrowError(Box::from(e), None))?; - interleave_timer.stop(); - - let mut encode_timer = self.metrics.encode_time.timer(); - self.streams[pid].write_batch(&sub_batch)?; - encode_timer.stop(); + self.partition_buffers[pid].append_rows(batch, &self.partition_row_indices[pid])?; - interleave_timer = self.metrics.interleave_time.timer(); + // Flush full partitions + if self.partition_buffers[pid].is_full() { + let (builder_bytes, written) = self.flush_partition(pid)?; + flushed_builder_bytes += builder_bytes; + ipc_bytes += written; + } } - interleave_timer.stop(); - Ok(()) + Ok((flushed_builder_bytes, ipc_bytes)) + } + + /// Flush a partition's builders to an IPC block in its output stream. + /// Returns `(flushed_batch_memory, ipc_bytes_written)`. + fn flush_partition(&mut self, pid: usize) -> Result<(usize, usize)> { + let output_batch = self.partition_buffers[pid].flush()?; + let batch_mem = output_batch.get_array_memory_size(); + let mut encode_timer = self.metrics.encode_time.timer(); + let ipc_bytes = self.streams[pid].write_ipc_block(&output_batch)?; + encode_timer.stop(); + Ok((batch_mem, ipc_bytes)) } - /// Spill all partition buffers to per-partition temp files. + /// Spill all partition IPC buffers to per-partition temp files. fn spill_all(&mut self) -> Result<()> { let mut spilled_bytes = 0usize; + // Flush any partially-filled partition builders + for pid in 0..self.partition_buffers.len() { + if self.partition_buffers[pid].has_data() { + self.flush_partition(pid)?; + } + } + + // Drain IPC buffers to disk for pid in 0..self.streams.len() { - // Flush coalescer so buffered rows are serialized before draining - self.streams[pid].flush()?; let buf = self.streams[pid].drain_buffer(); if buf.is_empty() { continue; } - // Create spill file on first spill for this partition if self.spill_files[pid].is_none() { let temp_file = self .runtime @@ -459,17 +571,11 @@ impl ImmediateModePartitioner { } } - // Flush all spill files so data is visible when re-opened for reading in shuffle_write for spill in self.spill_files.iter_mut().flatten() { spill.file.flush()?; } - // Shrink reservation to coalescer_budget — buffer memory was spilled - // but the BatchCoalescer InProgressArray builders persist across spills. - let target = self.coalescer_budget; - if self.reservation.size() > target { - self.reservation.shrink(self.reservation.size() - target); - } + self.reservation.free(); if spilled_bytes > 0 { self.metrics.spill_count.add(1); self.metrics.spilled_bytes.add(spilled_bytes); @@ -488,7 +594,8 @@ impl ShufflePartitioner for ImmediateModePartitioner { let start_time = Instant::now(); - self.metrics.data_size.add(batch.get_array_memory_size()); + let batch_mem = batch.get_array_memory_size(); + self.metrics.data_size.add(batch_mem); self.metrics.baseline.record_output(batch.num_rows()); let repart_start = Instant::now(); @@ -497,13 +604,20 @@ impl ShufflePartitioner for ImmediateModePartitioner { .repart_time .add_duration(repart_start.elapsed()); - let capacity_before: usize = self.streams.iter().map(|s| s.buffer_capacity()).sum(); - self.write_partitioned_rows(&batch)?; - let capacity_after: usize = self.streams.iter().map(|s| s.buffer_capacity()).sum(); - let capacity_growth = capacity_after.saturating_sub(capacity_before); + let (flushed_builder_bytes, ipc_growth) = self.repartition_batch(&batch)?; + let builder_growth = batch_mem; - if capacity_growth > 0 && self.reservation.try_grow(capacity_growth).is_err() { - self.spill_all()?; + // Net memory change: data entered builders, some was flushed to IPC + let net_growth = (builder_growth + ipc_growth).saturating_sub(flushed_builder_bytes); + + if net_growth > 0 { + // Use our own memory limit rather than relying solely on the pool, + // since the pool doesn't see builder allocations directly. + if self.reservation.size() + net_growth > self.memory_limit + || self.reservation.try_grow(net_growth).is_err() + { + self.spill_all()?; + } } self.metrics.input_batches.add(1); @@ -531,7 +645,6 @@ impl ShufflePartitioner for ImmediateModePartitioner { for pid in 0..num_output_partitions { offsets[pid] = output_data.stream_position()? as i64; - // Copy spill file contents if any if let Some(spill) = &self.spill_files[pid] { let path = spill._temp_file.path().to_owned(); let spill_reader = File::open(&path).map_err(|e| { @@ -544,8 +657,10 @@ impl ShufflePartitioner for ImmediateModePartitioner { write_timer.stop(); } - // Flush coalescer and write remaining in-memory buffer - self.streams[pid].flush()?; + if self.partition_buffers[pid].has_data() { + self.flush_partition(pid)?; + } + let buf = self.streams[pid].drain_buffer(); if !buf.is_empty() { let mut write_timer = self.metrics.write_time.timer(); @@ -554,15 +669,12 @@ impl ShufflePartitioner for ImmediateModePartitioner { } } - // Drop spill files now that their contents have been copied to the output for spill in self.spill_files.iter_mut() { *spill = None; } - // Record final offset offsets[num_output_partitions] = output_data.stream_position()? as i64; - // Write index file let mut write_timer = self.metrics.write_time.timer(); let mut output_index = BufWriter::new( File::create(&self.output_index_file) @@ -587,12 +699,11 @@ impl ShufflePartitioner for ImmediateModePartitioner { mod tests { use super::*; use crate::read_ipc_compressed; - use arrow::array::Int32Array; + use arrow::array::{Int32Array, StringArray}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion::execution::memory_pool::GreedyMemoryPool; use datafusion::execution::runtime_env::RuntimeEnvBuilder; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; - use std::sync::Arc; fn make_test_batch(values: &[i32]) -> RecordBatch { let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); @@ -600,6 +711,57 @@ mod tests { RecordBatch::try_new(schema, vec![Arc::new(array)]).unwrap() } + #[test] + fn test_scatter_append_primitives() { + let array: ArrayRef = Arc::new(Int32Array::from(vec![10, 20, 30, 40, 50])); + let mut builder = make_builder(&DataType::Int32, 8); + scatter_append(builder.as_mut(), array.as_ref(), &[0, 2, 4]).unwrap(); + let result = builder.finish(); + let result = result.as_primitive::(); + assert_eq!(result.values().as_ref(), &[10, 30, 50]); + } + + #[test] + fn test_scatter_append_strings() { + let array: ArrayRef = Arc::new(StringArray::from(vec!["a", "b", "c", "d"])); + let mut builder = make_builder(&DataType::Utf8, 4); + scatter_append(builder.as_mut(), array.as_ref(), &[1, 3]).unwrap(); + let result = builder.finish(); + let result = result.as_string::(); + assert_eq!(result.value(0), "b"); + assert_eq!(result.value(1), "d"); + } + + #[test] + fn test_scatter_append_nulls() { + let array: ArrayRef = Arc::new(Int32Array::from(vec![Some(1), None, Some(3)])); + let mut builder = make_builder(&DataType::Int32, 4); + scatter_append(builder.as_mut(), array.as_ref(), &[0, 1, 2]).unwrap(); + let result = builder.finish(); + let result = result.as_primitive::(); + assert!(result.is_valid(0)); + assert!(result.is_null(1)); + assert!(result.is_valid(2)); + } + + #[test] + fn test_partition_buffer_flush_reuse() { + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); + let batch = make_test_batch(&[1, 2, 3, 4, 5]); + + let mut buf = PartitionBuffer::new(&schema, 3); + buf.append_rows(&batch, &[0, 1, 2]).unwrap(); + assert!(buf.is_full()); + + let flushed = buf.flush().unwrap(); + assert_eq!(flushed.num_rows(), 3); + assert_eq!(buf.num_rows, 0); + + // Builders are reused after flush + buf.append_rows(&batch, &[3, 4]).unwrap(); + assert_eq!(buf.num_rows, 2); + } + #[test] fn test_partition_output_stream_write_and_read() { let batch = make_test_batch(&[1, 2, 3, 4, 5]); @@ -611,91 +773,22 @@ mod tests { CompressionCodec::Zstd(1), CompressionCodec::Snappy, ] { - // Use batch_size=1 to force immediate serialization (no coalescing) - let mut stream = PartitionOutputStream::try_new(Arc::clone(&schema), codec, 1).unwrap(); - stream.write_batch(&batch).unwrap(); - stream.flush().unwrap(); + let mut stream = PartitionOutputStream::try_new(Arc::clone(&schema), codec).unwrap(); + stream.write_ipc_block(&batch).unwrap(); let buf = stream.finish().unwrap(); assert!(!buf.is_empty()); - // Parse the first block: 8 bytes length, 8 bytes field_count, then codec+data let ipc_length = u64::from_le_bytes(buf[0..8].try_into().unwrap()) as usize; assert!(ipc_length > 0); - let field_count = usize::from_le_bytes(buf[8..16].try_into().unwrap()); - assert_eq!(field_count, 1); // one field "a" - - // read_ipc_compressed expects data starting at the codec tag let block_end = 8 + ipc_length; let ipc_data = &buf[16..block_end]; let batch2 = read_ipc_compressed(ipc_data).unwrap(); - assert!(batch2.num_rows() > 0); + assert_eq!(batch2.num_rows(), 5); } } - #[test] - fn test_partition_output_stream_coalesces_small_batches() { - let batch1 = make_test_batch(&[1, 2, 3]); - let batch2 = make_test_batch(&[4, 5, 6, 7]); - let schema = batch1.schema(); - - // batch_size=10 means both batches (3+4=7 rows) fit in one coalesced block - let mut stream = - PartitionOutputStream::try_new(schema, CompressionCodec::None, 10).unwrap(); - - // Small batches sit in coalescer, no IPC block written yet - stream.write_batch(&batch1).unwrap(); - assert_eq!(stream.buffered_bytes(), 0); - - stream.write_batch(&batch2).unwrap(); - assert_eq!(stream.buffered_bytes(), 0); - - // Flush produces one coalesced block with all 7 rows - stream.flush().unwrap(); - let buf = stream.finish().unwrap(); - assert!(!buf.is_empty()); - - let ipc_length = u64::from_le_bytes(buf[0..8].try_into().unwrap()) as usize; - let ipc_data = &buf[16..8 + ipc_length]; - let decoded = read_ipc_compressed(ipc_data).unwrap(); - assert_eq!(decoded.num_rows(), 7); - } - - #[test] - fn test_partition_output_stream_empty_batch() { - let batch = make_test_batch(&[]); - let schema = batch.schema(); - - let mut stream = - PartitionOutputStream::try_new(schema, CompressionCodec::None, 8192).unwrap(); - stream.write_batch(&batch).unwrap(); - assert_eq!(stream.buffered_bytes(), 0); - } - - #[test] - fn test_partition_output_stream_drain_after_flush() { - let batch = make_test_batch(&[1, 2, 3]); - let schema = batch.schema(); - - let mut stream = - PartitionOutputStream::try_new(schema, CompressionCodec::None, 8192).unwrap(); - stream.write_batch(&batch).unwrap(); - - // Flush coalescer then drain - stream.flush().unwrap(); - assert!(stream.buffered_bytes() > 0); - - let drained = stream.drain_buffer(); - assert!(!drained.is_empty()); - assert_eq!(stream.buffered_bytes(), 0); - - // Can still write after drain - stream.write_batch(&batch).unwrap(); - stream.flush().unwrap(); - assert!(stream.buffered_bytes() > 0); - } - fn make_hash_partitioning(col_name: &str, num_partitions: usize) -> CometPartitioning { use datafusion::physical_expr::expressions::Column; let expr: Arc = @@ -729,12 +822,12 @@ mod tests { partitioner.insert_batch(batch).await.unwrap(); - // Flush coalesced data so it appears in the byte buffer - for stream in &mut partitioner.streams { - stream.flush().unwrap(); - } - let total_buffered: usize = partitioner.streams.iter().map(|s| s.buffered_bytes()).sum(); - assert!(total_buffered > 0, "Expected some buffered bytes"); + let total_rows: usize = partitioner + .partition_buffers + .iter() + .map(|b| b.num_rows) + .sum(); + assert_eq!(total_rows, 8); } #[tokio::test] @@ -767,15 +860,12 @@ mod tests { partitioner.insert_batch(batch2).await.unwrap(); partitioner.shuffle_write().unwrap(); - // Verify index file has (num_partitions + 1) * 8 bytes let index_data = std::fs::read(&index_path).unwrap(); assert_eq!(index_data.len(), (num_partitions + 1) * 8); - // First offset should be 0 let first_offset = i64::from_le_bytes(index_data[0..8].try_into().unwrap()); assert_eq!(first_offset, 0); - // Last offset should equal data file size let data_file_size = std::fs::metadata(&data_path).unwrap().len(); let last_offset = i64::from_le_bytes( index_data[num_partitions * 8..(num_partitions + 1) * 8] @@ -783,8 +873,6 @@ mod tests { .unwrap(), ); assert_eq!(last_offset as u64, data_file_size); - - // Data file should be non-empty assert!(data_file_size > 0); } @@ -798,7 +886,6 @@ mod tests { let num_partitions = 2; let metrics = ShufflePartitionerMetrics::new(&ExecutionPlanMetricsSet::new(), 0); - // Use a tiny memory pool to force spilling let runtime = Arc::new( RuntimeEnvBuilder::new() .with_memory_pool(Arc::new(GreedyMemoryPool::new(256))) @@ -819,7 +906,6 @@ mod tests { ) .unwrap(); - // Insert enough data to exceed the tiny memory pool and trigger spills for i in 0..10 { let values: Vec = ((i * 10)..((i + 1) * 10)).collect(); let batch = make_test_batch(&values); @@ -828,13 +914,9 @@ mod tests { partitioner.shuffle_write().unwrap(); - // Verify output is valid let index_data = std::fs::read(&index_path).unwrap(); assert_eq!(index_data.len(), (num_partitions + 1) * 8); - let first_offset = i64::from_le_bytes(index_data[0..8].try_into().unwrap()); - assert_eq!(first_offset, 0); - let data_file_size = std::fs::metadata(&data_path).unwrap().len(); let last_offset = i64::from_le_bytes( index_data[num_partitions * 8..(num_partitions + 1) * 8] @@ -857,6 +939,7 @@ mod tests { let metrics = ShufflePartitionerMetrics::new(&ExecutionPlanMetricsSet::new(), 0); let runtime = Arc::new(RuntimeEnvBuilder::new().build().unwrap()); + // Small target to trigger flush during insert let mut partitioner = ImmediateModePartitioner::try_new( 0, data_path.clone(), @@ -865,7 +948,7 @@ mod tests { make_hash_partitioning("a", num_partitions), metrics, runtime, - 8192, + 4, CompressionCodec::Lz4Frame, ) .unwrap(); @@ -873,7 +956,6 @@ mod tests { partitioner.insert_batch(batch).await.unwrap(); partitioner.shuffle_write().unwrap(); - // Read index file to get partition offsets let index_data = std::fs::read(&index_path).unwrap(); let mut offsets = Vec::new(); for i in 0..=num_partitions { @@ -881,61 +963,36 @@ mod tests { offsets.push(offset as usize); } - // Read entire data file let data = std::fs::read(&data_path).unwrap(); - let mut total_rows = 0; for pid in 0..num_partitions { - let partition_start = offsets[pid]; - let partition_end = offsets[pid + 1]; - if partition_start == partition_end { + let (start, end) = (offsets[pid], offsets[pid + 1]); + if start == end { continue; } - - // Parse blocks within this partition's byte range - let mut pos = partition_start; - while pos < partition_end { - // Read 8-byte length prefix + let mut pos = start; + while pos < end { let payload_len = u64::from_le_bytes(data[pos..pos + 8].try_into().unwrap()) as usize; - assert!(payload_len > 0, "Block payload length should be > 0"); - - // Skip 8-byte field_count - let field_count = - u64::from_le_bytes(data[pos + 8..pos + 16].try_into().unwrap()) as usize; - assert_eq!(field_count, 1, "Expected 1 field"); - - // Pass codec tag + IPC data to read_ipc_compressed + assert!(payload_len > 0); let block_end = pos + 8 + payload_len; let ipc_data = &data[pos + 16..block_end]; let decoded = read_ipc_compressed(ipc_data).unwrap(); - assert_eq!(decoded.num_columns(), 1); assert!(decoded.num_rows() > 0); - - // Verify values are valid Int32 let col = decoded .column(0) .as_any() .downcast_ref::() - .expect("Expected Int32Array"); + .unwrap(); for i in 0..col.len() { - let v = col.value(i); - assert!( - (1..=10).contains(&v), - "Value {v} not in expected range 1..=10" - ); + assert!((1..=10).contains(&col.value(i))); } - total_rows += decoded.num_rows(); pos = block_end; } - assert_eq!( - pos, partition_end, - "Block parsing should consume exactly the partition's bytes" - ); + assert_eq!(pos, end); } - - assert_eq!(total_rows, 10, "Total decoded rows should match input"); + assert_eq!(total_rows, 10); } } From 4509985b0aaf94ef312bb27c6fc81e308322f8f4 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 31 Mar 2026 09:23:16 -0600 Subject: [PATCH 25/46] refactor: column-first loop in scatter-write partitioner Restructure repartition_batch to iterate column-first instead of partition-first. Each column's type dispatch now happens once per batch (16 times) instead of once per partition per column (3,200 times). --- .../src/partitioners/immediate_mode.rs | 42 +++++++++++-------- 1 file changed, 24 insertions(+), 18 deletions(-) diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index 1cd7a40aa5..42bff67c4f 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -199,19 +199,6 @@ impl PartitionBuffer { } } - /// Scatter-write selected rows from `batch` into this partition's builders. - fn append_rows(&mut self, batch: &RecordBatch, row_indices: &[usize]) -> Result<()> { - for (col_idx, builder) in self.builders.iter_mut().enumerate() { - scatter_append( - builder.as_mut(), - batch.column(col_idx).as_ref(), - row_indices, - )?; - } - self.num_rows += row_indices.len(); - Ok(()) - } - fn is_full(&self) -> bool { self.num_rows >= self.target_batch_size } @@ -486,6 +473,10 @@ impl ImmediateModePartitioner { /// Scatter-write rows from batch into per-partition builders, flushing /// any partition that reaches target_batch_size. Returns /// `(flushed_builder_bytes, ipc_bytes_written)`. + /// + /// Uses column-first iteration so each column's type dispatch happens once + /// per batch (num_columns times) rather than once per partition per column + /// (num_columns × num_partitions times). fn repartition_batch(&mut self, batch: &RecordBatch) -> Result<(usize, usize)> { let num_partitions = self.partition_buffers.len(); let num_rows = batch.num_rows(); @@ -499,16 +490,31 @@ impl ImmediateModePartitioner { self.partition_row_indices[pid].push(row_idx); } - // Scatter-write into partition builders + // Column-first scatter: resolve each column's type once, then + // scatter across all partitions with the same typed path. + for col_idx in 0..batch.num_columns() { + let source = batch.column(col_idx).as_ref(); + for pid in 0..num_partitions { + if self.partition_row_indices[pid].is_empty() { + continue; + } + scatter_append( + self.partition_buffers[pid].builders[col_idx].as_mut(), + source, + &self.partition_row_indices[pid], + )?; + } + } + + // Update row counts and flush full partitions let mut flushed_builder_bytes = 0usize; let mut ipc_bytes = 0usize; for pid in 0..num_partitions { - if self.partition_row_indices[pid].is_empty() { + let added = self.partition_row_indices[pid].len(); + if added == 0 { continue; } - self.partition_buffers[pid].append_rows(batch, &self.partition_row_indices[pid])?; - - // Flush full partitions + self.partition_buffers[pid].num_rows += added; if self.partition_buffers[pid].is_full() { let (builder_bytes, written) = self.flush_partition(pid)?; flushed_builder_bytes += builder_bytes; From e45d73304c39fce119fdfe8610ad4f4fe8529eca Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 31 Mar 2026 09:50:49 -0600 Subject: [PATCH 26/46] fix: update test for column-first scatter-write API --- native/shuffle/src/partitioners/immediate_mode.rs | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index 42bff67c4f..f6401f153b 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -756,7 +756,13 @@ mod tests { let batch = make_test_batch(&[1, 2, 3, 4, 5]); let mut buf = PartitionBuffer::new(&schema, 3); - buf.append_rows(&batch, &[0, 1, 2]).unwrap(); + scatter_append( + buf.builders[0].as_mut(), + batch.column(0).as_ref(), + &[0, 1, 2], + ) + .unwrap(); + buf.num_rows += 3; assert!(buf.is_full()); let flushed = buf.flush().unwrap(); @@ -764,7 +770,8 @@ mod tests { assert_eq!(buf.num_rows, 0); // Builders are reused after flush - buf.append_rows(&batch, &[3, 4]).unwrap(); + scatter_append(buf.builders[0].as_mut(), batch.column(0).as_ref(), &[3, 4]).unwrap(); + buf.num_rows += 2; assert_eq!(buf.num_rows, 2); } From 32194bf10f3c73af138ebc6baf42ca617d1b9995 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 31 Mar 2026 13:44:48 -0600 Subject: [PATCH 27/46] fix: support complex types (List, Map, Struct) in scatter-write For types without direct scatter-append support (List, Map, Struct, Dictionary, etc.), fall back to arrow::compute::take to produce sub-arrays which are accumulated and concatenated at flush time. Primitive and string types continue using the fast scatter path. --- .../src/partitioners/immediate_mode.rs | 120 ++++++++++++++---- 1 file changed, 98 insertions(+), 22 deletions(-) diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index f6401f153b..5d9c2c2c5f 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -23,7 +23,10 @@ use arrow::array::builder::{ LargeBinaryBuilder, LargeStringBuilder, NullBuilder, PrimitiveBuilder, StringBuilder, StringViewBuilder, }; -use arrow::array::{Array, ArrayRef, AsArray, BinaryViewArray, RecordBatch, StringViewArray}; +use arrow::array::{ + Array, ArrayRef, AsArray, BinaryViewArray, RecordBatch, StringViewArray, UInt32Array, +}; +use arrow::compute::take; use arrow::datatypes::{ DataType, Date32Type, Date64Type, Decimal128Type, Decimal256Type, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type, Int8Type, SchemaRef, TimeUnit, TimestampMicrosecondType, @@ -177,8 +180,49 @@ fn scatter_append( Ok(()) } +/// Per-column strategy: scatter-write via builder for primitive/string types, +/// or accumulate taken sub-arrays for complex types (List, Map, Struct, etc.). +enum ColumnBuffer { + /// Fast path: direct scatter into a pre-allocated builder. + Builder(Box), + /// Fallback for complex types: accumulate `take`-produced sub-arrays, + /// concatenate at flush time. + Accumulator(Vec), +} + +/// Returns true if `scatter_append` can handle this data type directly. +fn has_scatter_support(dt: &DataType) -> bool { + use DataType::*; + matches!( + dt, + Boolean + | Int8 + | Int16 + | Int32 + | Int64 + | UInt8 + | UInt16 + | UInt32 + | UInt64 + | Float32 + | Float64 + | Date32 + | Date64 + | Timestamp(_, _) + | Decimal128(_, _) + | Decimal256(_, _) + | Utf8 + | LargeUtf8 + | Binary + | LargeBinary + | Utf8View + | BinaryView + | Null + ) +} + struct PartitionBuffer { - builders: Vec>, + columns: Vec, schema: SchemaRef, num_rows: usize, target_batch_size: usize, @@ -186,13 +230,19 @@ struct PartitionBuffer { impl PartitionBuffer { fn new(schema: &SchemaRef, target_batch_size: usize) -> Self { - let builders = schema + let columns = schema .fields() .iter() - .map(|f| make_builder(f.data_type(), target_batch_size)) + .map(|f| { + if has_scatter_support(f.data_type()) { + ColumnBuffer::Builder(make_builder(f.data_type(), target_batch_size)) + } else { + ColumnBuffer::Accumulator(Vec::new()) + } + }) .collect(); Self { - builders, + columns, schema: Arc::clone(schema), num_rows: 0, target_batch_size, @@ -203,10 +253,23 @@ impl PartitionBuffer { self.num_rows >= self.target_batch_size } - /// Finish builders into a RecordBatch. Builders are reset but retain - /// their allocated capacity for reuse. + /// Finish all columns into a RecordBatch. Builders are reset (retaining + /// capacity); accumulators are concatenated and cleared. fn flush(&mut self) -> Result { - let arrays: Vec = self.builders.iter_mut().map(|b| b.finish()).collect(); + let arrays: Vec = self + .columns + .iter_mut() + .map(|col| match col { + ColumnBuffer::Builder(b) => b.finish(), + ColumnBuffer::Accumulator(chunks) => { + let refs: Vec<&dyn Array> = chunks.iter().map(|a| a.as_ref()).collect(); + let result = arrow::compute::concat(&refs) + .expect("concat failed for accumulated arrays"); + chunks.clear(); + result + } + }) + .collect(); let batch = RecordBatch::try_new(Arc::clone(&self.schema), arrays) .map_err(|e| DataFusionError::ArrowError(Box::new(e), None))?; self.num_rows = 0; @@ -493,16 +556,24 @@ impl ImmediateModePartitioner { // Column-first scatter: resolve each column's type once, then // scatter across all partitions with the same typed path. for col_idx in 0..batch.num_columns() { - let source = batch.column(col_idx).as_ref(); + let source = batch.column(col_idx); for pid in 0..num_partitions { - if self.partition_row_indices[pid].is_empty() { + let indices = &self.partition_row_indices[pid]; + if indices.is_empty() { continue; } - scatter_append( - self.partition_buffers[pid].builders[col_idx].as_mut(), - source, - &self.partition_row_indices[pid], - )?; + match &mut self.partition_buffers[pid].columns[col_idx] { + ColumnBuffer::Builder(builder) => { + scatter_append(builder.as_mut(), source.as_ref(), indices)?; + } + ColumnBuffer::Accumulator(chunks) => { + let idx_array = + UInt32Array::from_iter_values(indices.iter().map(|&i| i as u32)); + let taken = take(source.as_ref(), &idx_array, None) + .map_err(|e| DataFusionError::ArrowError(Box::new(e), None))?; + chunks.push(taken); + } + } } } @@ -756,12 +827,12 @@ mod tests { let batch = make_test_batch(&[1, 2, 3, 4, 5]); let mut buf = PartitionBuffer::new(&schema, 3); - scatter_append( - buf.builders[0].as_mut(), - batch.column(0).as_ref(), - &[0, 1, 2], - ) - .unwrap(); + match &mut buf.columns[0] { + ColumnBuffer::Builder(b) => { + scatter_append(b.as_mut(), batch.column(0).as_ref(), &[0, 1, 2]).unwrap() + } + _ => panic!("expected Builder"), + } buf.num_rows += 3; assert!(buf.is_full()); @@ -770,7 +841,12 @@ mod tests { assert_eq!(buf.num_rows, 0); // Builders are reused after flush - scatter_append(buf.builders[0].as_mut(), batch.column(0).as_ref(), &[3, 4]).unwrap(); + match &mut buf.columns[0] { + ColumnBuffer::Builder(b) => { + scatter_append(b.as_mut(), batch.column(0).as_ref(), &[3, 4]).unwrap() + } + _ => panic!("expected Builder"), + } buf.num_rows += 2; assert_eq!(buf.num_rows, 2); } From 7ea12aca3e99feed58e781a98eed35215213eff7 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 31 Mar 2026 15:48:01 -0600 Subject: [PATCH 28/46] fix: skip spill test under miri (copy_file_range unsupported) --- native/shuffle/src/partitioners/immediate_mode.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index 5d9c2c2c5f..e756dd64af 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -966,6 +966,7 @@ mod tests { } #[tokio::test] + #[cfg_attr(miri, ignore)] // spill uses std::io::copy which triggers copy_file_range async fn test_immediate_mode_spill() { let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); let dir = tempfile::tempdir().unwrap(); From d06bfdb890fcc69d808c291c9214c52d5fd34282 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 1 Apr 2026 04:33:28 -0600 Subject: [PATCH 29/46] fix: ignore miri for test_partition_output_stream_write_and_read --- native/shuffle/src/partitioners/immediate_mode.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index e756dd64af..ae039d16ef 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -852,6 +852,7 @@ mod tests { } #[test] + #[cfg_attr(miri, ignore)] fn test_partition_output_stream_write_and_read() { let batch = make_test_batch(&[1, 2, 3, 4, 5]); let schema = batch.schema(); From 17264d69952bd99da2309cd7d22e63a9808d0aca Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 1 Apr 2026 04:40:23 -0600 Subject: [PATCH 30/46] docs: document buffered vs immediate native shuffle partitioner modes Update the tuning guide and contributor guide to describe the two native shuffle partitioner modes (immediate and buffered), including architecture diagrams, data flow, memory characteristics, and configuration. --- .../contributor-guide/native_shuffle.md | 93 +++++++++++++------ docs/source/user-guide/latest/tuning.md | 11 +++ 2 files changed, 76 insertions(+), 28 deletions(-) diff --git a/docs/source/contributor-guide/native_shuffle.md b/docs/source/contributor-guide/native_shuffle.md index 18e80a90c8..2ea8bb71f8 100644 --- a/docs/source/contributor-guide/native_shuffle.md +++ b/docs/source/contributor-guide/native_shuffle.md @@ -81,10 +81,18 @@ Native shuffle (`CometExchange`) is selected when all of the following condition └─────────────────────────────────────────────────────────────────────────────┘ │ │ ▼ ▼ -┌───────────────────────────────────┐ ┌───────────────────────────────────┐ -│ MultiPartitionShuffleRepartitioner │ │ SinglePartitionShufflePartitioner │ -│ (hash/range partitioning) │ │ (single partition case) │ -└───────────────────────────────────┘ └───────────────────────────────────┘ +┌───────────────────────────────────────────────────────────────────────┐ +│ Partitioner Selection │ +│ Controlled by spark.comet.exec.shuffle.partitionerMode │ +├───────────────────────────┬───────────────────────────────────────────┤ +│ immediate (default) │ buffered │ +│ ImmediateModePartitioner │ MultiPartitionShuffleRepartitioner │ +│ (hash/range/round-robin) │ (hash/range/round-robin) │ +│ Writes IPC blocks as │ Buffers all rows in memory │ +│ batches arrive │ before writing │ +├───────────────────────────┴───────────────────────────────────────────┤ +│ SinglePartitionShufflePartitioner (single partition case) │ +└───────────────────────────────────────────────────────────────────────┘ │ ▼ ┌───────────────────────────────────┐ @@ -113,11 +121,13 @@ Native shuffle (`CometExchange`) is selected when all of the following condition ### Rust Side -| File | Location | Description | -| ----------------------- | ------------------------------------ | ------------------------------------------------------------------------------------ | -| `shuffle_writer.rs` | `native/core/src/execution/shuffle/` | `ShuffleWriterExec` plan and partitioners. Main shuffle logic. | -| `codec.rs` | `native/core/src/execution/shuffle/` | `ShuffleBlockWriter` for Arrow IPC encoding with compression. Also handles decoding. | -| `comet_partitioning.rs` | `native/core/src/execution/shuffle/` | `CometPartitioning` enum defining partition schemes (Hash, Range, Single). | +| File | Location | Description | +| ----------------------- | ---------------------------------- | --------------------------------------------------------------------------------------------------------------------- | +| `shuffle_writer.rs` | `native/shuffle/src/` | `ShuffleWriterExec` plan. Selects partitioner based on `immediate_mode` flag. | +| `immediate_mode.rs` | `native/shuffle/src/partitioners/` | `ImmediateModePartitioner`. Scatter-writes rows into per-partition Arrow builders and flushes IPC blocks immediately. | +| `multi_partition.rs` | `native/shuffle/src/partitioners/` | `MultiPartitionShuffleRepartitioner`. Buffers all rows in memory, then writes partitions. | +| `codec.rs` | `native/shuffle/src/` | `ShuffleBlockWriter` for Arrow IPC encoding with compression. Also handles decoding. | +| `comet_partitioning.rs` | `native/shuffle/src/` | `CometPartitioning` enum defining partition schemes (Hash, Range, Single). | ## Data Flow @@ -129,23 +139,32 @@ Native shuffle (`CometExchange`) is selected when all of the following condition 2. **Native execution**: `CometExec.getCometIterator()` executes the plan in Rust. -3. **Partitioning**: `ShuffleWriterExec` receives batches and routes to the appropriate partitioner: - - `MultiPartitionShuffleRepartitioner`: For hash/range/round-robin partitioning - - `SinglePartitionShufflePartitioner`: For single partition (simpler path) +3. **Partitioning**: `ShuffleWriterExec` receives batches and routes to the appropriate partitioner + based on the `partitionerMode` configuration: + - **Immediate mode** (`ImmediateModePartitioner`): For hash/range/round-robin partitioning. + As each batch arrives, rows are scattered into per-partition Arrow array builders. When a + partition's builder reaches the target batch size, it is flushed as a compressed Arrow IPC + block directly to the output stream. This keeps memory usage proportional to the number of + partitions times the batch size, rather than proportional to the total input size. -4. **Buffering and spilling**: The partitioner buffers rows per partition. When memory pressure - exceeds the threshold, partitions spill to temporary files. + - **Buffered mode** (`MultiPartitionShuffleRepartitioner`): For hash/range/round-robin + partitioning. Buffers all input `RecordBatch`es in memory, then partitions and writes + them in a single pass. When memory pressure exceeds the threshold, partitions spill to + temporary files. -5. **Encoding**: `ShuffleBlockWriter` encodes each partition's data as compressed Arrow IPC: + - `SinglePartitionShufflePartitioner`: For single partition (simpler path, used regardless + of partitioner mode). + +4. **Encoding**: `ShuffleBlockWriter` encodes each partition's data as compressed Arrow IPC: - Writes compression type header - Writes field count header - Writes compressed IPC stream -6. **Output files**: Two files are produced: +5. **Output files**: Two files are produced: - **Data file**: Concatenated partition data - **Index file**: Array of 8-byte little-endian offsets marking partition boundaries -7. **Commit**: Back in JVM, `CometNativeShuffleWriter` reads the index file to get partition +6. **Commit**: Back in JVM, `CometNativeShuffleWriter` reads the index file to get partition lengths and commits via Spark's `IndexShuffleBlockResolver`. ### Read Path @@ -201,10 +220,27 @@ sizes. ## Memory Management -Native shuffle uses DataFusion's memory management with spilling support: +Native shuffle uses DataFusion's memory management. The memory characteristics differ +between the two partitioner modes: + +### Immediate Mode + +Immediate mode keeps memory usage low by writing data to disk as it arrives: + +- **Per-partition builders**: Each partition has a set of Arrow array builders sized to the + target batch size. When a builder fills up, it is flushed as a compressed IPC block. +- **Memory footprint**: Proportional to `num_partitions × batch_size`, independent of total + input size. +- **Spilling**: When memory pressure is detected via DataFusion's `MemoryConsumer` trait, all + partition buffers are flushed to disk regardless of how full they are. + +### Buffered Mode + +Buffered mode holds all input data in memory before writing: -- **Memory pool**: Tracks memory usage across the shuffle operation. -- **Spill threshold**: When buffered data exceeds the threshold, partitions spill to disk. +- **Buffered batches**: All incoming `RecordBatch`es are accumulated in a `Vec`. +- **Spill threshold**: When buffered data exceeds the memory threshold, partitions spill to + temporary files on disk. - **Per-partition spilling**: Each partition has its own spill file. Multiple spills for a partition are concatenated when writing the final output. - **Scratch space**: Reusable buffers for partition ID computation to reduce allocations. @@ -232,14 +268,15 @@ independently compressed, allowing parallel decompression during reads. ## Configuration -| Config | Default | Description | -| ------------------------------------------------- | ------- | ---------------------------------------- | -| `spark.comet.exec.shuffle.enabled` | `true` | Enable Comet shuffle | -| `spark.comet.exec.shuffle.mode` | `auto` | Shuffle mode: `native`, `jvm`, or `auto` | -| `spark.comet.exec.shuffle.compression.codec` | `zstd` | Compression codec | -| `spark.comet.exec.shuffle.compression.zstd.level` | `1` | Zstd compression level | -| `spark.comet.shuffle.write.buffer.size` | `1MB` | Write buffer size | -| `spark.comet.columnar.shuffle.batch.size` | `8192` | Target rows per batch | +| Config | Default | Description | +| ------------------------------------------------- | ----------- | ------------------------------------------- | +| `spark.comet.exec.shuffle.enabled` | `true` | Enable Comet shuffle | +| `spark.comet.exec.shuffle.mode` | `auto` | Shuffle mode: `native`, `jvm`, or `auto` | +| `spark.comet.exec.shuffle.partitionerMode` | `immediate` | Partitioner mode: `immediate` or `buffered` | +| `spark.comet.exec.shuffle.compression.codec` | `zstd` | Compression codec | +| `spark.comet.exec.shuffle.compression.zstd.level` | `1` | Zstd compression level | +| `spark.comet.shuffle.write.buffer.size` | `1MB` | Write buffer size | +| `spark.comet.columnar.shuffle.batch.size` | `8192` | Target rows per batch | ## Comparison with JVM Shuffle diff --git a/docs/source/user-guide/latest/tuning.md b/docs/source/user-guide/latest/tuning.md index 5939e89ef3..c47fe0a644 100644 --- a/docs/source/user-guide/latest/tuning.md +++ b/docs/source/user-guide/latest/tuning.md @@ -144,6 +144,17 @@ Comet provides a fully native shuffle implementation, which generally provides t supports `HashPartitioning`, `RangePartitioning` and `SinglePartitioning` but currently only supports primitive type partitioning keys. Columns that are not partitioning keys may contain complex types like maps, structs, and arrays. +Native shuffle has two partitioner modes, configured via +`spark.comet.exec.shuffle.partitionerMode`: + +- **`immediate`** (default): Writes partitioned Arrow IPC blocks to disk immediately as each batch + arrives. This mode uses less memory because it does not need to buffer the entire input before + writing. It is recommended for most workloads, especially large datasets. + +- **`buffered`**: Buffers all input rows in memory before partitioning and writing to disk. This + may improve performance for small datasets that fit in memory, but uses significantly more + memory. + #### Columnar (JVM) Shuffle Comet Columnar shuffle is JVM-based and supports `HashPartitioning`, `RoundRobinPartitioning`, `RangePartitioning`, and From 2b6f7742735367185925af9c8838f4d2dad7dcc4 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 1 Apr 2026 13:44:03 -0600 Subject: [PATCH 31/46] refactor: revert metrics changes and fix imprecise docs Revert metrics instrumentation changes to the buffered shuffle partitioner so this branch only contains the new immediate mode implementation and benchmark. Fix docs that incorrectly described immediate mode as writing directly to disk. --- .../contributor-guide/native_shuffle.md | 23 +++++++++++-------- native/shuffle/src/metrics.rs | 12 ---------- .../src/partitioners/multi_partition.rs | 15 ++++-------- .../partitioned_batch_iterator.rs | 17 ++------------ .../src/partitioners/single_partition.rs | 10 ++------ native/shuffle/src/shuffle_writer.rs | 17 ++++---------- .../shuffle/src/writers/buf_batch_writer.rs | 6 ----- native/shuffle/src/writers/spill.rs | 15 +++--------- 8 files changed, 29 insertions(+), 86 deletions(-) diff --git a/docs/source/contributor-guide/native_shuffle.md b/docs/source/contributor-guide/native_shuffle.md index 2ea8bb71f8..4a8c54f1bf 100644 --- a/docs/source/contributor-guide/native_shuffle.md +++ b/docs/source/contributor-guide/native_shuffle.md @@ -124,7 +124,7 @@ Native shuffle (`CometExchange`) is selected when all of the following condition | File | Location | Description | | ----------------------- | ---------------------------------- | --------------------------------------------------------------------------------------------------------------------- | | `shuffle_writer.rs` | `native/shuffle/src/` | `ShuffleWriterExec` plan. Selects partitioner based on `immediate_mode` flag. | -| `immediate_mode.rs` | `native/shuffle/src/partitioners/` | `ImmediateModePartitioner`. Scatter-writes rows into per-partition Arrow builders and flushes IPC blocks immediately. | +| `immediate_mode.rs` | `native/shuffle/src/partitioners/` | `ImmediateModePartitioner`. Scatter-writes rows into per-partition Arrow builders and flushes IPC blocks to in-memory buffers eagerly. | | `multi_partition.rs` | `native/shuffle/src/partitioners/` | `MultiPartitionShuffleRepartitioner`. Buffers all rows in memory, then writes partitions. | | `codec.rs` | `native/shuffle/src/` | `ShuffleBlockWriter` for Arrow IPC encoding with compression. Also handles decoding. | | `comet_partitioning.rs` | `native/shuffle/src/` | `CometPartitioning` enum defining partition schemes (Hash, Range, Single). | @@ -144,8 +144,9 @@ Native shuffle (`CometExchange`) is selected when all of the following condition - **Immediate mode** (`ImmediateModePartitioner`): For hash/range/round-robin partitioning. As each batch arrives, rows are scattered into per-partition Arrow array builders. When a partition's builder reaches the target batch size, it is flushed as a compressed Arrow IPC - block directly to the output stream. This keeps memory usage proportional to the number of - partitions times the batch size, rather than proportional to the total input size. + block to an in-memory buffer. Under memory pressure, these buffers are spilled to + per-partition temporary files. This keeps memory usage much lower than buffered mode since + data is encoded into compact IPC format eagerly rather than held as raw Arrow arrays. - **Buffered mode** (`MultiPartitionShuffleRepartitioner`): For hash/range/round-robin partitioning. Buffers all input `RecordBatch`es in memory, then partitions and writes @@ -225,14 +226,18 @@ between the two partitioner modes: ### Immediate Mode -Immediate mode keeps memory usage low by writing data to disk as it arrives: +Immediate mode keeps memory usage low by partitioning and encoding data eagerly as it arrives, +rather than buffering all input rows before writing: - **Per-partition builders**: Each partition has a set of Arrow array builders sized to the - target batch size. When a builder fills up, it is flushed as a compressed IPC block. -- **Memory footprint**: Proportional to `num_partitions × batch_size`, independent of total - input size. -- **Spilling**: When memory pressure is detected via DataFusion's `MemoryConsumer` trait, all - partition buffers are flushed to disk regardless of how full they are. + target batch size. When a builder fills up, it is flushed as a compressed IPC block to an + in-memory buffer. +- **Memory footprint**: Proportional to `num_partitions × batch_size` for the builders, plus + the accumulated IPC buffers. This is typically much smaller than buffered mode since IPC + encoding is more compact than raw Arrow arrays. +- **Spilling**: When memory pressure is detected via DataFusion's `MemoryConsumer` trait, + partition builders are flushed and all IPC buffers are drained to per-partition temporary + files on disk. ### Buffered Mode diff --git a/native/shuffle/src/metrics.rs b/native/shuffle/src/metrics.rs index 8755e2c65f..1de751cf41 100644 --- a/native/shuffle/src/metrics.rs +++ b/native/shuffle/src/metrics.rs @@ -33,15 +33,6 @@ pub(crate) struct ShufflePartitionerMetrics { /// Time spent writing to disk. Maps to "shuffleWriteTime" in Spark SQL Metrics. pub(crate) write_time: Time, - /// Time spent in interleave_record_batch gathering rows into shuffled batches - pub(crate) interleave_time: Time, - - /// Time spent coalescing small batches before serialization - pub(crate) coalesce_time: Time, - - /// Time spent buffering partition indices and memory accounting - pub(crate) memcopy_time: Time, - /// Number of input batches pub(crate) input_batches: Count, @@ -62,9 +53,6 @@ impl ShufflePartitionerMetrics { repart_time: MetricBuilder::new(metrics).subset_time("repart_time", partition), encode_time: MetricBuilder::new(metrics).subset_time("encode_time", partition), write_time: MetricBuilder::new(metrics).subset_time("write_time", partition), - interleave_time: MetricBuilder::new(metrics).subset_time("interleave_time", partition), - coalesce_time: MetricBuilder::new(metrics).subset_time("coalesce_time", partition), - memcopy_time: MetricBuilder::new(metrics).subset_time("memcopy_time", partition), input_batches: MetricBuilder::new(metrics).counter("input_batches", partition), spill_count: MetricBuilder::new(metrics).spill_count(partition), spilled_bytes: MetricBuilder::new(metrics).spilled_bytes(partition), diff --git a/native/shuffle/src/partitioners/multi_partition.rs b/native/shuffle/src/partitioners/multi_partition.rs index 69f2878834..7de9314f54 100644 --- a/native/shuffle/src/partitioners/multi_partition.rs +++ b/native/shuffle/src/partitioners/multi_partition.rs @@ -398,7 +398,6 @@ impl MultiPartitionShuffleRepartitioner { partition_row_indices: &[u32], partition_starts: &[u32], ) -> datafusion::common::Result<()> { - let start_time = Instant::now(); let mut mem_growth: usize = input.get_array_memory_size(); let buffered_partition_idx = self.buffered_batches.len() as u32; self.buffered_batches.push(input); @@ -427,7 +426,6 @@ impl MultiPartitionShuffleRepartitioner { let after_size = indices.allocated_size(); mem_growth += after_size.saturating_sub(before_size); } - self.metrics.memcopy_time.add_duration(start_time.elapsed()); if self.reservation.try_grow(mem_growth).is_err() { self.spill()?; @@ -436,14 +434,12 @@ impl MultiPartitionShuffleRepartitioner { Ok(()) } - #[allow(clippy::too_many_arguments)] fn shuffle_write_partition( partition_iter: &mut PartitionedBatchIterator, shuffle_block_writer: &mut ShuffleBlockWriter, output_data: &mut BufWriter, encode_time: &Time, write_time: &Time, - coalesce_time: &Time, write_buffer_size: usize, batch_size: usize, ) -> datafusion::common::Result<()> { @@ -455,9 +451,9 @@ impl MultiPartitionShuffleRepartitioner { ); for batch in partition_iter { let batch = batch?; - buf_batch_writer.write(&batch, encode_time, write_time, coalesce_time)?; + buf_batch_writer.write(&batch, encode_time, write_time)?; } - buf_batch_writer.flush(encode_time, write_time, coalesce_time)?; + buf_batch_writer.flush(encode_time, write_time)?; Ok(()) } @@ -510,8 +506,7 @@ impl MultiPartitionShuffleRepartitioner { for partition_id in 0..num_output_partitions { let partition_writer = &mut self.partition_writers[partition_id]; - let mut iter = - partitioned_batches.produce(partition_id, &self.metrics.interleave_time); + let mut iter = partitioned_batches.produce(partition_id); spilled_bytes += partition_writer.spill( &mut iter, &self.runtime, @@ -596,15 +591,13 @@ impl ShufflePartitioner for MultiPartitionShuffleRepartitioner { } // Write in memory batches to output data file - let mut partition_iter = - partitioned_batches.produce(i, &self.metrics.interleave_time); + let mut partition_iter = partitioned_batches.produce(i); Self::shuffle_write_partition( &mut partition_iter, &mut self.shuffle_block_writer, &mut output_data, &self.metrics.encode_time, &self.metrics.write_time, - &self.metrics.coalesce_time, self.write_buffer_size, self.batch_size, )?; diff --git a/native/shuffle/src/partitioners/partitioned_batch_iterator.rs b/native/shuffle/src/partitioners/partitioned_batch_iterator.rs index f124d98ff2..8309a8ed4a 100644 --- a/native/shuffle/src/partitioners/partitioned_batch_iterator.rs +++ b/native/shuffle/src/partitioners/partitioned_batch_iterator.rs @@ -18,7 +18,6 @@ use arrow::array::RecordBatch; use arrow::compute::interleave_record_batch; use datafusion::common::DataFusionError; -use datafusion::physical_plan::metrics::Time; /// A helper struct to produce shuffled batches. /// This struct takes ownership of the buffered batches and partition indices from the @@ -42,16 +41,11 @@ impl PartitionedBatchesProducer { } } - pub(super) fn produce<'a>( - &'a mut self, - partition_id: usize, - interleave_time: &'a Time, - ) -> PartitionedBatchIterator<'a> { + pub(super) fn produce(&mut self, partition_id: usize) -> PartitionedBatchIterator<'_> { PartitionedBatchIterator::new( &self.partition_indices[partition_id], &self.buffered_batches, self.batch_size, - interleave_time, ) } } @@ -62,7 +56,6 @@ pub(crate) struct PartitionedBatchIterator<'a> { batch_size: usize, indices: Vec<(usize, usize)>, pos: usize, - interleave_time: &'a Time, } impl<'a> PartitionedBatchIterator<'a> { @@ -70,7 +63,6 @@ impl<'a> PartitionedBatchIterator<'a> { indices: &'a [(u32, u32)], buffered_batches: &'a [RecordBatch], batch_size: usize, - interleave_time: &'a Time, ) -> Self { if indices.is_empty() { // Avoid unnecessary allocations when the partition is empty @@ -79,7 +71,6 @@ impl<'a> PartitionedBatchIterator<'a> { batch_size, indices: vec![], pos: 0, - interleave_time, }; } let record_batches = buffered_batches.iter().collect::>(); @@ -92,7 +83,6 @@ impl<'a> PartitionedBatchIterator<'a> { batch_size, indices: current_indices, pos: 0, - interleave_time, } } } @@ -107,10 +97,7 @@ impl Iterator for PartitionedBatchIterator<'_> { let indices_end = std::cmp::min(self.pos + self.batch_size, self.indices.len()); let indices = &self.indices[self.pos..indices_end]; - let mut timer = self.interleave_time.timer(); - let result = interleave_record_batch(&self.record_batches, indices); - timer.stop(); - match result { + match interleave_record_batch(&self.record_batches, indices) { Ok(batch) => { self.pos = indices_end; Some(Ok(batch)) diff --git a/native/shuffle/src/partitioners/single_partition.rs b/native/shuffle/src/partitioners/single_partition.rs index d866aae330..5801ef613b 100644 --- a/native/shuffle/src/partitioners/single_partition.rs +++ b/native/shuffle/src/partitioners/single_partition.rs @@ -127,7 +127,6 @@ impl ShufflePartitioner for SinglePartitionShufflePartitioner { &batch, &self.metrics.encode_time, &self.metrics.write_time, - &self.metrics.coalesce_time, )?; } @@ -137,7 +136,6 @@ impl ShufflePartitioner for SinglePartitionShufflePartitioner { &batch, &self.metrics.encode_time, &self.metrics.write_time, - &self.metrics.coalesce_time, )?; } else { // Add the new batch to the buffer @@ -166,14 +164,10 @@ impl ShufflePartitioner for SinglePartitionShufflePartitioner { &batch, &self.metrics.encode_time, &self.metrics.write_time, - &self.metrics.coalesce_time, )?; } - self.output_data_writer.flush( - &self.metrics.encode_time, - &self.metrics.write_time, - &self.metrics.coalesce_time, - )?; + self.output_data_writer + .flush(&self.metrics.encode_time, &self.metrics.write_time)?; // Write index file. It should only contain 2 entries: 0 and the total number of bytes written let index_file = OpenOptions::new() diff --git a/native/shuffle/src/shuffle_writer.rs b/native/shuffle/src/shuffle_writer.rs index ca1e886737..6c8598cb88 100644 --- a/native/shuffle/src/shuffle_writer.rs +++ b/native/shuffle/src/shuffle_writer.rs @@ -639,7 +639,6 @@ mod test { let codec = CompressionCodec::Lz4Frame; let encode_time = Time::default(); let write_time = Time::default(); - let coalesce_time = Time::default(); // Write with coalescing (batch_size=8192) let mut coalesced_output = Vec::new(); @@ -652,13 +651,9 @@ mod test { 8192, ); for batch in &small_batches { - buf_writer - .write(batch, &encode_time, &write_time, &coalesce_time) - .unwrap(); + buf_writer.write(batch, &encode_time, &write_time).unwrap(); } - buf_writer - .flush(&encode_time, &write_time, &coalesce_time) - .unwrap(); + buf_writer.flush(&encode_time, &write_time).unwrap(); } // Write without coalescing (batch_size=1) @@ -672,13 +667,9 @@ mod test { 1, ); for batch in &small_batches { - buf_writer - .write(batch, &encode_time, &write_time, &coalesce_time) - .unwrap(); + buf_writer.write(batch, &encode_time, &write_time).unwrap(); } - buf_writer - .flush(&encode_time, &write_time, &coalesce_time) - .unwrap(); + buf_writer.flush(&encode_time, &write_time).unwrap(); } // Coalesced output should be smaller due to fewer IPC schema blocks diff --git a/native/shuffle/src/writers/buf_batch_writer.rs b/native/shuffle/src/writers/buf_batch_writer.rs index f6bb15137e..cfddb46539 100644 --- a/native/shuffle/src/writers/buf_batch_writer.rs +++ b/native/shuffle/src/writers/buf_batch_writer.rs @@ -63,9 +63,7 @@ impl, W: Write> BufBatchWriter { batch: &RecordBatch, encode_time: &Time, write_time: &Time, - coalesce_time: &Time, ) -> datafusion::common::Result { - let mut coalesce_timer = coalesce_time.timer(); let coalescer = self .coalescer .get_or_insert_with(|| BatchCoalescer::new(batch.schema(), self.batch_size)); @@ -77,7 +75,6 @@ impl, W: Write> BufBatchWriter { while let Some(batch) = coalescer.next_completed_batch() { completed.push(batch); } - coalesce_timer.stop(); let mut bytes_written = 0; for batch in &completed { @@ -113,10 +110,8 @@ impl, W: Write> BufBatchWriter { &mut self, encode_time: &Time, write_time: &Time, - coalesce_time: &Time, ) -> datafusion::common::Result<()> { // Finish any remaining buffered rows in the coalescer - let mut coalesce_timer = coalesce_time.timer(); let mut remaining = Vec::new(); if let Some(coalescer) = &mut self.coalescer { coalescer.finish_buffered_batch()?; @@ -124,7 +119,6 @@ impl, W: Write> BufBatchWriter { remaining.push(batch); } } - coalesce_timer.stop(); for batch in &remaining { self.write_batch_to_buffer(batch, encode_time, write_time)?; } diff --git a/native/shuffle/src/writers/spill.rs b/native/shuffle/src/writers/spill.rs index 05492791d1..c16caddbf9 100644 --- a/native/shuffle/src/writers/spill.rs +++ b/native/shuffle/src/writers/spill.rs @@ -93,26 +93,17 @@ impl PartitionWriter { write_buffer_size, batch_size, ); - let mut bytes_written = buf_batch_writer.write( - &batch?, - &metrics.encode_time, - &metrics.write_time, - &metrics.coalesce_time, - )?; + let mut bytes_written = + buf_batch_writer.write(&batch?, &metrics.encode_time, &metrics.write_time)?; for batch in iter { let batch = batch?; bytes_written += buf_batch_writer.write( &batch, &metrics.encode_time, &metrics.write_time, - &metrics.coalesce_time, )?; } - buf_batch_writer.flush( - &metrics.encode_time, - &metrics.write_time, - &metrics.coalesce_time, - )?; + buf_batch_writer.flush(&metrics.encode_time, &metrics.write_time)?; bytes_written }; From cbd75cff197fcf29b87c679dff6377ce87c7302c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 1 Apr 2026 15:41:58 -0600 Subject: [PATCH 32/46] chore: format markdown table alignment in native_shuffle docs --- docs/source/contributor-guide/native_shuffle.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/source/contributor-guide/native_shuffle.md b/docs/source/contributor-guide/native_shuffle.md index 4a8c54f1bf..9e1a17d349 100644 --- a/docs/source/contributor-guide/native_shuffle.md +++ b/docs/source/contributor-guide/native_shuffle.md @@ -121,13 +121,13 @@ Native shuffle (`CometExchange`) is selected when all of the following condition ### Rust Side -| File | Location | Description | -| ----------------------- | ---------------------------------- | --------------------------------------------------------------------------------------------------------------------- | -| `shuffle_writer.rs` | `native/shuffle/src/` | `ShuffleWriterExec` plan. Selects partitioner based on `immediate_mode` flag. | +| File | Location | Description | +| ----------------------- | ---------------------------------- | -------------------------------------------------------------------------------------------------------------------------------------- | +| `shuffle_writer.rs` | `native/shuffle/src/` | `ShuffleWriterExec` plan. Selects partitioner based on `immediate_mode` flag. | | `immediate_mode.rs` | `native/shuffle/src/partitioners/` | `ImmediateModePartitioner`. Scatter-writes rows into per-partition Arrow builders and flushes IPC blocks to in-memory buffers eagerly. | -| `multi_partition.rs` | `native/shuffle/src/partitioners/` | `MultiPartitionShuffleRepartitioner`. Buffers all rows in memory, then writes partitions. | -| `codec.rs` | `native/shuffle/src/` | `ShuffleBlockWriter` for Arrow IPC encoding with compression. Also handles decoding. | -| `comet_partitioning.rs` | `native/shuffle/src/` | `CometPartitioning` enum defining partition schemes (Hash, Range, Single). | +| `multi_partition.rs` | `native/shuffle/src/partitioners/` | `MultiPartitionShuffleRepartitioner`. Buffers all rows in memory, then writes partitions. | +| `codec.rs` | `native/shuffle/src/` | `ShuffleBlockWriter` for Arrow IPC encoding with compression. Also handles decoding. | +| `comet_partitioning.rs` | `native/shuffle/src/` | `CometPartitioning` enum defining partition schemes (Hash, Range, Single). | ## Data Flow From bbb4d7ff9875f53737a42a899d980b93478c3caa Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Apr 2026 09:08:21 -0600 Subject: [PATCH 33/46] docs: design spec for one IPC stream per partition shuffle format MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Addresses #3882 — shuffle format overhead with default batch size. --- ...shuffle-ipc-stream-per-partition-design.md | 250 ++++++++++++++++++ 1 file changed, 250 insertions(+) create mode 100644 docs/superpowers/specs/2026-04-02-shuffle-ipc-stream-per-partition-design.md diff --git a/docs/superpowers/specs/2026-04-02-shuffle-ipc-stream-per-partition-design.md b/docs/superpowers/specs/2026-04-02-shuffle-ipc-stream-per-partition-design.md new file mode 100644 index 0000000000..ca2f0839fa --- /dev/null +++ b/docs/superpowers/specs/2026-04-02-shuffle-ipc-stream-per-partition-design.md @@ -0,0 +1,250 @@ +# Shuffle Format: One IPC Stream Per Partition + +**Issue:** [#3882](https://github.com/apache/datafusion-comet/issues/3882) — Current shuffle format has too much overhead with default batch size + +**Date:** 2026-04-02 + +## Problem + +The current shuffle format writes each batch as an independent Arrow IPC stream, repeating the schema for every batch. Each block also has a custom 20-byte header (8-byte length + 8-byte field count + 4-byte codec tag) and creates a new compression codec instance. With the default batch size of 8192 rows, Comet shuffle files are ~50% larger than Spark shuffle files and overall query performance is ~10% slower than Spark. + +## Solution + +Write one Arrow IPC stream per partition instead of one per batch. Use Arrow IPC's built-in per-buffer body compression instead of wrapping the entire IPC stream in an external compression codec. Move IPC stream parsing to Rust, with Rust pulling bytes from JVM's `InputStream` via JNI callbacks. + +## Current Format (Per Partition) + +``` +[block 1: header(20 bytes) | compressed(schema + batch)] +[block 2: header(20 bytes) | compressed(schema + batch)] +... +[block N: header(20 bytes) | compressed(schema + batch)] +``` + +Each block is a self-contained compressed Arrow IPC stream with a custom header: +- 8 bytes: IPC block length (little-endian u64) +- 8 bytes: field count (little-endian u64) +- 4 bytes: compression codec tag (ASCII: `SNAP`, `LZ4_`, `ZSTD`, `NONE`) +- Variable: compressed Arrow IPC stream bytes (schema message + one record batch message) + +The JVM reader parses the custom header, reads the exact byte blob, and passes it to Rust via JNI for decompression and decoding. + +## New Format (Per Partition) + +``` +[standard Arrow IPC stream: schema | batch | batch | ... | EOS] +``` + +Each partition's data is a single standard Arrow IPC stream: +- One schema message (written once when the stream is created) +- N record batch messages (each batch's buffers individually compressed via Arrow IPC body compression) +- End-of-stream marker + +No custom header. No external compression wrapper. Standard Arrow IPC framing handles message boundaries. + +## Architecture Changes + +### Write Side (Rust) + +#### Immediate Mode (`ImmediateModePartitioner`) + +`PartitionOutputStream` currently creates a new `StreamWriter` per `write_ipc_block()` call. Change to hold a persistent `StreamWriter` per partition: + +- On first flush: create `StreamWriter` with `IpcWriteOptions::try_with_compression(...)`, which writes the schema message once +- On subsequent flushes: call `writer.write(batch)` which appends only the record batch message +- On `shuffle_write()`: call `writer.finish()` to write the EOS marker, then drain the buffer to the output file + +The `StreamWriter` writes into the existing `Vec` buffer. Spill behavior is unchanged — when memory pressure triggers a spill, the buffer (now containing partial IPC stream bytes) is written to a spill file. On final write, spill files are concatenated before the remaining in-memory bytes. Since all bytes belong to the same IPC stream, concatenation produces a valid stream. + +#### Buffered Mode (`MultiPartitionShuffleRepartitioner`) + +`PartitionWriter` wraps `ShuffleBlockWriter`, which creates a new `StreamWriter` per `write_batch()` call. Change to hold a persistent `StreamWriter` per partition: + +- Each `PartitionWriter` creates a `StreamWriter` on first batch write +- Subsequent batches append record batch messages only +- On final write: finish the stream and write to the output file + +#### `ShuffleBlockWriter` + +This struct is eliminated. Its responsibilities (compression codec selection, header writing, per-batch IPC encoding) are replaced by: +- `IpcWriteOptions` for compression codec selection +- Arrow `StreamWriter` for IPC encoding +- No custom headers needed + +#### `SinglePartitionShufflePartitioner` + +Same change — use a persistent `StreamWriter` for the single output partition. + +### Read Side + +#### New: `JniInputStream` (Rust) + +A Rust struct that wraps a JVM `InputStream` object reference and implements `std::io::Read`: + +```rust +struct JniInputStream { + jvm: JavaVM, + stream: GlobalRef, // reference to JVM InputStream + buffer: Vec, // internal read-ahead buffer (e.g., 64KB) + pos: usize, + len: usize, +} + +impl Read for JniInputStream { + fn read(&mut self, buf: &mut [u8]) -> io::Result { + // If internal buffer is exhausted, refill via JNI: + // byte[] jbuf = new byte[capacity]; + // int n = inputStream.read(jbuf, 0, capacity); + // Copy bytes from jbuf into self.buffer + // Then copy from self.buffer into buf + } +} +``` + +The internal buffer minimizes JNI boundary crossings. A 64KB buffer means one JNI call per 64KB of data, which keeps overhead negligible. + +#### New: JNI Function `decodeShuffleStream` + +```rust +pub unsafe extern "system" fn Java_org_apache_comet_Native_decodeShuffleStream( + e: JNIEnv, + _class: JClass, + input_stream: JObject, // JVM InputStream + array_addrs: JLongArray, + schema_addrs: JLongArray, + tracing_enabled: jboolean, +) -> jlong { + // 1. Wrap input_stream in JniInputStream + // 2. Create Arrow StreamReader over it + // 3. Read next batch from the stream + // 4. Export batch via Arrow FFI (prepare_output) + // 5. Return row count (or 0 if stream exhausted) +} +``` + +This function is called repeatedly by the JVM iterator until it returns 0 (stream exhausted). The `StreamReader` must persist across calls — either by storing it in a native handle that the JVM holds, or by restructuring the JVM iterator to make a single JNI call that iterates all batches. + +**Recommended approach:** Use a native handle pattern: +1. `openShuffleStream(inputStream) -> long handle` — creates `JniInputStream` + `StreamReader`, returns opaque handle +2. `nextShuffleStreamBatch(handle, arrayAddrs, schemaAddrs) -> long rowCount` — reads next batch, exports via FFI +3. `closeShuffleStream(handle)` — drops the reader and stream + +This avoids recreating the `StreamReader` per batch and cleanly manages the lifecycle. + +#### JVM: `NativeBatchDecoderIterator` + +Simplify to use the native handle pattern: + +```scala +case class NativeBatchDecoderIterator(in: InputStream, ...) extends Iterator[ColumnarBatch] { + // On construction: call nativeLib.openShuffleStream(in) to get handle + private val handle = nativeLib.openShuffleStream(in) + + private def fetchNext(): Option[ColumnarBatch] = { + // Call nativeLib.nextShuffleStreamBatch(handle, arrayAddrs, schemaAddrs) + // Returns batch or None if stream exhausted + } + + def close(): Unit = { + nativeLib.closeShuffleStream(handle) + in.close() + } +} +``` + +Remove all manual header parsing (length bytes, field count, codec tag). + +#### Removed + +- `read_ipc_compressed()` in `ipc.rs` — no longer needed +- `decodeShuffleBlock` JNI function — replaced by the stream-based API +- Custom header format parsing in `NativeBatchDecoderIterator` + +### Compression + +#### Arrow IPC Body Compression + +Arrow IPC supports per-buffer compression via `IpcWriteOptions`: + +```rust +let options = IpcWriteOptions::try_with_compression( + Some(CompressionType::LZ4_FRAME) // or CompressionType::ZSTD +)?; +let writer = StreamWriter::try_new_with_options(output, &schema, options)?; +``` + +Each record batch's data buffers are individually compressed. The schema message and IPC framing metadata are not compressed (they're small). The `StreamReader` handles decompression transparently. + +#### Supported Codecs + +- **LZ4_FRAME** — fast compression/decompression, moderate ratio +- **ZSTD** — better compression ratio, slightly slower +- **None** — no compression + +#### Dropped Codec + +- **Snappy** — not supported by Arrow IPC body compression. This is acceptable because LZ4 provides similar speed characteristics with better compression ratios. The `CompressionCodec::Snappy` variant and all Snappy-related code paths are removed. + +### Configuration + +Map the existing `spark.comet.exec.shuffle.compression.codec` config (`COMET_EXEC_SHUFFLE_COMPRESSION_CODEC`) to Arrow IPC compression types: + +| Config Value | Arrow IPC CompressionType | +|---|---| +| `lz4` | `CompressionType::LZ4_FRAME` | +| `zstd` | `CompressionType::ZSTD` | +| `none` | `None` | +| `snappy` | Error or fall back to `LZ4_FRAME` with warning | + +### Format Compatibility + +This is a **breaking change** to the shuffle format. The new format is not readable by old readers and vice versa. This is acceptable because: + +- Shuffle data is ephemeral — it exists only for the duration of a job +- There is no cross-version shuffle data exchange +- All writers and readers within a single Comet deployment use the same version + +### What Stays the Same + +- **Index file format** — partition offset table written at end of shuffle write +- **Block fetching** — `ShuffleBlockFetcherIterator` and Spark's shuffle block resolution +- **Partition assignment** — hash/range partitioning logic in both modes +- **Spill file handling** — immediate mode spill/restore behavior (though spill files now contain partial IPC stream bytes) +- **`readAsRawStream()`** — still concatenates partition InputStreams; now each stream is a valid Arrow IPC stream + +## Files Modified + +### Rust (native/) + +| File | Change | +|---|---| +| `shuffle/src/partitioners/immediate_mode.rs` | `PartitionOutputStream`: persistent `StreamWriter`, remove custom headers, use `IpcWriteOptions` compression | +| `shuffle/src/partitioners/multi_partition.rs` | `PartitionWriter`: persistent `StreamWriter` per partition | +| `shuffle/src/partitioners/single_partition.rs` | Same persistent `StreamWriter` change | +| `shuffle/src/writers/shuffle_block_writer.rs` | **Remove** — replaced by direct `StreamWriter` usage | +| `shuffle/src/ipc.rs` | **Remove** `read_ipc_compressed` — no longer needed | +| `core/src/execution/jni_api.rs` | Replace `decodeShuffleBlock` with `openShuffleStream`, `nextShuffleStreamBatch`, `closeShuffleStream` | +| `shuffle/src/lib.rs` | New `JniInputStream` struct, update exports | + +### Scala (spark/) + +| File | Change | +|---|---| +| `NativeBatchDecoderIterator.scala` | Replace header parsing with native handle pattern (`open`/`next`/`close`) | +| `Native.scala` (or equivalent) | Add new JNI method declarations | +| `CometConf.scala` | Update codec config to map to Arrow IPC types, deprecate Snappy | + +### Removed + +| File | Reason | +|---|---| +| `shuffle/src/writers/shuffle_block_writer.rs` | Replaced by persistent `StreamWriter` with `IpcWriteOptions` | +| `shuffle/src/ipc.rs` (partially) | `read_ipc_compressed` no longer needed | + +## Testing + +- Existing shuffle tests should pass with the new format (they test end-to-end behavior, not wire format) +- Add unit test for `JniInputStream` — mock a JVM `InputStream` and verify `Read` impl +- Add integration test verifying a partition with multiple batches produces a valid Arrow IPC stream +- Verify shuffle benchmark shows reduced file size and improved performance with default batch size +- Test with LZ4, ZSTD, and no compression From 38e5285ba38a0bdddf9bf62a14150897c7208113 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Apr 2026 09:12:27 -0600 Subject: [PATCH 34/46] docs: add validation skip requirement to shuffle stream reader spec --- .../specs/2026-04-02-shuffle-ipc-stream-per-partition-design.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/superpowers/specs/2026-04-02-shuffle-ipc-stream-per-partition-design.md b/docs/superpowers/specs/2026-04-02-shuffle-ipc-stream-per-partition-design.md index ca2f0839fa..4707cf42ca 100644 --- a/docs/superpowers/specs/2026-04-02-shuffle-ipc-stream-per-partition-design.md +++ b/docs/superpowers/specs/2026-04-02-shuffle-ipc-stream-per-partition-design.md @@ -122,6 +122,8 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_decodeShuffleStream( } ``` +The `StreamReader` must be created with validation disabled (`with_skip_validation(true)`) since we control both the write and read sides and validation adds unnecessary overhead. + This function is called repeatedly by the JVM iterator until it returns 0 (stream exhausted). The `StreamReader` must persist across calls — either by storing it in a native handle that the JVM holds, or by restructuring the JVM iterator to make a single JNI call that iterates all batches. **Recommended approach:** Use a native handle pattern: From 26a9032e99fb7898e48c7aa40cf2cb9af2115fc8 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Apr 2026 09:21:07 -0600 Subject: [PATCH 35/46] docs: implementation plan for IPC stream per partition shuffle format --- ...-04-02-shuffle-ipc-stream-per-partition.md | 1438 +++++++++++++++++ 1 file changed, 1438 insertions(+) create mode 100644 docs/superpowers/plans/2026-04-02-shuffle-ipc-stream-per-partition.md diff --git a/docs/superpowers/plans/2026-04-02-shuffle-ipc-stream-per-partition.md b/docs/superpowers/plans/2026-04-02-shuffle-ipc-stream-per-partition.md new file mode 100644 index 0000000000..a6fae86cff --- /dev/null +++ b/docs/superpowers/plans/2026-04-02-shuffle-ipc-stream-per-partition.md @@ -0,0 +1,1438 @@ +# Shuffle IPC Stream Per Partition — 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:** Reduce shuffle format overhead by writing one Arrow IPC stream per partition (schema once, N batches) instead of one IPC stream per batch, and move IPC stream reading to Rust via JNI InputStream callbacks. + +**Architecture:** Replace the custom header format + per-batch IPC stream with standard Arrow IPC streams using built-in body compression (LZ4_FRAME/ZSTD). On the read side, Rust pulls bytes from JVM's InputStream via a `JniInputStream` adapter and uses Arrow's `StreamReader` to decode batches. A native handle pattern (`open`/`next`/`close`) manages the reader lifecycle across JNI calls. + +**Tech Stack:** Arrow IPC (`StreamWriter`/`StreamReader` with `IpcWriteOptions`), JNI (`jni` crate), Arrow FFI + +**Spec:** `docs/superpowers/specs/2026-04-02-shuffle-ipc-stream-per-partition-design.md` + +--- + +## File Structure + +### Rust — Modified + +| File | Responsibility | +|---|---| +| `native/Cargo.toml` | Add `ipc_compression` feature to arrow dependency | +| `native/shuffle/Cargo.toml` | Remove `snap` dependency | +| `native/shuffle/src/lib.rs` | Update exports (remove `read_ipc_compressed`, `ShuffleBlockWriter`) | +| `native/shuffle/src/writers/mod.rs` | Remove `shuffle_block_writer` module, remove `ShuffleBlockWriter` export | +| `native/shuffle/src/writers/shuffle_block_writer.rs` | **Delete** — replaced by persistent `StreamWriter` | +| `native/shuffle/src/writers/buf_batch_writer.rs` | Remove `ShuffleBlockWriter` dependency, write directly via `StreamWriter` | +| `native/shuffle/src/writers/spill.rs` | Remove `ShuffleBlockWriter` dependency, write via `StreamWriter` | +| `native/shuffle/src/ipc.rs` | Replace `read_ipc_compressed` with `JniInputStream` and stream reader handle | +| `native/shuffle/src/partitioners/immediate_mode.rs` | `PartitionOutputStream`: persistent `StreamWriter`, Arrow-native compression | +| `native/shuffle/src/partitioners/single_partition.rs` | Use persistent `StreamWriter` directly | +| `native/shuffle/src/partitioners/multi_partition.rs` | Use persistent `StreamWriter` per partition | +| `native/shuffle/src/shuffle_writer.rs` | Update `CompressionCodec` → Arrow IPC `CompressionType`, update tests | +| `native/core/src/execution/jni_api.rs` | Replace `decodeShuffleBlock` with `openShuffleStream`/`nextShuffleStreamBatch`/`closeShuffleStream` | + +### Scala — Modified + +| File | Responsibility | +|---|---| +| `spark/src/main/scala/org/apache/comet/Native.scala` | Replace `decodeShuffleBlock` with new JNI method declarations | +| `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala` | Use native handle pattern, remove header parsing | +| `common/src/main/scala/org/apache/comet/CometConf.scala` | Remove `snappy` from codec options | + +--- + +## Task 1: Enable Arrow IPC Compression Feature + +**Files:** +- Modify: `native/Cargo.toml:37` + +- [ ] **Step 1: Add `ipc_compression` feature to arrow dependency** + +In `native/Cargo.toml`, change line 37 from: +```toml +arrow = { version = "57.3.0", features = ["prettyprint", "ffi", "chrono-tz"] } +``` +to: +```toml +arrow = { version = "57.3.0", features = ["prettyprint", "ffi", "chrono-tz", "ipc_compression"] } +``` + +- [ ] **Step 2: Verify it compiles** + +Run: `cargo check --manifest-path native/Cargo.toml` +Expected: Compilation succeeds with no errors. + +- [ ] **Step 3: Commit** + +```bash +git add native/Cargo.toml +git commit -m "feat: enable Arrow IPC compression feature for shuffle format" +``` + +--- + +## Task 2: Replace `CompressionCodec` with Arrow IPC Compression + +The existing `CompressionCodec` enum in `shuffle_block_writer.rs` manages Snappy/LZ4/Zstd/None with external compression wrappers. Replace it with a thin wrapper around Arrow IPC's `CompressionType` so the `StreamWriter` handles compression internally. + +**Files:** +- Modify: `native/shuffle/src/writers/shuffle_block_writer.rs` (will be replaced with just the codec enum) +- Modify: `native/shuffle/src/writers/mod.rs` +- Modify: `native/shuffle/src/lib.rs` + +- [ ] **Step 1: Replace `CompressionCodec` in `shuffle_block_writer.rs`** + +Replace the entire contents of `native/shuffle/src/writers/shuffle_block_writer.rs` with: + +```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 arrow::ipc::writer::IpcWriteOptions; +use arrow::ipc::CompressionType; + +/// Compression codec for shuffle IPC streams. +/// +/// Maps to Arrow IPC's built-in body compression. Each record batch's data +/// buffers are individually compressed by the `StreamWriter`. +#[derive(Debug, Clone)] +pub enum CompressionCodec { + None, + Lz4Frame, + Zstd(i32), +} + +impl CompressionCodec { + /// Create `IpcWriteOptions` with the appropriate compression setting. + pub fn ipc_write_options(&self) -> datafusion::error::Result { + let compression = match self { + CompressionCodec::None => None, + CompressionCodec::Lz4Frame => Some(CompressionType::LZ4_FRAME), + CompressionCodec::Zstd(_) => Some(CompressionType::ZSTD), + }; + IpcWriteOptions::try_with_compression(8, false, arrow::ipc::MetadataVersion::V5, compression) + .map_err(|e| datafusion::common::DataFusionError::ArrowError(Box::from(e), None)) + } +} +``` + +- [ ] **Step 2: Update `writers/mod.rs` exports** + +Replace the contents of `native/shuffle/src/writers/mod.rs`: + +```rust +// (keep license header) + +mod buf_batch_writer; +mod checksum; +mod shuffle_block_writer; +mod spill; + +pub(crate) use buf_batch_writer::BufBatchWriter; +pub(crate) use checksum::Checksum; +pub use shuffle_block_writer::CompressionCodec; +pub(crate) use spill::PartitionWriter; +``` + +Note: `ShuffleBlockWriter` is no longer exported — only `CompressionCodec` remains. + +- [ ] **Step 3: Update `lib.rs` exports** + +In `native/shuffle/src/lib.rs`, change: +```rust +pub use writers::{CompressionCodec, ShuffleBlockWriter}; +``` +to: +```rust +pub use writers::CompressionCodec; +``` + +Also remove the `read_ipc_compressed` export: +```rust +pub use ipc::read_ipc_compressed; +``` +becomes: +```rust +// (remove this line entirely — will be replaced in Task 6) +``` + +- [ ] **Step 4: Verify it compiles (expect errors in dependent code)** + +Run: `cargo check --manifest-path native/Cargo.toml 2>&1 | head -50` +Expected: Errors in files that still reference `ShuffleBlockWriter` and `read_ipc_compressed`. This is expected — we'll fix them in subsequent tasks. + +- [ ] **Step 5: Commit** + +```bash +git add native/shuffle/src/writers/shuffle_block_writer.rs native/shuffle/src/writers/mod.rs native/shuffle/src/lib.rs +git commit -m "refactor: replace ShuffleBlockWriter with CompressionCodec wrapper for Arrow IPC" +``` + +--- + +## Task 3: Update `PartitionOutputStream` (Immediate Mode) to Persistent StreamWriter + +Currently `PartitionOutputStream` creates a new `StreamWriter` per `write_ipc_block()` call. Change it to hold a persistent `StreamWriter` that writes the schema once and appends record batch messages for subsequent writes. + +**Files:** +- Modify: `native/shuffle/src/partitioners/immediate_mode.rs:284-369` (the `PartitionOutputStream` struct and impl) + +- [ ] **Step 1: Replace `PartitionOutputStream`** + +Replace the `PartitionOutputStream` struct and its `impl` block (lines 284-369) with: + +```rust +pub(crate) struct PartitionOutputStream { + /// Buffer that the StreamWriter writes into. + buffer: Vec, + /// Persistent Arrow IPC stream writer. Created lazily on first write. + writer: Option>>, + /// IPC write options with compression settings. + write_options: IpcWriteOptions, + /// Schema for creating the writer on first use. + schema: SchemaRef, +} + +impl PartitionOutputStream { + pub(crate) fn try_new(schema: SchemaRef, codec: CompressionCodec) -> Result { + let write_options = codec.ipc_write_options()?; + Ok(Self { + buffer: Vec::new(), + writer: None, + write_options, + schema, + }) + } + + fn write_batch(&mut self, batch: &RecordBatch) -> Result { + let start_pos = self.buffer.len(); + + if self.writer.is_none() { + // SAFETY: We ensure the buffer outlives the writer by owning both + // in this struct and only dropping the writer before the buffer. + let buf_ptr: *mut Vec = &mut self.buffer; + let buf_ref: &'static mut Vec = unsafe { &mut *buf_ptr }; + let w = StreamWriter::try_new_with_options(buf_ref, &self.schema, self.write_options.clone())?; + self.writer = Some(w); + } + + self.writer.as_mut().unwrap().write(batch)?; + + Ok(self.buffer.len() - start_pos) + } + + /// Finish the IPC stream (writes EOS marker) and return the buffer. + fn finish(&mut self) -> Result> { + if let Some(mut w) = self.writer.take() { + w.finish()?; + } + Ok(std::mem::take(&mut self.buffer)) + } + + fn drain_buffer(&mut self) -> Vec { + // When draining mid-stream (for spilling), we take the buffer contents + // but keep the writer alive. The writer's internal reference still points + // to self.buffer, which is now empty but still valid. + std::mem::take(&mut self.buffer) + } + + fn buffer_len(&self) -> usize { + self.buffer.len() + } +} +``` + +**Important:** The unsafe lifetime trick with the buffer pointer is needed because `StreamWriter` borrows the `Write` target. We need the writer to persist across calls while the buffer is drained. An alternative is to use `Cursor>` or a shared buffer wrapper — the implementer should evaluate which is cleanest. If the unsafe approach is concerning, consider using `StreamWriter>` and calling `into_inner()` when draining. + +**Simpler alternative using `StreamWriter>`:** + +Actually, a cleaner approach: since `Vec` implements `Write`, we can have the `StreamWriter` own the `Vec`. When we need to drain, we `finish()` the writer, take the buffer, and set `writer = None` so the next write creates a fresh stream. This means each spill creates a separate IPC stream. On read, the reader sees multiple concatenated IPC streams per partition (from spills), which Arrow `StreamReader` handles — it reads until EOS, then the JVM-side iterator moves to the next stream. + +Replace the above with this cleaner version: + +```rust +pub(crate) struct PartitionOutputStream { + /// Persistent Arrow IPC stream writer. Owns the buffer. + /// None before first write or after a drain (spill). + writer: Option>>, + /// IPC write options with compression settings. + write_options: IpcWriteOptions, + /// Schema for creating the writer. + schema: SchemaRef, + /// Accumulated bytes from finished (drained) streams. + spilled_bytes: Vec, +} + +impl PartitionOutputStream { + pub(crate) fn try_new(schema: SchemaRef, codec: CompressionCodec) -> Result { + let write_options = codec.ipc_write_options()?; + Ok(Self { + writer: None, + write_options, + schema, + spilled_bytes: Vec::new(), + }) + } + + fn ensure_writer(&mut self) -> Result<()> { + if self.writer.is_none() { + let w = StreamWriter::try_new_with_options( + Vec::new(), + &self.schema, + self.write_options.clone(), + )?; + self.writer = Some(w); + } + Ok(()) + } + + fn write_batch(&mut self, batch: &RecordBatch) -> Result { + self.ensure_writer()?; + let w = self.writer.as_mut().unwrap(); + // Get position before write to calculate bytes written + // StreamWriter writes to Vec which grows, so track via get_ref().len() + let before = w.get_ref().len(); + w.write(batch)?; + let after = w.get_ref().len(); + Ok(after - before) + } + + /// Finish the current IPC stream and drain all accumulated bytes. + /// After this call, the next write_batch creates a new IPC stream. + fn drain_buffer(&mut self) -> Result> { + let mut result = std::mem::take(&mut self.spilled_bytes); + if let Some(mut w) = self.writer.take() { + w.finish()?; + result.extend(w.into_inner()?); + } + Ok(result) + } + + /// Finish the IPC stream and return all bytes (spilled + in-memory). + fn finish(mut self) -> Result> { + self.drain_buffer() + } + + fn buffer_len(&self) -> usize { + self.spilled_bytes.len() + + self.writer.as_ref().map_or(0, |w| w.get_ref().len()) + } +} +``` + +- [ ] **Step 2: Update `flush_partition` to use new API** + +In `ImmediateModePartitioner::flush_partition` (around line 600), change: +```rust +let ipc_bytes = self.streams[pid].write_ipc_block(&output_batch)?; +``` +to: +```rust +let ipc_bytes = self.streams[pid].write_batch(&output_batch)?; +``` + +- [ ] **Step 3: Update `spill_all` to use `drain_buffer`** + +In `ImmediateModePartitioner::spill_all` (around line 622), the drain call changes from: +```rust +let buf = self.streams[pid].drain_buffer(); +``` +to: +```rust +let buf = self.streams[pid].drain_buffer()?; +``` +(It now returns `Result>` because it finishes the IPC stream.) + +- [ ] **Step 4: Update `shuffle_write` to use `drain_buffer`** + +In `ImmediateModePartitioner::shuffle_write` (around line 741), change: +```rust +let buf = self.streams[pid].drain_buffer(); +``` +to: +```rust +let buf = self.streams[pid].drain_buffer()?; +``` + +- [ ] **Step 5: Remove old imports** + +In `immediate_mode.rs`, remove the `CompressionCodec` import from `crate::` (if it was used for the codec tag matching) and remove `use arrow::ipc::writer::StreamWriter;` at the top — it's now used inside `PartitionOutputStream` via the `StreamWriter` from `arrow::ipc::writer`. Actually, keep the `StreamWriter` import since `PartitionOutputStream` uses it. Remove imports that are no longer needed: +- Remove the old `CompressionCodec` matching arms for `Snappy` codec tags (`b"SNAP"` etc.) + +Add new import: +```rust +use arrow::ipc::writer::{IpcWriteOptions, StreamWriter}; +``` + +- [ ] **Step 6: Update test `test_partition_output_stream_write_and_read`** + +Replace the test (around line 856) with: + +```rust +#[test] +#[cfg_attr(miri, ignore)] +fn test_partition_output_stream_write_and_read() { + use arrow::ipc::reader::StreamReader; + + let batch = make_test_batch(&[1, 2, 3, 4, 5]); + let schema = batch.schema(); + + for codec in [ + CompressionCodec::None, + CompressionCodec::Lz4Frame, + CompressionCodec::Zstd(1), + ] { + let mut stream = + PartitionOutputStream::try_new(Arc::clone(&schema), codec).unwrap(); + stream.write_batch(&batch).unwrap(); + stream.write_batch(&batch).unwrap(); // write 2 batches + + let buf = stream.finish().unwrap(); + assert!(!buf.is_empty()); + + // Read back using standard Arrow IPC StreamReader + let cursor = std::io::Cursor::new(&buf); + let reader = StreamReader::try_new(cursor, None).unwrap(); + let batches: Vec = reader.map(|r| r.unwrap()).collect(); + assert_eq!(batches.len(), 2); + assert_eq!(batches[0].num_rows(), 5); + assert_eq!(batches[1].num_rows(), 5); + } +} +``` + +- [ ] **Step 7: Run tests** + +Run: `cd native && cargo test -p datafusion-comet-shuffle test_partition_output_stream` +Expected: `test_partition_output_stream_write_and_read` passes. + +- [ ] **Step 8: Commit** + +```bash +git add native/shuffle/src/partitioners/immediate_mode.rs +git commit -m "feat: persistent StreamWriter per partition in immediate mode shuffle" +``` + +--- + +## Task 4: Update Buffered Mode (`MultiPartitionShuffleRepartitioner`) Writers + +The buffered mode uses `ShuffleBlockWriter` via `PartitionWriter` and `BufBatchWriter`. Replace these with persistent `StreamWriter` per partition. + +**Files:** +- Modify: `native/shuffle/src/writers/spill.rs` +- Modify: `native/shuffle/src/writers/buf_batch_writer.rs` +- Modify: `native/shuffle/src/partitioners/multi_partition.rs` + +- [ ] **Step 1: Rewrite `PartitionWriter` in `spill.rs`** + +Replace the entire contents of `native/shuffle/src/writers/spill.rs` with: + +```rust +// (keep license header) + +use arrow::array::RecordBatch; +use arrow::datatypes::SchemaRef; +use arrow::ipc::writer::{IpcWriteOptions, StreamWriter}; +use datafusion::common::DataFusionError; +use datafusion::execution::disk_manager::RefCountedTempFile; +use datafusion::execution::runtime_env::RuntimeEnv; +use datafusion::physical_plan::metrics::Time; +use std::fs::{File, OpenOptions}; +use std::io::Write; + +use crate::partitioners::PartitionedBatchIterator; +use crate::metrics::ShufflePartitionerMetrics; + +/// A temporary disk file for spilling a partition's intermediate shuffle data. +struct SpillFile { + temp_file: RefCountedTempFile, + file: File, +} + +/// Manages encoding and optional disk spilling for a single shuffle partition. +/// Each partition gets one IPC stream (schema written once), with spills creating +/// additional IPC streams that are concatenated in the final output. +pub(crate) struct PartitionWriter { + spill_file: Option, + schema: SchemaRef, + write_options: IpcWriteOptions, +} + +impl PartitionWriter { + pub(crate) fn try_new( + schema: SchemaRef, + write_options: IpcWriteOptions, + ) -> datafusion::common::Result { + Ok(Self { + spill_file: None, + schema, + write_options, + }) + } + + fn ensure_spill_file_created( + &mut self, + runtime: &RuntimeEnv, + ) -> datafusion::common::Result<()> { + if self.spill_file.is_none() { + let spill_file = runtime + .disk_manager + .create_tmp_file("shuffle writer spill")?; + let spill_data = OpenOptions::new() + .write(true) + .create(true) + .truncate(true) + .open(spill_file.path()) + .map_err(|e| { + DataFusionError::Execution(format!("Error occurred while spilling {e}")) + })?; + self.spill_file = Some(SpillFile { + temp_file: spill_file, + file: spill_data, + }); + } + Ok(()) + } + + pub(crate) fn spill( + &mut self, + iter: &mut PartitionedBatchIterator, + runtime: &RuntimeEnv, + metrics: &ShufflePartitionerMetrics, + ) -> datafusion::common::Result { + if let Some(batch) = iter.next() { + self.ensure_spill_file_created(runtime)?; + + let file = &mut self.spill_file.as_mut().unwrap().file; + let mut writer = StreamWriter::try_new_with_options( + file, + &self.schema, + self.write_options.clone(), + )?; + + let batch = batch?; + let mut encode_timer = metrics.encode_time.timer(); + writer.write(&batch)?; + encode_timer.stop(); + + for batch in iter { + let batch = batch?; + let mut encode_timer = metrics.encode_time.timer(); + writer.write(&batch)?; + encode_timer.stop(); + } + + let mut encode_timer = metrics.encode_time.timer(); + writer.finish()?; + let file = writer.into_inner()?; + file.flush()?; + encode_timer.stop(); + + let bytes_written = file.metadata() + .map(|m| m.len() as usize) + .unwrap_or(0); + + Ok(bytes_written) + } else { + Ok(0) + } + } + + pub(crate) fn path(&self) -> Option<&std::path::Path> { + self.spill_file + .as_ref() + .map(|spill_file| spill_file.temp_file.path()) + } + + #[cfg(test)] + pub(crate) fn has_spill_file(&self) -> bool { + self.spill_file.is_some() + } +} +``` + +- [ ] **Step 2: Rewrite `BufBatchWriter` in `buf_batch_writer.rs`** + +The `BufBatchWriter` now wraps a `StreamWriter` directly instead of a `ShuffleBlockWriter`. Replace the entire contents of `native/shuffle/src/writers/buf_batch_writer.rs` with: + +```rust +// (keep license header) + +use arrow::array::RecordBatch; +use arrow::compute::kernels::coalesce::BatchCoalescer; +use arrow::datatypes::SchemaRef; +use arrow::ipc::writer::{IpcWriteOptions, StreamWriter}; +use datafusion::physical_plan::metrics::Time; +use std::io::Write; + +/// Write batches to an Arrow IPC stream with coalescing. +/// +/// Small batches are coalesced using Arrow's [`BatchCoalescer`] before serialization, +/// producing exactly `batch_size`-row output batches to reduce per-batch overhead. +/// Uses a persistent `StreamWriter` — schema is written once. +pub(crate) struct BufBatchWriter { + writer: StreamWriter, + /// Coalesces small batches into target_batch_size before serialization. + coalescer: Option, + schema: SchemaRef, + batch_size: usize, +} + +impl BufBatchWriter { + pub(crate) fn new( + inner: W, + schema: &SchemaRef, + write_options: &IpcWriteOptions, + batch_size: usize, + ) -> datafusion::common::Result { + let writer = StreamWriter::try_new_with_options( + inner, + schema, + write_options.clone(), + )?; + Ok(Self { + writer, + coalescer: None, + schema: schema.clone(), + batch_size, + }) + } + + pub(crate) fn write( + &mut self, + batch: &RecordBatch, + encode_time: &Time, + _write_time: &Time, + ) -> datafusion::common::Result { + let coalescer = self + .coalescer + .get_or_insert_with(|| BatchCoalescer::new(self.schema.clone(), self.batch_size)); + coalescer.push_batch(batch.clone())?; + + let mut completed = Vec::new(); + while let Some(batch) = coalescer.next_completed_batch() { + completed.push(batch); + } + + let mut bytes_written = 0; + for batch in &completed { + let mut timer = encode_time.timer(); + let before = self.writer.get_ref().len(); + self.writer.write(batch)?; + let after = self.writer.get_ref().len(); + bytes_written += after - before; + timer.stop(); + } + Ok(bytes_written) + } + + pub(crate) fn flush( + &mut self, + encode_time: &Time, + _write_time: &Time, + ) -> datafusion::common::Result<()> { + let mut remaining = Vec::new(); + if let Some(coalescer) = &mut self.coalescer { + coalescer.finish_buffered_batch()?; + while let Some(batch) = coalescer.next_completed_batch() { + remaining.push(batch); + } + } + for batch in &remaining { + let mut timer = encode_time.timer(); + self.writer.write(batch)?; + timer.stop(); + } + + let mut timer = encode_time.timer(); + self.writer.finish()?; + timer.stop(); + Ok(()) + } +} +``` + +**Note:** The `write()` return value (bytes written) is only used for metrics. Since the `StreamWriter` writes directly to the underlying `W`, and for file-backed writers we track offsets via `stream_position()`, the return value from `write()` can be dropped. Change the signature to return `Result<()>` and remove the byte counting logic. Callers that previously used the return value for offset tracking should use `stream_position()` on the underlying file instead. + +- [ ] **Step 3: Update `writers/mod.rs`** + +```rust +// (keep license header) + +mod buf_batch_writer; +mod checksum; +mod shuffle_block_writer; +mod spill; + +pub(crate) use buf_batch_writer::BufBatchWriter; +pub(crate) use checksum::Checksum; +pub use shuffle_block_writer::CompressionCodec; +pub(crate) use spill::PartitionWriter; +``` + +(No change needed — `BufBatchWriter` and `PartitionWriter` are still exported with the same names.) + +- [ ] **Step 4: Update `MultiPartitionShuffleRepartitioner`** + +In `native/shuffle/src/partitioners/multi_partition.rs`: + +Change the struct fields — remove `shuffle_block_writer: ShuffleBlockWriter` and add `write_options: IpcWriteOptions`: + +At approximately line 113, replace: +```rust + shuffle_block_writer: ShuffleBlockWriter, +``` +with: +```rust + write_options: IpcWriteOptions, +``` + +In `try_new`, change the construction of `partition_writers` from: +```rust +let shuffle_block_writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone())?; +// ... +.map(|_| PartitionWriter::try_new(shuffle_block_writer.clone())) +``` +to: +```rust +let write_options = codec.ipc_write_options()?; +// ... +.map(|_| PartitionWriter::try_new(schema.clone(), write_options.clone())) +``` + +Update `shuffle_write_partition` to create a `BufBatchWriter` with the new API: +```rust +fn shuffle_write_partition( + partition_iter: &mut PartitionedBatchIterator, + output_data: &mut BufWriter, + schema: &SchemaRef, + write_options: &IpcWriteOptions, + encode_time: &Time, + write_time: &Time, + batch_size: usize, +) -> datafusion::common::Result<()> { + let mut buf_batch_writer = BufBatchWriter::new( + output_data, + schema, + write_options, + batch_size, + )?; + for batch in partition_iter { + let batch = batch?; + buf_batch_writer.write(&batch, encode_time, write_time)?; + } + buf_batch_writer.flush(encode_time, write_time)?; + Ok(()) +} +``` + +Update the call site in `shuffle_write` (around line 595): +```rust +Self::shuffle_write_partition( + &mut partition_iter, + &mut output_data, + &self.schema, + &self.write_options, + &self.metrics.encode_time, + &self.metrics.write_time, + self.batch_size, +)?; +``` + +Update `spill` to pass the new args: +```rust +spilled_bytes += partition_writer.spill( + &mut iter, + &self.runtime, + &self.metrics, +)?; +``` +(Remove `write_buffer_size` and `batch_size` args from `spill` — the spill writer now creates its own `StreamWriter`.) + +- [ ] **Step 5: Update `SinglePartitionShufflePartitioner`** + +In `native/shuffle/src/partitioners/single_partition.rs`, change the struct to hold a `BufBatchWriter` created with the new API: + +```rust +pub(crate) struct SinglePartitionShufflePartitioner { + output_data_writer: BufBatchWriter, + output_index_path: String, + metrics: ShufflePartitionerMetrics, + batch_size: usize, +} +``` + +Update `try_new`: +```rust +pub(crate) fn try_new( + output_data_path: String, + output_index_path: String, + schema: SchemaRef, + metrics: ShufflePartitionerMetrics, + batch_size: usize, + codec: CompressionCodec, + _write_buffer_size: usize, +) -> datafusion::common::Result { + let write_options = codec.ipc_write_options()?; + let output_data_file = OpenOptions::new() + .write(true) + .create(true) + .truncate(true) + .open(output_data_path)?; + + let output_data_writer = BufBatchWriter::new( + output_data_file, + &schema, + &write_options, + batch_size, + )?; + + Ok(Self { + output_data_writer, + output_index_path, + metrics, + batch_size, + }) +} +``` + +Remove `buffered_batches`, `num_buffered_rows`, `add_buffered_batch`, and `concat_buffered_batches` — the `BufBatchWriter` handles coalescing internally. Simplify `insert_batch` to just call `self.output_data_writer.write(...)`. + +- [ ] **Step 6: Verify Rust compilation** + +Run: `cargo check --manifest-path native/Cargo.toml` +Expected: Compiles with only remaining errors in `jni_api.rs` (the read side, Task 6). + +- [ ] **Step 7: Run existing shuffle writer tests** + +Run: `cd native && cargo test -p datafusion-comet-shuffle` +Expected: All write-side tests pass. Read-side tests (`read_ipc_compressed`) will fail — expected. + +- [ ] **Step 8: Commit** + +```bash +git add native/shuffle/src/writers/ native/shuffle/src/partitioners/ +git commit -m "feat: persistent StreamWriter per partition in buffered mode shuffle" +``` + +--- + +## Task 5: Remove Snappy Support from Config and Dependencies + +**Files:** +- Modify: `native/shuffle/Cargo.toml` +- Modify: `common/src/main/scala/org/apache/comet/CometConf.scala:433-442` + +- [ ] **Step 1: Remove `snap` dependency from Cargo.toml** + +In `native/shuffle/Cargo.toml`, remove line: +```toml +snap = "1.1" +``` + +- [ ] **Step 2: Update CometConf to remove snappy** + +In `common/src/main/scala/org/apache/comet/CometConf.scala`, change line 441: +```scala + .checkValues(Set("zstd", "lz4", "snappy")) +``` +to: +```scala + .checkValues(Set("zstd", "lz4")) +``` + +And update the doc string (line 437-438): +```scala + "The codec of Comet native shuffle used to compress shuffle data. lz4, zstd, and " + + "snappy are supported. Compression can be disabled by setting " + +``` +to: +```scala + "The codec of Comet native shuffle used to compress shuffle data. " + + "lz4 and zstd are supported. Compression can be disabled by setting " + +``` + +- [ ] **Step 3: Remove any remaining Snappy references in Rust code** + +Search for and remove any `snap::` or `Snappy` references in the shuffle crate. After Tasks 2-4, there should be none left in the writer code. Check with: + +Run: `grep -rn "snap\|Snappy\|SNAP" native/shuffle/src/` + +Remove any remaining references. + +- [ ] **Step 4: Commit** + +```bash +git add native/shuffle/Cargo.toml common/src/main/scala/org/apache/comet/CometConf.scala +git commit -m "chore: remove Snappy codec support from shuffle (Arrow IPC supports LZ4/ZSTD)" +``` + +--- + +## Task 6: Implement `JniInputStream` and Stream Reader Handle (Rust Read Side) + +Create the Rust-side JNI infrastructure for reading Arrow IPC streams from JVM InputStreams. + +**Files:** +- Modify: `native/shuffle/src/ipc.rs` — replace `read_ipc_compressed` with `JniInputStream` and `ShuffleStreamReader` +- Modify: `native/core/src/execution/jni_api.rs` — replace `decodeShuffleBlock` with new JNI functions + +- [ ] **Step 1: Rewrite `ipc.rs` with `JniInputStream` and `ShuffleStreamReader`** + +Replace the entire contents of `native/shuffle/src/ipc.rs` with: + +```rust +// (keep license header) + +use arrow::array::RecordBatch; +use arrow::ipc::reader::StreamReader; +use jni::objects::{GlobalRef, JByteArray, JObject, JValue}; +use jni::sys::jint; +use jni::{JNIEnv, JavaVM}; +use std::io::{self, Read}; + +/// Buffer size for JNI read-ahead. Minimizes JNI boundary crossings. +const JNI_READ_BUFFER_SIZE: usize = 64 * 1024; // 64KB + +/// A `Read` adapter that pulls bytes from a JVM `InputStream` via JNI callbacks. +/// +/// Uses an internal read-ahead buffer to minimize JNI call overhead. +pub struct JniInputStream { + jvm: JavaVM, + stream: GlobalRef, + jbuf: JByteArray, + buf: Vec, + pos: usize, + len: usize, +} + +impl JniInputStream { + /// Create a new `JniInputStream` wrapping a JVM `InputStream`. + /// + /// The `input_stream` object must be a valid `java.io.InputStream`. + pub fn new(env: &mut JNIEnv, input_stream: &JObject) -> jni::errors::Result { + let jvm = env.get_java_vm()?; + let stream = env.new_global_ref(input_stream)?; + let jbuf = env.new_byte_array(JNI_READ_BUFFER_SIZE as jint)?; + let jbuf = JByteArray::from(env.new_global_ref(jbuf)?.as_ref().as_raw()); + Ok(Self { + jvm, + stream, + jbuf, + buf: vec![0u8; JNI_READ_BUFFER_SIZE], + pos: 0, + len: 0, + }) + } + + fn refill(&mut self) -> io::Result<()> { + let mut env = self.jvm.attach_current_thread().map_err(|e| { + io::Error::new(io::ErrorKind::Other, format!("JNI attach failed: {e}")) + })?; + + let n = env + .call_method( + &self.stream, + "read", + "([BII)I", + &[ + JValue::Object(self.jbuf.as_ref().into()), + JValue::Int(0), + JValue::Int(JNI_READ_BUFFER_SIZE as jint), + ], + ) + .and_then(|v| v.i()) + .map_err(|e| { + io::Error::new(io::ErrorKind::Other, format!("JNI read failed: {e}")) + })?; + + if n <= 0 { + self.pos = 0; + self.len = 0; + return Ok(()); + } + + let n = n as usize; + env.get_byte_array_region(&self.jbuf, 0, &mut self.buf[..n].iter().map(|b| *b as i8).collect::>()) + .map_err(|e| { + io::Error::new(io::ErrorKind::Other, format!("JNI get_byte_array_region failed: {e}")) + })?; + + // Actually, get_byte_array_region writes into an &mut [i8], not returns. + // Let's use a simpler approach: get_byte_array_elements or copy into our buffer. + // The cleanest approach: + let mut i8_buf = vec![0i8; n]; + env.get_byte_array_region(&self.jbuf, 0, &mut i8_buf) + .map_err(|e| { + io::Error::new(io::ErrorKind::Other, format!("JNI copy failed: {e}")) + })?; + // Reinterpret i8 as u8 + self.buf[..n].copy_from_slice(unsafe { + std::slice::from_raw_parts(i8_buf.as_ptr() as *const u8, n) + }); + + self.pos = 0; + self.len = n; + Ok(()) + } +} + +impl Read for JniInputStream { + fn read(&mut self, buf: &mut [u8]) -> io::Result { + if self.pos >= self.len { + self.refill()?; + if self.len == 0 { + return Ok(0); // EOF + } + } + + let available = self.len - self.pos; + let to_copy = available.min(buf.len()); + buf[..to_copy].copy_from_slice(&self.buf[self.pos..self.pos + to_copy]); + self.pos += to_copy; + Ok(to_copy) + } +} + +/// Holds a persistent Arrow IPC `StreamReader` across JNI calls. +/// +/// The reader is created once when the shuffle stream is opened, and +/// `next_batch()` is called repeatedly to read individual batches. +pub struct ShuffleStreamReader { + reader: StreamReader, + num_fields: usize, +} + +impl ShuffleStreamReader { + /// Create a new reader wrapping a JVM InputStream. + pub fn new(env: &mut JNIEnv, input_stream: &JObject) -> Result { + let jni_stream = JniInputStream::new(env, input_stream) + .map_err(|e| format!("Failed to create JniInputStream: {e}"))?; + + let reader = unsafe { + StreamReader::try_new(jni_stream, None) + .map_err(|e| format!("Failed to create StreamReader: {e}"))? + .with_skip_validation(true) + }; + + let num_fields = reader.schema().fields().len(); + + Ok(Self { reader, num_fields }) + } + + /// Read the next batch from the stream. + /// Returns `None` when the stream is exhausted. + pub fn next_batch(&mut self) -> Result, String> { + match self.reader.next() { + Some(Ok(batch)) => Ok(Some(batch)), + Some(Err(e)) => Err(format!("Error reading batch: {e}")), + None => Ok(None), + } + } + + /// Number of fields (columns) in the schema. + pub fn num_fields(&self) -> usize { + self.num_fields + } +} +``` + +**Note to implementer:** The `JByteArray` handling in `refill()` above is pseudocode — the exact JNI API for byte array handling with the `jni` crate needs careful implementation. The `jni` crate's `get_byte_array_region` takes `&mut [jbyte]` (which is `&mut [i8]`). The key pattern is: +1. Call `inputStream.read(byte[], 0, len)` via `call_method` +2. Copy the Java byte[] contents to the Rust buffer via `get_byte_array_region` +3. Reinterpret `i8` as `u8` + +Also, the `JByteArray` from a `GlobalRef` conversion needs care — the implementer should verify this compiles. A `GlobalRef` to the byte array must be stored, and a local ref obtained for each JNI call. + +- [ ] **Step 2: Update `lib.rs` exports** + +In `native/shuffle/src/lib.rs`, replace: +```rust +pub use ipc::read_ipc_compressed; +``` +with: +```rust +pub use ipc::{JniInputStream, ShuffleStreamReader}; +``` + +- [ ] **Step 3: Add JNI functions in `jni_api.rs`** + +In `native/core/src/execution/jni_api.rs`, replace the `decodeShuffleBlock` function (lines 878-900) with three new functions: + +```rust +#[no_mangle] +/// Open a shuffle stream reader over a JVM InputStream. +/// Returns an opaque handle (pointer) to a ShuffleStreamReader. +/// # Safety +/// This function is inherently unsafe since it deals with JNI objects. +pub unsafe extern "system" fn Java_org_apache_comet_Native_openShuffleStream( + e: JNIEnv, + _class: JClass, + input_stream: JObject, +) -> jlong { + try_unwrap_or_throw(&e, |mut env| { + let reader = datafusion_comet_shuffle::ShuffleStreamReader::new(&mut env, &input_stream) + .map_err(|e| CometError::Internal(e))?; + let boxed = Box::new(reader); + Ok(Box::into_raw(boxed) as jlong) + }) +} + +#[no_mangle] +/// Read the next batch from a shuffle stream. +/// Returns the number of rows, or -1 if the stream is exhausted. +/// # Safety +/// This function is inherently unsafe since it deals with raw pointers from JNI. +pub unsafe extern "system" fn Java_org_apache_comet_Native_nextShuffleStreamBatch( + e: JNIEnv, + _class: JClass, + handle: jlong, + array_addrs: JLongArray, + schema_addrs: JLongArray, +) -> jlong { + try_unwrap_or_throw(&e, |mut env| { + let reader = unsafe { &mut *(handle as *mut datafusion_comet_shuffle::ShuffleStreamReader) }; + match reader.next_batch() { + Ok(Some(batch)) => { + prepare_output(&mut env, array_addrs, schema_addrs, batch, false) + } + Ok(None) => Ok(-1), + Err(e) => Err(CometError::Internal(e)), + } + }) +} + +#[no_mangle] +/// Close and drop a shuffle stream reader. +/// # Safety +/// This function is inherently unsafe since it deals with raw pointers from JNI. +pub unsafe extern "system" fn Java_org_apache_comet_Native_closeShuffleStream( + e: JNIEnv, + _class: JClass, + handle: jlong, +) { + try_unwrap_or_throw(&e, |_env| { + if handle != 0 { + unsafe { drop(Box::from_raw(handle as *mut datafusion_comet_shuffle::ShuffleStreamReader)) }; + } + Ok(()) + }); +} +``` + +Remove the old `decodeShuffleBlock` function. + +- [ ] **Step 4: Verify Rust compilation** + +Run: `cargo check --manifest-path native/Cargo.toml` +Expected: Compiles (possibly with warnings about unused code). JNI type details may need adjustment. + +- [ ] **Step 5: Commit** + +```bash +git add native/shuffle/src/ipc.rs native/shuffle/src/lib.rs native/core/src/execution/jni_api.rs +git commit -m "feat: JniInputStream and stream reader handle for shuffle read path" +``` + +--- + +## Task 7: Update JVM Read Side (Native.scala + NativeBatchDecoderIterator) + +**Files:** +- Modify: `spark/src/main/scala/org/apache/comet/Native.scala` +- Modify: `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala` + +- [ ] **Step 1: Update `Native.scala` JNI declarations** + +Replace the `decodeShuffleBlock` declaration (lines 175-180) with: + +```scala + /** + * Open a shuffle stream reader over a JVM InputStream. + * @param inputStream the InputStream to read from + * @return an opaque handle to the native stream reader + */ + @native def openShuffleStream(inputStream: java.io.InputStream): Long + + /** + * Read the next batch from a shuffle stream. + * @param handle the native stream reader handle + * @param arrayAddrs Arrow Array addresses for FFI export + * @param schemaAddrs Arrow Schema addresses for FFI export + * @return the number of rows in the batch, or -1 if stream is exhausted + */ + @native def nextShuffleStreamBatch( + handle: Long, + arrayAddrs: Array[Long], + schemaAddrs: Array[Long]): Long + + /** + * Close and release a native shuffle stream reader. + * @param handle the native stream reader handle + */ + @native def closeShuffleStream(handle: Long): Unit +``` + +Also remove the `import java.nio.ByteBuffer` if it's no longer needed (check other usages first). + +- [ ] **Step 2: Rewrite `NativeBatchDecoderIterator.scala`** + +Replace the entire file with: + +```scala +/* + * (keep license header) + */ + +package org.apache.spark.sql.comet.execution.shuffle + +import java.io.InputStream + +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.vectorized.ColumnarBatch + +import org.apache.comet.Native +import org.apache.comet.vector.NativeUtil + +/** + * Iterator that reads shuffle blocks from a JVM InputStream using native Arrow IPC + * stream decoding. The native side pulls bytes from the InputStream via JNI callbacks. + */ +case class NativeBatchDecoderIterator( + in: InputStream, + decodeTime: SQLMetric, + nativeLib: Native, + nativeUtil: NativeUtil, + tracingEnabled: Boolean) + extends Iterator[ColumnarBatch] { + + private var isClosed = false + private var currentBatch: ColumnarBatch = null + + // Open the native stream reader + private val handle: Long = if (in != null) { + nativeLib.openShuffleStream(in) + } else { + 0L + } + + private var batch = fetchNext() + + def hasNext(): Boolean = { + if (handle == 0L || isClosed) { + return false + } + if (batch.isDefined) { + return true + } + + if (currentBatch != null) { + currentBatch.close() + currentBatch = null + } + + batch = fetchNext() + if (batch.isEmpty) { + close() + return false + } + true + } + + def next(): ColumnarBatch = { + if (!hasNext) { + throw new NoSuchElementException + } + val nextBatch = batch.get + currentBatch = nextBatch + batch = None + currentBatch + } + + private def fetchNext(): Option[ColumnarBatch] = { + if (handle == 0L || isClosed) { + return None + } + + val startTime = System.nanoTime() + + // Query the field count from the native reader (it parsed the schema on open). + val batch = nativeUtil.getNextBatch( + nativeLib.shuffleStreamNumFields(handle).toInt, + (arrayAddrs, schemaAddrs) => { + nativeLib.nextShuffleStreamBatch(handle, arrayAddrs, schemaAddrs) + }) + + decodeTime.add(System.nanoTime() - startTime) + batch + } + + def close(): Unit = { + synchronized { + if (!isClosed) { + if (currentBatch != null) { + currentBatch.close() + currentBatch = null + } + if (handle != 0L) { + nativeLib.closeShuffleStream(handle) + } + if (in != null) { + in.close() + } + isClosed = true + } + } + } +} +``` + +**Note:** This requires adding a `shuffleStreamNumFields` JNI method. Add to `Native.scala`: + +```scala + /** + * Get the number of fields (columns) in a shuffle stream's schema. + * @param handle the native stream reader handle + * @return the number of fields + */ + @native def shuffleStreamNumFields(handle: Long): Long +``` + +And add the corresponding Rust JNI function in `jni_api.rs`: + +```rust +#[no_mangle] +/// Get the number of fields in the shuffle stream's schema. +/// # Safety +/// This function is inherently unsafe since it deals with raw pointers from JNI. +pub unsafe extern "system" fn Java_org_apache_comet_Native_shuffleStreamNumFields( + e: JNIEnv, + _class: JClass, + handle: jlong, +) -> jlong { + try_unwrap_or_throw(&e, |_env| { + let reader = unsafe { &*(handle as *const datafusion_comet_shuffle::ShuffleStreamReader) }; + Ok(reader.num_fields() as jlong) + }) +} +``` + +- [ ] **Step 3: Remove the companion object with thread-local buffer** + +The old `NativeBatchDecoderIterator` companion object (lines 185-198) with `threadLocalDataBuf` and `resetDataBuf` is no longer needed. Remove it entirely. The new version has no companion object. + +- [ ] **Step 4: Commit** + +```bash +git add spark/src/main/scala/org/apache/comet/Native.scala \ + spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala \ + native/core/src/execution/jni_api.rs +git commit -m "feat: JVM shuffle reader uses native IPC stream decoding via JNI" +``` + +--- + +## Task 8: Build and Run End-to-End Tests + +Verify the full stack works together. + +**Files:** No new files — testing existing test suites. + +- [ ] **Step 1: Build native code** + +Run: `make core` +Expected: Native Rust code compiles successfully. + +- [ ] **Step 2: Run Rust shuffle tests** + +Run: `cd native && cargo test -p datafusion-comet-shuffle` +Expected: All tests pass. Tests using `read_ipc_compressed` have been updated in previous tasks. + +- [ ] **Step 3: Run clippy** + +Run: `cd native && cargo clippy --all-targets --workspace -- -D warnings` +Expected: No warnings or errors. + +- [ ] **Step 4: Build JVM** + +Run: `make` +Expected: Full build succeeds (native + JVM). + +- [ ] **Step 5: Run JVM shuffle tests** + +Run: `./mvnw test -Dsuites="org.apache.comet.exec.CometShuffleSuite"` +Expected: All shuffle tests pass with the new format. + +- [ ] **Step 6: Run broader test suite to check for regressions** + +Run: `./mvnw test -DwildcardSuites="CometShuffle"` +Expected: All shuffle-related tests pass. + +- [ ] **Step 7: Commit any test fixes** + +If any tests needed adjustments (e.g., tests that checked the old custom header format), commit them: + +```bash +git add -A +git commit -m "test: update shuffle tests for IPC stream-per-partition format" +``` + +--- + +## Task 9: Clean Up Removed Code + +Remove any dead code left over from the migration. + +**Files:** +- Potentially: `native/shuffle/src/writers/shuffle_block_writer.rs` (if not already reduced to just `CompressionCodec`) +- Any remaining references to `read_ipc_compressed`, `decodeShuffleBlock`, Snappy + +- [ ] **Step 1: Search for dead code** + +Run: +```bash +grep -rn "read_ipc_compressed\|decodeShuffleBlock\|ShuffleBlockWriter\|snap::\|Snappy\|SNAP\|CompressionCodec::Snappy" native/shuffle/src/ spark/src/ common/src/ +``` + +- [ ] **Step 2: Remove any remaining references** + +Remove or update any files still referencing the old format. + +- [ ] **Step 3: Rename `shuffle_block_writer.rs` to `codec.rs`** + +Since the file now only contains the `CompressionCodec` enum, rename it: +- Rename `native/shuffle/src/writers/shuffle_block_writer.rs` to `native/shuffle/src/writers/codec.rs` +- Update `native/shuffle/src/writers/mod.rs`: change `mod shuffle_block_writer;` to `mod codec;` and update the use statement. + +- [ ] **Step 4: Run full test suite** + +Run: `cd native && cargo test -p datafusion-comet-shuffle` +Expected: All tests pass. + +- [ ] **Step 5: Format code** + +Run: `make format` +Expected: All code formatted. + +- [ ] **Step 6: Commit** + +```bash +git add -A +git commit -m "chore: clean up dead code from shuffle format migration" +``` From bd8b6ecc19be6e35e850c0a7bbcd62f99b8d0207 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Apr 2026 09:23:21 -0600 Subject: [PATCH 36/46] feat: enable Arrow IPC compression feature for shuffle format --- native/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/native/Cargo.toml b/native/Cargo.toml index c626743be1..3fb087e443 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -34,7 +34,7 @@ edition = "2021" rust-version = "1.88" [workspace.dependencies] -arrow = { version = "57.3.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow = { version = "57.3.0", features = ["prettyprint", "ffi", "chrono-tz", "ipc_compression"] } async-trait = { version = "0.1" } bytes = { version = "1.11.1" } parquet = { version = "57.3.0", default-features = false, features = ["experimental"] } From 16eae59cefa52bee306afa8d3c712215b7c3f36b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Apr 2026 09:37:12 -0600 Subject: [PATCH 37/46] feat: replace custom shuffle block format with Arrow IPC streams Replace the custom shuffle block format (per-batch IPC streams with custom length-prefix headers and external compression wrappers) with standard Arrow IPC streams using built-in body compression. Key changes: - Replace ShuffleBlockWriter with CompressionCodec::ipc_write_options() that creates IpcWriteOptions with LZ4_FRAME or ZSTD body compression - Rewrite BufBatchWriter to use a persistent StreamWriter that writes the schema once, then appends N record batch messages - Rewrite PartitionWriter (spill) to create StreamWriter over spill files - Rewrite PartitionOutputStream (immediate mode) to use persistent StreamWriter> with lazy creation and drain/finish lifecycle - Simplify SinglePartitionShufflePartitioner by removing manual batch coalescing (handled by BufBatchWriter's BatchCoalescer) - Update sort-based shuffle in spark_unsafe/row.rs to use StreamWriter - Remove snappy from shuffle compression options (keep Snappy variant in CompressionCodec enum for Parquet writer compatibility) - Update all tests to use Arrow StreamReader for roundtrip verification - Update shuffle_bench binary and criterion benchmarks The old ipc.rs read path is preserved for Task 6. The core crate will have expected compile errors in shuffle_scan.rs tests and jni_api.rs due to removed ShuffleBlockWriter export. --- .../scala/org/apache/comet/CometConf.scala | 6 +- native/Cargo.lock | 1 + native/shuffle/Cargo.toml | 1 + native/shuffle/benches/shuffle_writer.rs | 26 ++- native/shuffle/src/bin/shuffle_bench.rs | 21 +- native/shuffle/src/lib.rs | 2 +- .../src/partitioners/immediate_mode.rs | 179 ++++++++---------- .../src/partitioners/multi_partition.rs | 38 ++-- .../src/partitioners/single_partition.rs | 115 +++-------- native/shuffle/src/shuffle_writer.rs | 106 +++++------ native/shuffle/src/spark_unsafe/row.rs | 22 ++- .../shuffle/src/writers/buf_batch_writer.rs | 125 ++++-------- native/shuffle/src/writers/mod.rs | 2 +- .../src/writers/shuffle_block_writer.rs | 132 ++----------- native/shuffle/src/writers/spill.rs | 73 ++++--- 15 files changed, 311 insertions(+), 538 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index dce5a29f7e..2a45dae563 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -434,11 +434,11 @@ object CometConf extends ShimCometConf { conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.compression.codec") .category(CATEGORY_SHUFFLE) .doc( - "The codec of Comet native shuffle used to compress shuffle data. lz4, zstd, and " + - "snappy are supported. Compression can be disabled by setting " + + "The codec of Comet native shuffle used to compress shuffle data. " + + "Supported codecs: lz4, zstd. Compression can be disabled by setting " + "spark.shuffle.compress=false.") .stringConf - .checkValues(Set("zstd", "lz4", "snappy")) + .checkValues(Set("zstd", "lz4")) .createWithDefault("lz4") val COMET_EXEC_SHUFFLE_COMPRESSION_ZSTD_LEVEL: ConfigEntry[Int] = diff --git a/native/Cargo.lock b/native/Cargo.lock index 2111391f32..88e6bebf7c 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -333,6 +333,7 @@ dependencies = [ "arrow-select", "flatbuffers", "lz4_flex 0.12.1", + "zstd", ] [[package]] diff --git a/native/shuffle/Cargo.toml b/native/shuffle/Cargo.toml index 505879e319..6e2bd4d7bb 100644 --- a/native/shuffle/Cargo.toml +++ b/native/shuffle/Cargo.toml @@ -47,6 +47,7 @@ lz4_flex = { version = "0.13.0", default-features = false, features = ["frame"] # parquet is only used by the shuffle_bench binary (shuffle-bench feature) parquet = { workspace = true, optional = true } simd-adler32 = "0.3.9" +# snap, lz4_flex, zstd are still needed by ipc.rs (old read path) until Task 6 snap = "1.1" tokio = { version = "1", features = ["rt-multi-thread"] } zstd = "0.13.3" diff --git a/native/shuffle/benches/shuffle_writer.rs b/native/shuffle/benches/shuffle_writer.rs index 873e872adf..2b02c01ece 100644 --- a/native/shuffle/benches/shuffle_writer.rs +++ b/native/shuffle/benches/shuffle_writer.rs @@ -18,22 +18,19 @@ use arrow::array::builder::{Date32Builder, Decimal128Builder, Int32Builder}; use arrow::array::{builder::StringBuilder, Array, Int32Array, RecordBatch}; use arrow::datatypes::{DataType, Field, Schema}; +use arrow::ipc::writer::{IpcWriteOptions, StreamWriter}; use arrow::row::{RowConverter, SortField}; use criterion::{criterion_group, criterion_main, Criterion}; use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::source::DataSourceExec; use datafusion::physical_expr::expressions::{col, Column}; use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; -use datafusion::physical_plan::metrics::Time; use datafusion::{ physical_plan::{common::collect, ExecutionPlan}, prelude::SessionContext, }; -use datafusion_comet_shuffle::{ - CometPartitioning, CompressionCodec, ShuffleBlockWriter, ShuffleWriterExec, -}; +use datafusion_comet_shuffle::{CometPartitioning, CompressionCodec, ShuffleWriterExec}; use itertools::Itertools; -use std::io::Cursor; use std::sync::Arc; use tokio::runtime::Runtime; @@ -43,20 +40,22 @@ fn criterion_benchmark(c: &mut Criterion) { for compression_codec in &[ CompressionCodec::None, CompressionCodec::Lz4Frame, - CompressionCodec::Snappy, CompressionCodec::Zstd(1), CompressionCodec::Zstd(6), ] { let name = format!("shuffle_writer: write encoded (compression={compression_codec:?})"); group.bench_function(name, |b| { - let mut buffer = vec![]; - let ipc_time = Time::default(); - let w = - ShuffleBlockWriter::try_new(&batch.schema(), compression_codec.clone()).unwrap(); + let write_options = compression_codec.ipc_write_options().unwrap(); b.iter(|| { - buffer.clear(); - let mut cursor = Cursor::new(&mut buffer); - w.write_batch(&batch, &mut cursor, &ipc_time).unwrap(); + let mut buffer = Vec::new(); + let mut writer = StreamWriter::try_new_with_options( + &mut buffer, + &batch.schema(), + write_options.clone(), + ) + .unwrap(); + writer.write(&batch).unwrap(); + writer.finish().unwrap(); }); }); } @@ -64,7 +63,6 @@ fn criterion_benchmark(c: &mut Criterion) { for compression_codec in [ CompressionCodec::None, CompressionCodec::Lz4Frame, - CompressionCodec::Snappy, CompressionCodec::Zstd(1), CompressionCodec::Zstd(6), ] { diff --git a/native/shuffle/src/bin/shuffle_bench.rs b/native/shuffle/src/bin/shuffle_bench.rs index d1f85b409f..00dd0ee4cd 100644 --- a/native/shuffle/src/bin/shuffle_bench.rs +++ b/native/shuffle/src/bin/shuffle_bench.rs @@ -46,9 +46,8 @@ use datafusion::physical_plan::common::collect; use datafusion::physical_plan::metrics::{MetricValue, MetricsSet}; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::{ParquetReadOptions, SessionContext}; -use datafusion_comet_shuffle::{ - read_ipc_compressed, CometPartitioning, CompressionCodec, ShuffleWriterExec, -}; +use arrow::ipc::reader::StreamReader; +use datafusion_comet_shuffle::{CometPartitioning, CompressionCodec, ShuffleWriterExec}; use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; use std::fs; use std::path::{Path, PathBuf}; @@ -81,7 +80,7 @@ struct Args { #[arg(long, default_value = "0")] hash_columns: String, - /// Compression codec: none, lz4, zstd, snappy + /// Compression codec: none, lz4, zstd #[arg(long, default_value = "zstd")] codec: String, @@ -658,15 +657,14 @@ fn run_shuffle_read(data_file: &str, index_file: &str, num_partitions: usize) -> continue; } - let mut offset = start_offset; - while offset < end_offset { - let ipc_length = - u64::from_le_bytes(data_bytes[offset..offset + 8].try_into().unwrap()) as usize; - let block_data = &data_bytes[offset + 16..offset + 8 + ipc_length]; - let batch = read_ipc_compressed(block_data).expect("Failed to decode shuffle block"); + // Each partition's data contains one or more complete IPC streams + let partition_data = &data_bytes[start_offset..end_offset]; + let mut reader = + StreamReader::try_new(partition_data, None).expect("Failed to open IPC stream"); + while let Some(batch_result) = reader.next() { + let batch = batch_result.expect("Failed to decode record batch"); total_rows += batch.num_rows(); total_batches += 1; - offset += 8 + ipc_length; } } @@ -712,7 +710,6 @@ fn parse_codec(codec: &str, zstd_level: i32) -> CompressionCodec { "none" => CompressionCodec::None, "lz4" => CompressionCodec::Lz4Frame, "zstd" => CompressionCodec::Zstd(zstd_level), - "snappy" => CompressionCodec::Snappy, other => { eprintln!("Unknown codec: {other}. Using zstd."); CompressionCodec::Zstd(zstd_level) diff --git a/native/shuffle/src/lib.rs b/native/shuffle/src/lib.rs index dd3b900272..5f05b761ae 100644 --- a/native/shuffle/src/lib.rs +++ b/native/shuffle/src/lib.rs @@ -27,4 +27,4 @@ pub(crate) mod writers; pub use comet_partitioning::CometPartitioning; pub use ipc::read_ipc_compressed; pub use shuffle_writer::ShuffleWriterExec; -pub use writers::{CompressionCodec, ShuffleBlockWriter}; +pub use writers::CompressionCodec; diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index ae039d16ef..8bad02fe27 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -33,7 +33,7 @@ use arrow::datatypes::{ TimestampMillisecondType, TimestampNanosecondType, TimestampSecondType, UInt16Type, UInt32Type, UInt64Type, UInt8Type, }; -use arrow::ipc::writer::StreamWriter; +use arrow::ipc::writer::{IpcWriteOptions, StreamWriter}; use datafusion::common::{DataFusionError, Result}; use datafusion::execution::memory_pool::{MemoryConsumer, MemoryLimit, MemoryReservation}; use datafusion::execution::runtime_env::RuntimeEnv; @@ -281,90 +281,77 @@ impl PartitionBuffer { } } +/// Per-partition output stream that writes batches into a persistent Arrow IPC +/// `StreamWriter>`. The schema is written once when the writer is lazily +/// created. Arrow IPC body compression handles LZ4/ZSTD internally via `IpcWriteOptions`. pub(crate) struct PartitionOutputStream { schema: SchemaRef, - codec: CompressionCodec, - buffer: Vec, + write_options: IpcWriteOptions, + /// Lazily created IPC stream writer over an in-memory buffer + writer: Option>>, + /// Accumulated spill data (bytes from finished IPC streams that were drained) + spilled_bytes: Vec, } impl PartitionOutputStream { - pub(crate) fn try_new(schema: SchemaRef, codec: CompressionCodec) -> Result { + pub(crate) fn try_new(schema: SchemaRef, write_options: IpcWriteOptions) -> Result { Ok(Self { schema, - codec, - buffer: Vec::new(), + write_options, + writer: None, + spilled_bytes: Vec::new(), }) } - fn write_ipc_block(&mut self, batch: &RecordBatch) -> Result { - let start_pos = self.buffer.len(); - - self.buffer.extend_from_slice(&0u64.to_le_bytes()); - let field_count = self.schema.fields().len(); - self.buffer - .extend_from_slice(&(field_count as u64).to_le_bytes()); - let codec_tag: &[u8; 4] = match &self.codec { - CompressionCodec::Snappy => b"SNAP", - CompressionCodec::Lz4Frame => b"LZ4_", - CompressionCodec::Zstd(_) => b"ZSTD", - CompressionCodec::None => b"NONE", - }; - self.buffer.extend_from_slice(codec_tag); - - match &self.codec { - CompressionCodec::None => { - let mut w = StreamWriter::try_new(&mut self.buffer, &batch.schema())?; - w.write(batch)?; - w.finish()?; - w.into_inner()?; - } - CompressionCodec::Lz4Frame => { - let mut wtr = lz4_flex::frame::FrameEncoder::new(&mut self.buffer); - let mut w = StreamWriter::try_new(&mut wtr, &batch.schema())?; - w.write(batch)?; - w.finish()?; - wtr.finish().map_err(|e| { - DataFusionError::Execution(format!("lz4 compression error: {e}")) - })?; - } - CompressionCodec::Zstd(level) => { - let enc = zstd::Encoder::new(&mut self.buffer, *level)?; - let mut w = StreamWriter::try_new(enc, &batch.schema())?; - w.write(batch)?; - w.finish()?; - w.into_inner()?.finish()?; + /// Ensure the writer exists (lazy creation), write the batch, and return bytes written. + fn write_batch(&mut self, batch: &RecordBatch) -> Result { + let before = self.current_buffer_len(); + let writer = match &mut self.writer { + Some(w) => w, + None => { + let w = StreamWriter::try_new_with_options( + Vec::new(), + &self.schema, + self.write_options.clone(), + )?; + self.writer = Some(w); + self.writer.as_mut().unwrap() } - CompressionCodec::Snappy => { - let mut wtr = snap::write::FrameEncoder::new(&mut self.buffer); - let mut w = StreamWriter::try_new(&mut wtr, &batch.schema())?; - w.write(batch)?; - w.finish()?; - wtr.into_inner().map_err(|e| { - DataFusionError::Execution(format!("snappy compression error: {e}")) - })?; - } - } + }; + writer.write(batch)?; + let after = self.current_buffer_len(); + Ok(after.saturating_sub(before)) + } - let end_pos = self.buffer.len(); - let ipc_length = (end_pos - start_pos - 8) as u64; - if ipc_length > i32::MAX as u64 { - return Err(DataFusionError::Execution(format!( - "Shuffle block size {ipc_length} exceeds maximum size of {}", - i32::MAX - ))); + /// Finish the current IPC stream (if any), return all accumulated bytes + /// (spilled + current stream), and reset the writer to None. + fn drain_buffer(&mut self) -> Result> { + if let Some(mut writer) = self.writer.take() { + writer.finish()?; + let buf = writer.into_inner()?; + self.spilled_bytes.extend_from_slice(&buf); } - self.buffer[start_pos..start_pos + 8].copy_from_slice(&ipc_length.to_le_bytes()); - - Ok(end_pos - start_pos) + Ok(std::mem::take(&mut self.spilled_bytes)) } - fn drain_buffer(&mut self) -> Vec { - std::mem::take(&mut self.buffer) + /// Finish the current IPC stream and move its bytes into spilled_bytes, + /// resetting the writer to None so a new stream can be started later. + fn finish_current_stream(&mut self) -> Result<()> { + if let Some(mut writer) = self.writer.take() { + writer.finish()?; + let buf = writer.into_inner()?; + self.spilled_bytes.extend_from_slice(&buf); + } + Ok(()) } - #[cfg(test)] - fn finish(self) -> Result> { - Ok(self.buffer) + fn current_buffer_len(&self) -> usize { + let writer_len = self + .writer + .as_ref() + .map(|w| w.get_ref().len()) + .unwrap_or(0); + self.spilled_bytes.len() + writer_len } } @@ -410,13 +397,16 @@ impl ImmediateModePartitioner { codec: CompressionCodec, ) -> Result { let num_output_partitions = partitioning.partition_count(); + let write_options = codec.ipc_write_options()?; let partition_buffers = (0..num_output_partitions) .map(|_| PartitionBuffer::new(&schema, batch_size)) .collect(); let streams = (0..num_output_partitions) - .map(|_| PartitionOutputStream::try_new(Arc::clone(&schema), codec.clone())) + .map(|_| { + PartitionOutputStream::try_new(Arc::clone(&schema), write_options.clone()) + }) .collect::>>()?; let spill_files: Vec> = @@ -596,13 +586,13 @@ impl ImmediateModePartitioner { Ok((flushed_builder_bytes, ipc_bytes)) } - /// Flush a partition's builders to an IPC block in its output stream. + /// Flush a partition's builders to the IPC stream in its output stream. /// Returns `(flushed_batch_memory, ipc_bytes_written)`. fn flush_partition(&mut self, pid: usize) -> Result<(usize, usize)> { let output_batch = self.partition_buffers[pid].flush()?; let batch_mem = output_batch.get_array_memory_size(); let mut encode_timer = self.metrics.encode_time.timer(); - let ipc_bytes = self.streams[pid].write_ipc_block(&output_batch)?; + let ipc_bytes = self.streams[pid].write_batch(&output_batch)?; encode_timer.stop(); Ok((batch_mem, ipc_bytes)) } @@ -618,9 +608,12 @@ impl ImmediateModePartitioner { } } - // Drain IPC buffers to disk + // Finish current IPC streams and drain buffers to disk for pid in 0..self.streams.len() { - let buf = self.streams[pid].drain_buffer(); + // Finish the current IPC stream so it can be read back later + self.streams[pid].finish_current_stream()?; + + let buf = self.streams[pid].drain_buffer()?; if buf.is_empty() { continue; } @@ -738,7 +731,7 @@ impl ShufflePartitioner for ImmediateModePartitioner { self.flush_partition(pid)?; } - let buf = self.streams[pid].drain_buffer(); + let buf = self.streams[pid].drain_buffer()?; if !buf.is_empty() { let mut write_timer = self.metrics.write_time.timer(); output_data.write_all(&buf)?; @@ -775,9 +768,9 @@ impl ShufflePartitioner for ImmediateModePartitioner { #[cfg(test)] mod tests { use super::*; - use crate::read_ipc_compressed; use arrow::array::{Int32Array, StringArray}; use arrow::datatypes::{DataType, Field, Schema}; + use arrow::ipc::reader::StreamReader; use datafusion::execution::memory_pool::GreedyMemoryPool; use datafusion::execution::runtime_env::RuntimeEnvBuilder; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; @@ -861,20 +854,18 @@ mod tests { CompressionCodec::None, CompressionCodec::Lz4Frame, CompressionCodec::Zstd(1), - CompressionCodec::Snappy, ] { - let mut stream = PartitionOutputStream::try_new(Arc::clone(&schema), codec).unwrap(); - stream.write_ipc_block(&batch).unwrap(); + let write_options = codec.ipc_write_options().unwrap(); + let mut stream = + PartitionOutputStream::try_new(Arc::clone(&schema), write_options).unwrap(); + stream.write_batch(&batch).unwrap(); - let buf = stream.finish().unwrap(); + let buf = stream.drain_buffer().unwrap(); assert!(!buf.is_empty()); - let ipc_length = u64::from_le_bytes(buf[0..8].try_into().unwrap()) as usize; - assert!(ipc_length > 0); - - let block_end = 8 + ipc_length; - let ipc_data = &buf[16..block_end]; - let batch2 = read_ipc_compressed(ipc_data).unwrap(); + // Read back using standard Arrow StreamReader + let mut reader = StreamReader::try_new(&buf[..], None).unwrap(); + let batch2 = reader.next().unwrap().unwrap(); assert_eq!(batch2.num_rows(), 5); } } @@ -1019,7 +1010,7 @@ mod tests { } #[tokio::test] - async fn test_block_format_compatible_with_read_ipc_compressed() { + async fn test_ipc_stream_format_roundtrip() { let batch = make_test_batch(&[1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); let schema = batch.schema(); let dir = tempfile::tempdir().unwrap(); @@ -1061,14 +1052,12 @@ mod tests { if start == end { continue; } - let mut pos = start; - while pos < end { - let payload_len = - u64::from_le_bytes(data[pos..pos + 8].try_into().unwrap()) as usize; - assert!(payload_len > 0); - let block_end = pos + 8 + payload_len; - let ipc_data = &data[pos + 16..block_end]; - let decoded = read_ipc_compressed(ipc_data).unwrap(); + // Each partition's data is one or more complete IPC streams. + // Use StreamReader to decode them. + let partition_data = &data[start..end]; + let mut reader = StreamReader::try_new(partition_data, None).unwrap(); + while let Some(batch_result) = reader.next() { + let decoded = batch_result.unwrap(); assert_eq!(decoded.num_columns(), 1); assert!(decoded.num_rows() > 0); let col = decoded @@ -1080,9 +1069,7 @@ mod tests { assert!((1..=10).contains(&col.value(i))); } total_rows += decoded.num_rows(); - pos = block_end; } - assert_eq!(pos, end); } assert_eq!(total_rows, 10); } diff --git a/native/shuffle/src/partitioners/multi_partition.rs b/native/shuffle/src/partitioners/multi_partition.rs index 7de9314f54..a5077cf60f 100644 --- a/native/shuffle/src/partitioners/multi_partition.rs +++ b/native/shuffle/src/partitioners/multi_partition.rs @@ -21,9 +21,10 @@ use crate::partitioners::partitioned_batch_iterator::{ }; use crate::partitioners::ShufflePartitioner; use crate::writers::{BufBatchWriter, PartitionWriter}; -use crate::{comet_partitioning, CometPartitioning, CompressionCodec, ShuffleBlockWriter}; +use crate::{comet_partitioning, CometPartitioning, CompressionCodec}; use arrow::array::{ArrayRef, RecordBatch}; use arrow::datatypes::SchemaRef; +use arrow::ipc::writer::IpcWriteOptions; use datafusion::common::utils::proxy::VecAllocExt; use datafusion::common::DataFusionError; use datafusion::execution::memory_pool::{MemoryConsumer, MemoryReservation}; @@ -111,7 +112,10 @@ pub(crate) struct MultiPartitionShuffleRepartitioner { buffered_batches: Vec, partition_indices: Vec>, partition_writers: Vec, - shuffle_block_writer: ShuffleBlockWriter, + /// Schema of the input data + schema: SchemaRef, + /// IPC write options (includes compression settings) + write_options: IpcWriteOptions, /// Partitioning scheme to use partitioning: CometPartitioning, runtime: Arc, @@ -123,8 +127,6 @@ pub(crate) struct MultiPartitionShuffleRepartitioner { /// Reservation for repartitioning reservation: MemoryReservation, tracing_enabled: bool, - /// Size of the write buffer in bytes - write_buffer_size: usize, } impl MultiPartitionShuffleRepartitioner { @@ -140,7 +142,7 @@ impl MultiPartitionShuffleRepartitioner { batch_size: usize, codec: CompressionCodec, tracing_enabled: bool, - write_buffer_size: usize, + _write_buffer_size: usize, ) -> datafusion::common::Result { let num_output_partitions = partitioning.partition_count(); assert_ne!( @@ -165,10 +167,10 @@ impl MultiPartitionShuffleRepartitioner { partition_starts: vec![0; num_output_partitions + 1], }; - let shuffle_block_writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone())?; + let write_options = codec.ipc_write_options()?; let partition_writers = (0..num_output_partitions) - .map(|_| PartitionWriter::try_new(shuffle_block_writer.clone())) + .map(|_| PartitionWriter::try_new(Arc::clone(&schema), write_options.clone())) .collect::>>()?; let reservation = MemoryConsumer::new(format!("ShuffleRepartitioner[{partition}]")) @@ -181,7 +183,8 @@ impl MultiPartitionShuffleRepartitioner { buffered_batches: vec![], partition_indices: vec![vec![]; num_output_partitions], partition_writers, - shuffle_block_writer, + schema: Arc::clone(&schema), + write_options, partitioning, runtime, metrics, @@ -189,7 +192,6 @@ impl MultiPartitionShuffleRepartitioner { batch_size, reservation, tracing_enabled, - write_buffer_size, }) } @@ -436,19 +438,19 @@ impl MultiPartitionShuffleRepartitioner { fn shuffle_write_partition( partition_iter: &mut PartitionedBatchIterator, - shuffle_block_writer: &mut ShuffleBlockWriter, + schema: &SchemaRef, + write_options: &IpcWriteOptions, output_data: &mut BufWriter, encode_time: &Time, write_time: &Time, - write_buffer_size: usize, batch_size: usize, ) -> datafusion::common::Result<()> { - let mut buf_batch_writer = BufBatchWriter::new( - shuffle_block_writer, + let mut buf_batch_writer = BufBatchWriter::try_new( output_data, - write_buffer_size, + Arc::clone(schema), + write_options.clone(), batch_size, - ); + )?; for batch in partition_iter { let batch = batch?; buf_batch_writer.write(&batch, encode_time, write_time)?; @@ -511,8 +513,6 @@ impl MultiPartitionShuffleRepartitioner { &mut iter, &self.runtime, &self.metrics, - self.write_buffer_size, - self.batch_size, )?; } @@ -594,11 +594,11 @@ impl ShufflePartitioner for MultiPartitionShuffleRepartitioner { let mut partition_iter = partitioned_batches.produce(i); Self::shuffle_write_partition( &mut partition_iter, - &mut self.shuffle_block_writer, + &self.schema, + &self.write_options, &mut output_data, &self.metrics.encode_time, &self.metrics.write_time, - self.write_buffer_size, self.batch_size, )?; } diff --git a/native/shuffle/src/partitioners/single_partition.rs b/native/shuffle/src/partitioners/single_partition.rs index 5801ef613b..1d74a7939b 100644 --- a/native/shuffle/src/partitioners/single_partition.rs +++ b/native/shuffle/src/partitioners/single_partition.rs @@ -18,7 +18,7 @@ use crate::metrics::ShufflePartitionerMetrics; use crate::partitioners::ShufflePartitioner; use crate::writers::BufBatchWriter; -use crate::{CompressionCodec, ShuffleBlockWriter}; +use crate::CompressionCodec; use arrow::array::RecordBatch; use arrow::datatypes::SchemaRef; use datafusion::common::DataFusionError; @@ -26,19 +26,15 @@ use std::fs::{File, OpenOptions}; use std::io::{BufWriter, Write}; use tokio::time::Instant; -/// A partitioner that writes all shuffle data to a single file and a single index file +/// A partitioner that writes all shuffle data to a single file and a single index file. +/// Uses a persistent Arrow IPC StreamWriter via BufBatchWriter, so the schema is written +/// once and batches are appended with built-in body compression. pub(crate) struct SinglePartitionShufflePartitioner { - // output_data_file: File, - output_data_writer: BufBatchWriter, + output_data_writer: BufBatchWriter, + output_data_path: String, output_index_path: String, - /// Batches that are smaller than the batch size and to be concatenated - buffered_batches: Vec, - /// Number of rows in the concatenating batches - num_buffered_rows: usize, /// Metrics for the repartitioner metrics: ShufflePartitionerMetrics, - /// The configured batch size - batch_size: usize, } impl SinglePartitionShufflePartitioner { @@ -49,63 +45,26 @@ impl SinglePartitionShufflePartitioner { metrics: ShufflePartitionerMetrics, batch_size: usize, codec: CompressionCodec, - write_buffer_size: usize, + _write_buffer_size: usize, ) -> datafusion::common::Result { - let shuffle_block_writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone())?; + let write_options = codec.ipc_write_options()?; let output_data_file = OpenOptions::new() .write(true) .create(true) .truncate(true) - .open(output_data_path)?; + .open(&output_data_path)?; - let output_data_writer = BufBatchWriter::new( - shuffle_block_writer, - output_data_file, - write_buffer_size, - batch_size, - ); + let output_data_writer = + BufBatchWriter::try_new(output_data_file, schema, write_options, batch_size)?; Ok(Self { output_data_writer, + output_data_path, output_index_path, - buffered_batches: vec![], - num_buffered_rows: 0, metrics, - batch_size, }) } - - /// Add a batch to the buffer of the partitioner, these buffered batches will be concatenated - /// and written to the output data file when the number of rows in the buffer reaches the batch size. - fn add_buffered_batch(&mut self, batch: RecordBatch) { - self.num_buffered_rows += batch.num_rows(); - self.buffered_batches.push(batch); - } - - /// Consumes buffered batches and return a concatenated batch if successful - fn concat_buffered_batches(&mut self) -> datafusion::common::Result> { - if self.buffered_batches.is_empty() { - Ok(None) - } else if self.buffered_batches.len() == 1 { - let batch = self.buffered_batches.remove(0); - self.num_buffered_rows = 0; - Ok(Some(batch)) - } else { - let schema = &self.buffered_batches[0].schema(); - match arrow::compute::concat_batches(schema, self.buffered_batches.iter()) { - Ok(concatenated) => { - self.buffered_batches.clear(); - self.num_buffered_rows = 0; - Ok(Some(concatenated)) - } - Err(e) => Err(DataFusionError::ArrowError( - Box::from(e), - Some(DataFusionError::get_back_trace()), - )), - } - } - } } #[async_trait::async_trait] @@ -118,32 +77,11 @@ impl ShufflePartitioner for SinglePartitionShufflePartitioner { self.metrics.data_size.add(batch.get_array_memory_size()); self.metrics.baseline.record_output(num_rows); - if num_rows >= self.batch_size || num_rows + self.num_buffered_rows > self.batch_size { - let concatenated_batch = self.concat_buffered_batches()?; - - // Write the concatenated buffered batch - if let Some(batch) = concatenated_batch { - self.output_data_writer.write( - &batch, - &self.metrics.encode_time, - &self.metrics.write_time, - )?; - } - - if num_rows >= self.batch_size { - // Write the new batch - self.output_data_writer.write( - &batch, - &self.metrics.encode_time, - &self.metrics.write_time, - )?; - } else { - // Add the new batch to the buffer - self.add_buffered_batch(batch); - } - } else { - self.add_buffered_batch(batch); - } + self.output_data_writer.write( + &batch, + &self.metrics.encode_time, + &self.metrics.write_time, + )?; } self.metrics.input_batches.add(1); @@ -156,29 +94,24 @@ impl ShufflePartitioner for SinglePartitionShufflePartitioner { fn shuffle_write(&mut self) -> datafusion::common::Result<()> { let start_time = Instant::now(); - let concatenated_batch = self.concat_buffered_batches()?; - // Write the concatenated buffered batch - if let Some(batch) = concatenated_batch { - self.output_data_writer.write( - &batch, - &self.metrics.encode_time, - &self.metrics.write_time, - )?; - } self.output_data_writer .flush(&self.metrics.encode_time, &self.metrics.write_time)?; + // Get data file length via filesystem metadata + let data_file_length = std::fs::metadata(&self.output_data_path) + .map(|m| m.len()) + .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {e:?}")))?; + // Write index file. It should only contain 2 entries: 0 and the total number of bytes written let index_file = OpenOptions::new() .write(true) .create(true) .truncate(true) - .open(self.output_index_path.clone()) + .open(&self.output_index_path) .map_err(|e| DataFusionError::Execution(format!("shuffle write error: {e:?}")))?; let mut index_buf_writer = BufWriter::new(index_file); - let data_file_length = self.output_data_writer.writer_stream_position()?; - for offset in [0, data_file_length] { + for offset in [0u64, data_file_length] { index_buf_writer.write_all(&(offset as i64).to_le_bytes()[..])?; } index_buf_writer.flush()?; diff --git a/native/shuffle/src/shuffle_writer.rs b/native/shuffle/src/shuffle_writer.rs index 6c8598cb88..8afc709f05 100644 --- a/native/shuffle/src/shuffle_writer.rs +++ b/native/shuffle/src/shuffle_writer.rs @@ -284,9 +284,9 @@ async fn external_shuffle( #[cfg(test)] mod test { use super::*; - use crate::{read_ipc_compressed, ShuffleBlockWriter}; use arrow::array::{Array, StringArray, StringBuilder}; use arrow::datatypes::{DataType, Field, Schema}; + use arrow::ipc::reader::StreamReader; use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, SortField}; use datafusion::datasource::memory::MemorySourceConfig; @@ -299,30 +299,41 @@ mod test { use datafusion::physical_plan::metrics::Time; use datafusion::prelude::SessionContext; use itertools::Itertools; - use std::io::Cursor; use tokio::runtime::Runtime; #[test] #[cfg_attr(miri, ignore)] // miri can't call foreign function `ZSTD_createCCtx` fn roundtrip_ipc() { + use crate::writers::BufBatchWriter; + let batch = create_batch(8192); for codec in &[ CompressionCodec::None, CompressionCodec::Zstd(1), - CompressionCodec::Snappy, CompressionCodec::Lz4Frame, ] { - let mut output = vec![]; - let mut cursor = Cursor::new(&mut output); - let writer = - ShuffleBlockWriter::try_new(batch.schema().as_ref(), codec.clone()).unwrap(); - let length = writer - .write_batch(&batch, &mut cursor, &Time::default()) + let write_options = codec.ipc_write_options().unwrap(); + let mut output = Vec::new(); + let encode_time = Time::default(); + let write_time = Time::default(); + + { + let mut writer = BufBatchWriter::try_new( + &mut output, + batch.schema(), + write_options, + 8192, + ) .unwrap(); - assert_eq!(length, output.len()); + writer.write(&batch, &encode_time, &write_time).unwrap(); + writer.flush(&encode_time, &write_time).unwrap(); + } + + assert!(!output.is_empty()); - let ipc_without_length_prefix = &output[16..]; - let batch2 = read_ipc_compressed(ipc_without_length_prefix).unwrap(); + // Read back using standard Arrow StreamReader + let mut reader = StreamReader::try_new(&output[..], None).unwrap(); + let batch2 = reader.next().unwrap().unwrap(); assert_eq!(batch, batch2); } } @@ -607,15 +618,17 @@ mod test { let _ = fs::remove_file("/tmp/rr_index_1.out"); } - /// Test that batch coalescing in BufBatchWriter reduces output size by - /// writing fewer, larger IPC blocks instead of many small ones. + /// Test that batch coalescing in BufBatchWriter produces correct output. + /// With the new persistent StreamWriter format, schema is written once per stream + /// regardless of coalescing, but coalescing still reduces the number of record batch + /// messages in the stream. #[test] #[cfg_attr(miri, ignore)] - fn test_batch_coalescing_reduces_size() { + fn test_batch_coalescing_correct_output() { use crate::writers::BufBatchWriter; use arrow::array::Int32Array; - // Create a wide schema to amplify per-block schema overhead + // Create a wide schema to amplify per-batch message overhead let fields: Vec = (0..20) .map(|i| Field::new(format!("col_{i}"), DataType::Int32, false)) .collect(); @@ -637,19 +650,20 @@ mod test { .collect(); let codec = CompressionCodec::Lz4Frame; + let write_options = codec.ipc_write_options().unwrap(); let encode_time = Time::default(); let write_time = Time::default(); // Write with coalescing (batch_size=8192) let mut coalesced_output = Vec::new(); { - let mut writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone()).unwrap(); - let mut buf_writer = BufBatchWriter::new( - &mut writer, - Cursor::new(&mut coalesced_output), - 1024 * 1024, + let mut buf_writer = BufBatchWriter::try_new( + &mut coalesced_output, + Arc::clone(&schema), + write_options.clone(), 8192, - ); + ) + .unwrap(); for batch in &small_batches { buf_writer.write(batch, &encode_time, &write_time).unwrap(); } @@ -659,30 +673,22 @@ mod test { // Write without coalescing (batch_size=1) let mut uncoalesced_output = Vec::new(); { - let mut writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone()).unwrap(); - let mut buf_writer = BufBatchWriter::new( - &mut writer, - Cursor::new(&mut uncoalesced_output), - 1024 * 1024, + let mut buf_writer = BufBatchWriter::try_new( + &mut uncoalesced_output, + Arc::clone(&schema), + write_options, 1, - ); + ) + .unwrap(); for batch in &small_batches { buf_writer.write(batch, &encode_time, &write_time).unwrap(); } buf_writer.flush(&encode_time, &write_time).unwrap(); } - // Coalesced output should be smaller due to fewer IPC schema blocks - assert!( - coalesced_output.len() < uncoalesced_output.len(), - "Coalesced output ({} bytes) should be smaller than uncoalesced ({} bytes)", - coalesced_output.len(), - uncoalesced_output.len() - ); - - // Verify both roundtrip correctly by reading all IPC blocks - let coalesced_rows = read_all_ipc_blocks(&coalesced_output); - let uncoalesced_rows = read_all_ipc_blocks(&uncoalesced_output); + // Verify both roundtrip correctly by reading all batches via StreamReader + let coalesced_rows = read_all_ipc_stream_rows(&coalesced_output); + let uncoalesced_rows = read_all_ipc_stream_rows(&uncoalesced_output); assert_eq!( coalesced_rows, 5000, "Coalesced should contain all 5000 rows" @@ -693,24 +699,12 @@ mod test { ); } - /// Read all IPC blocks from a byte buffer written by BufBatchWriter/ShuffleBlockWriter, - /// returning the total number of rows. - fn read_all_ipc_blocks(data: &[u8]) -> usize { - let mut offset = 0; + /// Read all record batches from an Arrow IPC stream, returning total row count. + fn read_all_ipc_stream_rows(data: &[u8]) -> usize { + let mut reader = StreamReader::try_new(data, None).unwrap(); let mut total_rows = 0; - while offset < data.len() { - // First 8 bytes are the IPC length (little-endian u64) - let ipc_length = - u64::from_le_bytes(data[offset..offset + 8].try_into().unwrap()) as usize; - // Skip the 8-byte length prefix; the next 8 bytes are field_count + codec header - let block_start = offset + 8; - let block_end = block_start + ipc_length; - // read_ipc_compressed expects data starting after the 16-byte header - // (i.e., after length + field_count), at the codec tag - let ipc_data = &data[block_start + 8..block_end]; - let batch = read_ipc_compressed(ipc_data).unwrap(); - total_rows += batch.num_rows(); - offset = block_end; + while let Some(batch) = reader.next() { + total_rows += batch.unwrap().num_rows(); } total_rows } diff --git a/native/shuffle/src/spark_unsafe/row.rs b/native/shuffle/src/spark_unsafe/row.rs index 3c98677199..3aa7b66081 100644 --- a/native/shuffle/src/spark_unsafe/row.rs +++ b/native/shuffle/src/spark_unsafe/row.rs @@ -23,7 +23,7 @@ use crate::spark_unsafe::{ map::{append_map_elements, get_map_key_value_fields}, }; use crate::writers::Checksum; -use crate::writers::ShuffleBlockWriter; +use crate::CompressionCodec; use arrow::array::{ builder::{ ArrayBuilder, BinaryBuilder, BinaryDictionaryBuilder, BooleanBuilder, Date32Builder, @@ -37,7 +37,6 @@ use arrow::array::{ use arrow::compute::cast; use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; use arrow::error::ArrowError; -use datafusion::physical_plan::metrics::Time; use datafusion_comet_jni_bridge::errors::CometError; use jni::sys::{jint, jlong}; use std::{ @@ -197,7 +196,6 @@ macro_rules! get_field_builder { } // Expose the macro for other modules. -use crate::CompressionCodec; pub(crate) use downcast_builder_ref; /// Appends field of row to the given struct builder. `dt` is the data type of the field. @@ -1369,14 +1367,22 @@ pub fn process_sorted_row_partition( let batch = make_batch(array_refs?, n)?; frozen.clear(); - let mut cursor = Cursor::new(&mut frozen); - // we do not collect metrics in Native_writeSortedFileNative - let ipc_time = Time::default(); - let block_writer = ShuffleBlockWriter::try_new(batch.schema().as_ref(), codec.clone())?; - written += block_writer.write_batch(&batch, &mut cursor, &ipc_time)?; + // Write the batch as a complete IPC stream into `frozen` + let write_options = codec.ipc_write_options()?; + let mut stream_writer = arrow::ipc::writer::StreamWriter::try_new_with_options( + &mut frozen, + &batch.schema(), + write_options, + )?; + stream_writer.write(&batch)?; + stream_writer.finish()?; + stream_writer.into_inner()?; + + written += frozen.len(); if let Some(checksum) = &mut current_checksum { + let mut cursor = Cursor::new(&mut frozen); checksum.update(&mut cursor)?; } diff --git a/native/shuffle/src/writers/buf_batch_writer.rs b/native/shuffle/src/writers/buf_batch_writer.rs index cfddb46539..0ffedaa504 100644 --- a/native/shuffle/src/writers/buf_batch_writer.rs +++ b/native/shuffle/src/writers/buf_batch_writer.rs @@ -15,128 +15,71 @@ // specific language governing permissions and limitations // under the License. -use super::ShuffleBlockWriter; use arrow::array::RecordBatch; use arrow::compute::kernels::coalesce::BatchCoalescer; +use arrow::datatypes::SchemaRef; +use arrow::ipc::writer::{IpcWriteOptions, StreamWriter}; use datafusion::physical_plan::metrics::Time; -use std::borrow::Borrow; -use std::io::{Cursor, Seek, SeekFrom, Write}; +use std::io::Write; -/// Write batches to writer while using a buffer to avoid frequent system calls. -/// The record batches were first written by ShuffleBlockWriter into an internal buffer. -/// Once the buffer exceeds the max size, the buffer will be flushed to the writer. -/// -/// Small batches are coalesced using Arrow's [`BatchCoalescer`] before serialization, -/// producing exactly `batch_size`-row output batches to reduce per-block IPC schema overhead. -/// The coalescer is lazily initialized on the first write. -pub(crate) struct BufBatchWriter, W: Write> { - shuffle_block_writer: S, - writer: W, - buffer: Vec, - buffer_max_size: usize, +/// Writes batches to a persistent Arrow IPC `StreamWriter`. The schema is written once +/// when the writer is created. Small batches are coalesced via [`BatchCoalescer`] before +/// serialization, producing `batch_size`-row output batches. +pub(crate) struct BufBatchWriter { + writer: StreamWriter, /// Coalesces small batches into target_batch_size before serialization. - /// Lazily initialized on first write to capture the schema. - coalescer: Option, - /// Target batch size for coalescing - batch_size: usize, + coalescer: BatchCoalescer, } -impl, W: Write> BufBatchWriter { - pub(crate) fn new( - shuffle_block_writer: S, - writer: W, - buffer_max_size: usize, +impl BufBatchWriter { + pub(crate) fn try_new( + target: W, + schema: SchemaRef, + write_options: IpcWriteOptions, batch_size: usize, - ) -> Self { - Self { - shuffle_block_writer, - writer, - buffer: vec![], - buffer_max_size, - coalescer: None, - batch_size, - } + ) -> datafusion::common::Result { + let writer = StreamWriter::try_new_with_options(target, &schema, write_options)?; + let coalescer = BatchCoalescer::new(schema, batch_size); + Ok(Self { writer, coalescer }) } pub(crate) fn write( &mut self, batch: &RecordBatch, encode_time: &Time, - write_time: &Time, - ) -> datafusion::common::Result { - let coalescer = self - .coalescer - .get_or_insert_with(|| BatchCoalescer::new(batch.schema(), self.batch_size)); - coalescer.push_batch(batch.clone())?; + _write_time: &Time, + ) -> datafusion::common::Result<()> { + self.coalescer.push_batch(batch.clone())?; - // Drain completed batches into a local vec so the coalescer borrow ends - // before we call write_batch_to_buffer (which borrows &mut self). let mut completed = Vec::new(); - while let Some(batch) = coalescer.next_completed_batch() { + while let Some(batch) = self.coalescer.next_completed_batch() { completed.push(batch); } - let mut bytes_written = 0; for batch in &completed { - bytes_written += self.write_batch_to_buffer(batch, encode_time, write_time)?; + let mut timer = encode_time.timer(); + self.writer.write(batch)?; + timer.stop(); } - Ok(bytes_written) - } - - /// Serialize a single batch into the byte buffer, flushing to the writer if needed. - fn write_batch_to_buffer( - &mut self, - batch: &RecordBatch, - encode_time: &Time, - write_time: &Time, - ) -> datafusion::common::Result { - let mut cursor = Cursor::new(&mut self.buffer); - cursor.seek(SeekFrom::End(0))?; - let bytes_written = - self.shuffle_block_writer - .borrow() - .write_batch(batch, &mut cursor, encode_time)?; - let pos = cursor.position(); - if pos >= self.buffer_max_size as u64 { - let mut write_timer = write_time.timer(); - self.writer.write_all(&self.buffer)?; - write_timer.stop(); - self.buffer.clear(); - } - Ok(bytes_written) + Ok(()) } pub(crate) fn flush( &mut self, encode_time: &Time, - write_time: &Time, + _write_time: &Time, ) -> datafusion::common::Result<()> { // Finish any remaining buffered rows in the coalescer - let mut remaining = Vec::new(); - if let Some(coalescer) = &mut self.coalescer { - coalescer.finish_buffered_batch()?; - while let Some(batch) = coalescer.next_completed_batch() { - remaining.push(batch); - } - } - for batch in &remaining { - self.write_batch_to_buffer(batch, encode_time, write_time)?; + self.coalescer.finish_buffered_batch()?; + while let Some(batch) = self.coalescer.next_completed_batch() { + let mut timer = encode_time.timer(); + self.writer.write(&batch)?; + timer.stop(); } - // Flush the byte buffer to the underlying writer - let mut write_timer = write_time.timer(); - if !self.buffer.is_empty() { - self.writer.write_all(&self.buffer)?; - } - self.writer.flush()?; - write_timer.stop(); - self.buffer.clear(); + // Finish the IPC stream (writes the end-of-stream marker) + self.writer.finish()?; Ok(()) } -} -impl, W: Write + Seek> BufBatchWriter { - pub(crate) fn writer_stream_position(&mut self) -> datafusion::common::Result { - self.writer.stream_position().map_err(Into::into) - } } diff --git a/native/shuffle/src/writers/mod.rs b/native/shuffle/src/writers/mod.rs index 75caf9f3a3..5c6fc29bee 100644 --- a/native/shuffle/src/writers/mod.rs +++ b/native/shuffle/src/writers/mod.rs @@ -22,5 +22,5 @@ mod spill; pub(crate) use buf_batch_writer::BufBatchWriter; pub(crate) use checksum::Checksum; -pub use shuffle_block_writer::{CompressionCodec, ShuffleBlockWriter}; +pub use shuffle_block_writer::CompressionCodec; pub(crate) use spill::PartitionWriter; diff --git a/native/shuffle/src/writers/shuffle_block_writer.rs b/native/shuffle/src/writers/shuffle_block_writer.rs index 5ed5330e3a..92e416880c 100644 --- a/native/shuffle/src/writers/shuffle_block_writer.rs +++ b/native/shuffle/src/writers/shuffle_block_writer.rs @@ -15,132 +15,30 @@ // specific language governing permissions and limitations // under the License. -use arrow::array::RecordBatch; -use arrow::datatypes::Schema; -use arrow::ipc::writer::StreamWriter; -use datafusion::common::DataFusionError; -use datafusion::error::Result; -use datafusion::physical_plan::metrics::Time; -use std::io::{Cursor, Seek, SeekFrom, Write}; +use arrow::ipc::writer::IpcWriteOptions; +use arrow::ipc::CompressionType; -/// Compression algorithm applied to shuffle IPC blocks. +/// Compression algorithm applied to shuffle IPC streams and Parquet output. #[derive(Debug, Clone)] pub enum CompressionCodec { None, Lz4Frame, Zstd(i32), + /// Snappy is only used for Parquet output, not for shuffle IPC. Snappy, } -/// Writes a record batch as a length-prefixed, compressed Arrow IPC block. -#[derive(Clone)] -pub struct ShuffleBlockWriter { - codec: CompressionCodec, - header_bytes: Vec, -} - -impl ShuffleBlockWriter { - pub fn try_new(schema: &Schema, codec: CompressionCodec) -> Result { - let header_bytes = Vec::with_capacity(20); - let mut cursor = Cursor::new(header_bytes); - - // leave space for compressed message length - cursor.seek_relative(8)?; - - // write number of columns because JVM side needs to know how many addresses to allocate - let field_count = schema.fields().len(); - cursor.write_all(&field_count.to_le_bytes())?; - - // write compression codec to header - let codec_header = match &codec { - CompressionCodec::Snappy => b"SNAP", - CompressionCodec::Lz4Frame => b"LZ4_", - CompressionCodec::Zstd(_) => b"ZSTD", - CompressionCodec::None => b"NONE", +impl CompressionCodec { + pub fn ipc_write_options(&self) -> datafusion::error::Result { + let compression = match self { + CompressionCodec::None | CompressionCodec::Snappy => None, + CompressionCodec::Lz4Frame => Some(CompressionType::LZ4_FRAME), + CompressionCodec::Zstd(_) => Some(CompressionType::ZSTD), }; - cursor.write_all(codec_header)?; - - let header_bytes = cursor.into_inner(); - - Ok(Self { - codec, - header_bytes, - }) - } - - /// Writes given record batch as Arrow IPC bytes into given writer. - /// Returns number of bytes written. - pub fn write_batch( - &self, - batch: &RecordBatch, - output: &mut W, - ipc_time: &Time, - ) -> Result { - if batch.num_rows() == 0 { - return Ok(0); - } - - let mut timer = ipc_time.timer(); - let start_pos = output.stream_position()?; - - // write header - output.write_all(&self.header_bytes)?; - - let output = match &self.codec { - CompressionCodec::None => { - let mut arrow_writer = StreamWriter::try_new(output, &batch.schema())?; - arrow_writer.write(batch)?; - arrow_writer.finish()?; - arrow_writer.into_inner()? - } - CompressionCodec::Lz4Frame => { - let mut wtr = lz4_flex::frame::FrameEncoder::new(output); - let mut arrow_writer = StreamWriter::try_new(&mut wtr, &batch.schema())?; - arrow_writer.write(batch)?; - arrow_writer.finish()?; - wtr.finish().map_err(|e| { - DataFusionError::Execution(format!("lz4 compression error: {e}")) - })? - } - - CompressionCodec::Zstd(level) => { - let encoder = zstd::Encoder::new(output, *level)?; - let mut arrow_writer = StreamWriter::try_new(encoder, &batch.schema())?; - arrow_writer.write(batch)?; - arrow_writer.finish()?; - let zstd_encoder = arrow_writer.into_inner()?; - zstd_encoder.finish()? - } - - CompressionCodec::Snappy => { - let mut wtr = snap::write::FrameEncoder::new(output); - let mut arrow_writer = StreamWriter::try_new(&mut wtr, &batch.schema())?; - arrow_writer.write(batch)?; - arrow_writer.finish()?; - wtr.into_inner().map_err(|e| { - DataFusionError::Execution(format!("snappy compression error: {e}")) - })? - } - }; - - // fill ipc length - let end_pos = output.stream_position()?; - let ipc_length = end_pos - start_pos - 8; - let max_size = i32::MAX as u64; - if ipc_length > max_size { - return Err(DataFusionError::Execution(format!( - "Shuffle block size {ipc_length} exceeds maximum size of {max_size}. \ - Try reducing batch size or increasing compression level" - ))); - } - - // fill ipc length - output.seek(SeekFrom::Start(start_pos))?; - output.write_all(&ipc_length.to_le_bytes())?; - output.seek(SeekFrom::Start(end_pos))?; - - timer.stop(); - - Ok((end_pos - start_pos) as usize) + let options = IpcWriteOptions::try_new(8, false, arrow::ipc::MetadataVersion::V5) + .map_err(|e| datafusion::common::DataFusionError::ArrowError(Box::from(e), None))?; + options + .try_with_compression(compression) + .map_err(|e| datafusion::common::DataFusionError::ArrowError(Box::from(e), None)) } } diff --git a/native/shuffle/src/writers/spill.rs b/native/shuffle/src/writers/spill.rs index c16caddbf9..9155798d23 100644 --- a/native/shuffle/src/writers/spill.rs +++ b/native/shuffle/src/writers/spill.rs @@ -15,10 +15,10 @@ // specific language governing permissions and limitations // under the License. -use super::ShuffleBlockWriter; use crate::metrics::ShufflePartitionerMetrics; use crate::partitioners::PartitionedBatchIterator; -use crate::writers::buf_batch_writer::BufBatchWriter; +use arrow::datatypes::SchemaRef; +use arrow::ipc::writer::{IpcWriteOptions, StreamWriter}; use datafusion::common::DataFusionError; use datafusion::execution::disk_manager::RefCountedTempFile; use datafusion::execution::runtime_env::RuntimeEnv; @@ -36,17 +36,21 @@ pub(crate) struct PartitionWriter { /// will append to this file and the contents will be copied to the shuffle file at /// the end of processing. spill_file: Option, - /// Writer that performs encoding and compression - shuffle_block_writer: ShuffleBlockWriter, + /// Schema used for creating IPC stream writers + schema: SchemaRef, + /// IPC write options (includes compression settings) + write_options: IpcWriteOptions, } impl PartitionWriter { pub(crate) fn try_new( - shuffle_block_writer: ShuffleBlockWriter, + schema: SchemaRef, + write_options: IpcWriteOptions, ) -> datafusion::common::Result { Ok(Self { spill_file: None, - shuffle_block_writer, + schema, + write_options, }) } @@ -80,34 +84,45 @@ impl PartitionWriter { iter: &mut PartitionedBatchIterator, runtime: &RuntimeEnv, metrics: &ShufflePartitionerMetrics, - write_buffer_size: usize, - batch_size: usize, ) -> datafusion::common::Result { if let Some(batch) = iter.next() { self.ensure_spill_file_created(runtime)?; - let total_bytes_written = { - let mut buf_batch_writer = BufBatchWriter::new( - &mut self.shuffle_block_writer, - &mut self.spill_file.as_mut().unwrap().file, - write_buffer_size, - batch_size, - ); - let mut bytes_written = - buf_batch_writer.write(&batch?, &metrics.encode_time, &metrics.write_time)?; - for batch in iter { - let batch = batch?; - bytes_written += buf_batch_writer.write( - &batch, - &metrics.encode_time, - &metrics.write_time, - )?; - } - buf_batch_writer.flush(&metrics.encode_time, &metrics.write_time)?; - bytes_written - }; + let file = &mut self.spill_file.as_mut().unwrap().file; + let start_pos = file.metadata().map(|m| m.len()).unwrap_or(0); - Ok(total_bytes_written) + let mut writer = StreamWriter::try_new_with_options( + file, + &self.schema, + self.write_options.clone(), + )?; + + let batch = batch?; + let mut encode_timer = metrics.encode_time.timer(); + writer.write(&batch)?; + encode_timer.stop(); + + for batch in iter { + let batch = batch?; + let mut encode_timer = metrics.encode_time.timer(); + writer.write(&batch)?; + encode_timer.stop(); + } + + let mut write_timer = metrics.write_time.timer(); + writer.finish()?; + write_timer.stop(); + + let end_pos = self + .spill_file + .as_ref() + .unwrap() + .file + .metadata() + .map(|m| m.len()) + .unwrap_or(0); + + Ok((end_pos - start_pos) as usize) } else { Ok(0) } From a2ab532adb516bec3c455c635bf0994315dfa14d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Apr 2026 09:44:17 -0600 Subject: [PATCH 38/46] feat: add JniInputStream and ShuffleStreamReader for shuffle read path Add JniInputStream (implements std::io::Read by pulling bytes from a JVM InputStream via JNI with 64KB read-ahead buffer) and ShuffleStreamReader (wraps Arrow StreamReader for lifecycle management). Replace decodeShuffleBlock JNI function with four new streaming functions: openShuffleStream, nextShuffleStreamBatch, shuffleStreamNumFields, and closeShuffleStream. The old read_ipc_compressed is retained for the legacy ShuffleScanExec code path. --- native/core/src/execution/jni_api.rs | 71 +++++++++--- native/shuffle/src/ipc.rs | 157 +++++++++++++++++++++++++++ native/shuffle/src/lib.rs | 2 +- 3 files changed, 215 insertions(+), 15 deletions(-) diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index e0a395ebbf..c704d4613d 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -61,7 +61,6 @@ use datafusion_spark::function::string::space::SparkSpace; use futures::poll; use futures::stream::StreamExt; use futures::FutureExt; -use jni::objects::JByteBuffer; use jni::sys::{jlongArray, JNI_FALSE}; use jni::{ errors::Result as JNIResult, @@ -83,7 +82,7 @@ use crate::execution::memory_pools::{ create_memory_pool, handle_task_shared_pool_release, parse_memory_pool_config, MemoryPoolConfig, }; use crate::execution::operators::{ScanExec, ShuffleScanExec}; -use crate::execution::shuffle::{read_ipc_compressed, CompressionCodec}; +use crate::execution::shuffle::{CompressionCodec, ShuffleStreamReader}; use crate::execution::spark_plan::SparkPlan; use crate::execution::tracing::{log_memory_usage, trace_begin, trace_end, with_trace}; @@ -876,29 +875,73 @@ pub extern "system" fn Java_org_apache_comet_Native_sortRowPartitionsNative( } #[no_mangle] -/// Used by Comet native shuffle reader +/// Open a shuffle stream reader over a JVM InputStream. +/// Returns an opaque handle (pointer) to a `ShuffleStreamReader`. /// # Safety /// This function is inherently unsafe since it deals with raw pointers passed from JNI. -pub unsafe extern "system" fn Java_org_apache_comet_Native_decodeShuffleBlock( +pub unsafe extern "system" fn Java_org_apache_comet_Native_openShuffleStream( e: JNIEnv, _class: JClass, - byte_buffer: JByteBuffer, - length: jint, + input_stream: JObject, +) -> jlong { + try_unwrap_or_throw(&e, |mut env| { + let reader = + ShuffleStreamReader::new(&mut env, &input_stream).map_err(CometError::Internal)?; + let handle = Box::into_raw(Box::new(reader)); + Ok(handle as jlong) + }) +} + +#[no_mangle] +/// Read the next batch from a shuffle stream, exporting via Arrow FFI. +/// Returns the row count, or -1 if the stream is exhausted. +/// # Safety +/// This function is inherently unsafe since it deals with raw pointers passed from JNI. +pub unsafe extern "system" fn Java_org_apache_comet_Native_nextShuffleStreamBatch( + e: JNIEnv, + _class: JClass, + handle: jlong, array_addrs: JLongArray, schema_addrs: JLongArray, - tracing_enabled: jboolean, ) -> jlong { try_unwrap_or_throw(&e, |mut env| { - with_trace("decodeShuffleBlock", tracing_enabled != JNI_FALSE, || { - let raw_pointer = env.get_direct_buffer_address(&byte_buffer)?; - let length = length as usize; - let slice: &[u8] = unsafe { std::slice::from_raw_parts(raw_pointer, length) }; - let batch = read_ipc_compressed(slice)?; - prepare_output(&mut env, array_addrs, schema_addrs, batch, false) - }) + let reader = unsafe { &mut *(handle as *mut ShuffleStreamReader) }; + match reader.next_batch().map_err(CometError::Internal)? { + Some(batch) => prepare_output(&mut env, array_addrs, schema_addrs, batch, false), + None => Ok(-1_i64), + } }) } +#[no_mangle] +/// Get the number of fields in the shuffle stream's schema. +/// # Safety +/// This function is inherently unsafe since it deals with raw pointers passed from JNI. +pub unsafe extern "system" fn Java_org_apache_comet_Native_shuffleStreamNumFields( + _e: JNIEnv, + _class: JClass, + handle: jlong, +) -> jlong { + let reader = unsafe { &*(handle as *mut ShuffleStreamReader) }; + reader.num_fields() as jlong +} + +#[no_mangle] +/// Close and drop a shuffle stream reader. +/// # Safety +/// This function is inherently unsafe since it deals with raw pointers passed from JNI. +pub unsafe extern "system" fn Java_org_apache_comet_Native_closeShuffleStream( + _e: JNIEnv, + _class: JClass, + handle: jlong, +) { + if handle != 0 { + unsafe { + let _ = Box::from_raw(handle as *mut ShuffleStreamReader); + } + } +} + #[no_mangle] /// # Safety /// This function is inherently unsafe since it deals with raw pointers passed from JNI. diff --git a/native/shuffle/src/ipc.rs b/native/shuffle/src/ipc.rs index 81ee41332a..e573dd4694 100644 --- a/native/shuffle/src/ipc.rs +++ b/native/shuffle/src/ipc.rs @@ -19,7 +19,164 @@ use arrow::array::RecordBatch; use arrow::ipc::reader::StreamReader; use datafusion::common::DataFusionError; use datafusion::error::Result; +use jni::objects::{GlobalRef, JObject, JValue}; +use jni::JavaVM; +use std::io::Read; +/// Size of the internal read-ahead buffer (64 KB). +const READ_AHEAD_BUF_SIZE: usize = 64 * 1024; + +/// A Rust `Read` implementation that pulls bytes from a JVM `java.io.InputStream` +/// via JNI callbacks, using an internal read-ahead buffer to minimize JNI crossings. +pub struct JniInputStream { + /// Handle to the JVM for attaching threads. + vm: JavaVM, + /// Global reference to the JVM InputStream object. + input_stream: GlobalRef, + /// Global reference to the JVM byte[] used for bulk reads. + jbuf: GlobalRef, + /// Internal Rust-side buffer holding bytes read from JVM. + buf: Vec, + /// Current read position within `buf`. + pos: usize, + /// Number of valid bytes in `buf`. + len: usize, +} + +impl JniInputStream { + /// Create a new `JniInputStream` wrapping a JVM InputStream. + pub fn new(env: &mut jni::JNIEnv, input_stream: &JObject) -> jni::errors::Result { + let vm = env.get_java_vm()?; + let input_stream = env.new_global_ref(input_stream)?; + let jbuf_local = env.new_byte_array(READ_AHEAD_BUF_SIZE as i32)?; + let jbuf = env.new_global_ref(&jbuf_local)?; + Ok(Self { + vm, + input_stream, + jbuf, + buf: vec![0u8; READ_AHEAD_BUF_SIZE], + pos: 0, + len: 0, + }) + } + + /// Refill the internal buffer by calling `InputStream.read(byte[], 0, len)` via JNI. + fn refill(&mut self) -> std::io::Result { + let mut env = self + .vm + .attach_current_thread_as_daemon() + .map_err(|e| std::io::Error::other(e.to_string()))?; + + // Get a local reference from the global ref for the byte array + let jbuf_local = env + .new_local_ref(self.jbuf.as_obj()) + .map_err(|e| std::io::Error::other(e.to_string()))?; + + let n = env + .call_method( + &self.input_stream, + "read", + "([BII)I", + &[ + JValue::Object(&jbuf_local), + JValue::Int(0), + JValue::Int(READ_AHEAD_BUF_SIZE as i32), + ], + ) + .map_err(|e| std::io::Error::other(e.to_string()))? + .i() + .map_err(|e| std::io::Error::other(e.to_string()))?; + + if n <= 0 { + // -1 means end of stream + self.pos = 0; + self.len = 0; + return Ok(0); + } + + let n = n as usize; + + // Copy bytes from JVM byte[] into our Rust buffer. + // jbyte is i8; we read into a temporary i8 slice then reinterpret as u8. + let mut i8_buf = vec![0i8; n]; + let jbuf_array = unsafe { jni::objects::JByteArray::from_raw(jbuf_local.as_raw()) }; + env.get_byte_array_region(&jbuf_array, 0, &mut i8_buf) + .map_err(|e| std::io::Error::other(e.to_string()))?; + + // Don't let the JByteArray drop free the local ref — it was created from + // a local ref that we don't own (it came from new_local_ref). + // Actually, JByteArray::from_raw takes ownership conceptually, but the local + // ref table manages it. We need to forget it so the underlying JObject local + // ref doesn't get deleted twice. The new_local_ref created it, and from_raw + // wrapped it. We should not drop jbuf_array since that would call + // DeleteLocalRef on the same raw jobject that jbuf_local already points to. + // However, JByteArray doesn't impl Drop with DeleteLocalRef — jni objects + // are plain wrappers. So this is fine. + + let src = unsafe { std::slice::from_raw_parts(i8_buf.as_ptr() as *const u8, n) }; + self.buf[..n].copy_from_slice(src); + self.pos = 0; + self.len = n; + + Ok(n) + } +} + +impl Read for JniInputStream { + fn read(&mut self, out: &mut [u8]) -> std::io::Result { + if self.pos >= self.len { + // Buffer is empty, refill + let filled = self.refill()?; + if filled == 0 { + return Ok(0); // EOF + } + } + + let available = self.len - self.pos; + let to_copy = available.min(out.len()); + out[..to_copy].copy_from_slice(&self.buf[self.pos..self.pos + to_copy]); + self.pos += to_copy; + Ok(to_copy) + } +} + +/// Manages a `StreamReader` for reading Arrow IPC streams +/// from a JVM InputStream. +pub struct ShuffleStreamReader { + reader: StreamReader, +} + +impl ShuffleStreamReader { + /// Create a new `ShuffleStreamReader` over a JVM InputStream. + pub fn new(env: &mut jni::JNIEnv, input_stream: &JObject) -> Result { + let jni_stream = + JniInputStream::new(env, input_stream).map_err(|e| format!("JNI error: {e}"))?; + let reader = unsafe { + StreamReader::try_new(jni_stream, None) + .map_err(|e| format!("Arrow IPC error: {e}"))? + .with_skip_validation(true) + }; + Ok(Self { reader }) + } + + /// Read the next batch from the stream. Returns `None` when exhausted. + pub fn next_batch(&mut self) -> Result, String> { + match self.reader.next() { + Some(Ok(batch)) => Ok(Some(batch)), + Some(Err(e)) => Err(format!("Arrow IPC read error: {e}")), + None => Ok(None), + } + } + + /// Return the number of fields in the stream's schema. + pub fn num_fields(&self) -> usize { + self.reader.schema().fields().len() + } +} + +/// Read a single RecordBatch from a compressed IPC block. +/// The first 4 bytes indicate the compression codec, followed by the IPC stream data. +/// This is the legacy read path used by `ShuffleScanExec`. pub fn read_ipc_compressed(bytes: &[u8]) -> Result { match &bytes[0..4] { b"SNAP" => { diff --git a/native/shuffle/src/lib.rs b/native/shuffle/src/lib.rs index 5f05b761ae..cf4297404e 100644 --- a/native/shuffle/src/lib.rs +++ b/native/shuffle/src/lib.rs @@ -25,6 +25,6 @@ pub mod spark_unsafe; pub(crate) mod writers; pub use comet_partitioning::CometPartitioning; -pub use ipc::read_ipc_compressed; +pub use ipc::{read_ipc_compressed, JniInputStream, ShuffleStreamReader}; pub use shuffle_writer::ShuffleWriterExec; pub use writers::CompressionCodec; From 1777b753441a3502b684a1ec0c25eccbb2cffa19 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Apr 2026 09:46:55 -0600 Subject: [PATCH 39/46] feat: update JVM read side to use streaming shuffle decode API Replace decodeShuffleBlock JNI declaration with four new streaming methods: openShuffleStream, nextShuffleStreamBatch, shuffleStreamNumFields, and closeShuffleStream. Rewrite NativeBatchDecoderIterator to use a native handle pattern instead of manual header parsing and ByteBuffer management. --- .../main/scala/org/apache/comet/Native.scala | 16 +-- .../shuffle/NativeBatchDecoderIterator.scala | 119 ++++-------------- 2 files changed, 32 insertions(+), 103 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/Native.scala b/spark/src/main/scala/org/apache/comet/Native.scala index f6800626d6..abb321b683 100644 --- a/spark/src/main/scala/org/apache/comet/Native.scala +++ b/spark/src/main/scala/org/apache/comet/Native.scala @@ -19,8 +19,6 @@ package org.apache.comet -import java.nio.ByteBuffer - import org.apache.spark.CometTaskMemoryManager import org.apache.spark.sql.comet.CometMetricNode @@ -172,12 +170,16 @@ class Native extends NativeBase { * @param size * the size of the array. */ - @native def decodeShuffleBlock( - shuffleBlock: ByteBuffer, - length: Int, + @native def openShuffleStream(inputStream: java.io.InputStream): Long + + @native def nextShuffleStreamBatch( + handle: Long, arrayAddrs: Array[Long], - schemaAddrs: Array[Long], - tracingEnabled: Boolean): Long + schemaAddrs: Array[Long]): Long + + @native def shuffleStreamNumFields(handle: Long): Long + + @native def closeShuffleStream(handle: Long): Unit /** * Log the beginning of an event. diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala index f96c8f16dd..22fc14df97 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala @@ -19,9 +19,7 @@ package org.apache.spark.sql.comet.execution.shuffle -import java.io.{EOFException, InputStream} -import java.nio.{ByteBuffer, ByteOrder} -import java.nio.channels.{Channels, ReadableByteChannel} +import java.io.InputStream import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.vectorized.ColumnarBatch @@ -43,27 +41,32 @@ case class NativeBatchDecoderIterator( extends Iterator[ColumnarBatch] { private var isClosed = false - private val longBuf = ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN) private var currentBatch: ColumnarBatch = null - private var batch = fetchNext() - import NativeBatchDecoderIterator._ + // Open the native stream reader + private val handle: Long = if (in != null) { + nativeLib.openShuffleStream(in) + } else { + 0L + } - private val channel: ReadableByteChannel = if (in != null) { - Channels.newChannel(in) + // Get field count from the native reader (it parsed the schema on open) + private val numFields: Int = if (handle != 0L) { + nativeLib.shuffleStreamNumFields(handle).toInt } else { - null + 0 } + private var batch = fetchNext() + def hasNext(): Boolean = { - if (channel == null || isClosed) { + if (handle == 0L || isClosed) { return false } if (batch.isDefined) { return true } - // Release the previous batch. if (currentBatch != null) { currentBatch.close() currentBatch = null @@ -81,89 +84,24 @@ case class NativeBatchDecoderIterator( if (!hasNext) { throw new NoSuchElementException } - val nextBatch = batch.get - currentBatch = nextBatch batch = None currentBatch } private def fetchNext(): Option[ColumnarBatch] = { - if (channel == null || isClosed) { + if (handle == 0L || isClosed) { return None } - // read compressed batch size from header - try { - longBuf.clear() - while (longBuf.hasRemaining && channel.read(longBuf) >= 0) {} - } catch { - case _: EOFException => - close() - return None - } - - // If we reach the end of the stream, we are done, or if we read partial length - // then the stream is corrupted. - if (longBuf.hasRemaining) { - if (longBuf.position() == 0) { - close() - return None - } - throw new EOFException("Data corrupt: unexpected EOF while reading compressed ipc lengths") - } - - // get compressed length (including headers) - longBuf.flip() - val compressedLength = longBuf.getLong - - // read field count from header - longBuf.clear() - while (longBuf.hasRemaining && channel.read(longBuf) >= 0) {} - if (longBuf.hasRemaining) { - throw new EOFException("Data corrupt: unexpected EOF while reading field count") - } - longBuf.flip() - val fieldCount = longBuf.getLong.toInt - - // read body - val bytesToRead = compressedLength - 8 - if (bytesToRead > Integer.MAX_VALUE) { - // very unlikely that shuffle block will reach 2GB - throw new IllegalStateException( - s"Native shuffle block size of $bytesToRead exceeds " + - s"maximum of ${Integer.MAX_VALUE}. Try reducing shuffle batch size.") - } - var dataBuf = threadLocalDataBuf.get() - if (dataBuf.capacity() < bytesToRead) { - // it is unlikely that we would overflow here since it would - // require a 1GB compressed shuffle block but we check anyway - val newCapacity = (bytesToRead * 2L).min(Integer.MAX_VALUE).toInt - dataBuf = ByteBuffer.allocateDirect(newCapacity) - threadLocalDataBuf.set(dataBuf) - } - dataBuf.clear() - dataBuf.limit(bytesToRead.toInt) - while (dataBuf.hasRemaining && channel.read(dataBuf) >= 0) {} - if (dataBuf.hasRemaining) { - throw new EOFException("Data corrupt: unexpected EOF while reading compressed batch") - } - - // make native call to decode batch val startTime = System.nanoTime() val batch = nativeUtil.getNextBatch( - fieldCount, + numFields, (arrayAddrs, schemaAddrs) => { - nativeLib.decodeShuffleBlock( - dataBuf, - bytesToRead.toInt, - arrayAddrs, - schemaAddrs, - tracingEnabled) + nativeLib.nextShuffleStreamBatch(handle, arrayAddrs, schemaAddrs) }) decodeTime.add(System.nanoTime() - startTime) - batch } @@ -174,25 +112,14 @@ case class NativeBatchDecoderIterator( currentBatch.close() currentBatch = null } - in.close() - resetDataBuf() + if (handle != 0L) { + nativeLib.closeShuffleStream(handle) + } + if (in != null) { + in.close() + } isClosed = true } } } } - -object NativeBatchDecoderIterator { - - private val INITIAL_BUFFER_SIZE = 128 * 1024 - - private val threadLocalDataBuf: ThreadLocal[ByteBuffer] = ThreadLocal.withInitial(() => { - ByteBuffer.allocateDirect(INITIAL_BUFFER_SIZE) - }) - - private def resetDataBuf(): Unit = { - if (threadLocalDataBuf.get().capacity() > INITIAL_BUFFER_SIZE) { - threadLocalDataBuf.set(ByteBuffer.allocateDirect(INITIAL_BUFFER_SIZE)) - } - } -} From 0f330061deb1cef3ede4208c7f1ac375d5ded68d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Apr 2026 09:52:20 -0600 Subject: [PATCH 40/46] fix: resolve clippy warnings and update shuffle_scan tests for new IPC format --- .../src/execution/operators/shuffle_scan.rs | 66 +++++++++---------- native/shuffle/benches/shuffle_writer.rs | 2 +- .../src/partitioners/immediate_mode.rs | 14 ++-- native/shuffle/src/shuffle_writer.rs | 14 ++-- 4 files changed, 41 insertions(+), 55 deletions(-) diff --git a/native/core/src/execution/operators/shuffle_scan.rs b/native/core/src/execution/operators/shuffle_scan.rs index a1ad52310c..f90e02eea6 100644 --- a/native/core/src/execution/operators/shuffle_scan.rs +++ b/native/core/src/execution/operators/shuffle_scan.rs @@ -351,16 +351,15 @@ impl RecordBatchStream for ShuffleScanStream { #[cfg(test)] mod tests { - use crate::execution::shuffle::{CompressionCodec, ShuffleBlockWriter}; + use crate::execution::shuffle::CompressionCodec; use arrow::array::{Int32Array, StringArray}; use arrow::datatypes::{DataType, Field, Schema}; + use arrow::ipc::reader::StreamReader; + use arrow::ipc::writer::StreamWriter; use arrow::record_batch::RecordBatch; - use datafusion::physical_plan::metrics::Time; use std::io::Cursor; use std::sync::Arc; - use crate::execution::shuffle::ipc::read_ipc_compressed; - #[test] #[cfg_attr(miri, ignore)] // Miri cannot call FFI functions (zstd) fn test_read_compressed_ipc_block() { @@ -377,18 +376,18 @@ mod tests { ) .unwrap(); - // Write as compressed IPC - let writer = - ShuffleBlockWriter::try_new(&batch.schema(), CompressionCodec::Zstd(1)).unwrap(); - let mut buf = Cursor::new(Vec::new()); - let ipc_time = Time::new(); - writer.write_batch(&batch, &mut buf, &ipc_time).unwrap(); - - // Read back (skip 16-byte header: 8 compressed_length + 8 field_count) - let bytes = buf.into_inner(); - let body = &bytes[16..]; - - let decoded = read_ipc_compressed(body).unwrap(); + // Write as Arrow IPC stream with compression + let write_options = CompressionCodec::Zstd(1).ipc_write_options().unwrap(); + let mut buf = Vec::new(); + let mut writer = + StreamWriter::try_new_with_options(&mut buf, &batch.schema(), write_options).unwrap(); + writer.write(&batch).unwrap(); + writer.finish().unwrap(); + + // Read back using standard StreamReader + let cursor = Cursor::new(&buf); + let mut reader = StreamReader::try_new(cursor, None).unwrap(); + let decoded = reader.next().unwrap().unwrap(); assert_eq!(decoded.num_rows(), 3); assert_eq!(decoded.num_columns(), 2); @@ -404,9 +403,6 @@ mod tests { } /// Tests that ShuffleScanExec correctly unpacks dictionary-encoded columns. - /// Native shuffle may dictionary-encode string/binary columns, but the schema - /// declares value types (e.g. Utf8). Without unpacking, RecordBatch creation - /// fails with a schema mismatch. #[test] #[cfg_attr(miri, ignore)] fn test_dictionary_encoded_shuffle_block_is_unpacked() { @@ -416,15 +412,12 @@ mod tests { use datafusion::physical_plan::ExecutionPlan; use futures::StreamExt; - // Build a batch with a dictionary-encoded string column (simulating what - // the native shuffle writer produces for string columns). let mut dict_builder = StringDictionaryBuilder::::new(); dict_builder.append_value("hello"); dict_builder.append_value("world"); - dict_builder.append_value("hello"); // repeated value, good for dictionary + dict_builder.append_value("hello"); let dict_array = dict_builder.finish(); - // The IPC schema includes the dictionary type let dict_schema = Arc::new(Schema::new(vec![ Field::new("id", DataType::Int32, false), Field::new( @@ -442,19 +435,22 @@ mod tests { ) .unwrap(); - // Write as compressed IPC (preserves dictionary encoding) - let writer = - ShuffleBlockWriter::try_new(&dict_batch.schema(), CompressionCodec::Zstd(1)).unwrap(); - let mut buf = Cursor::new(Vec::new()); - let ipc_time = Time::new(); - writer - .write_batch(&dict_batch, &mut buf, &ipc_time) - .unwrap(); - let bytes = buf.into_inner(); - let body = &bytes[16..]; + // Write as Arrow IPC stream with compression + let write_options = CompressionCodec::Zstd(1).ipc_write_options().unwrap(); + let mut buf = Vec::new(); + let mut writer = StreamWriter::try_new_with_options( + &mut buf, + &dict_batch.schema(), + write_options, + ) + .unwrap(); + writer.write(&dict_batch).unwrap(); + writer.finish().unwrap(); - // Confirm that read_ipc_compressed returns dictionary-encoded arrays - let decoded = read_ipc_compressed(body).unwrap(); + // Read back using standard StreamReader + let cursor = Cursor::new(&buf); + let mut reader = StreamReader::try_new(cursor, None).unwrap(); + let decoded = reader.next().unwrap().unwrap(); assert!( matches!(decoded.column(1).data_type(), DataType::Dictionary(_, _)), "Expected dictionary-encoded column from IPC, got {:?}", diff --git a/native/shuffle/benches/shuffle_writer.rs b/native/shuffle/benches/shuffle_writer.rs index 2b02c01ece..e71f83f387 100644 --- a/native/shuffle/benches/shuffle_writer.rs +++ b/native/shuffle/benches/shuffle_writer.rs @@ -18,7 +18,7 @@ use arrow::array::builder::{Date32Builder, Decimal128Builder, Int32Builder}; use arrow::array::{builder::StringBuilder, Array, Int32Array, RecordBatch}; use arrow::datatypes::{DataType, Field, Schema}; -use arrow::ipc::writer::{IpcWriteOptions, StreamWriter}; +use arrow::ipc::writer::StreamWriter; use arrow::row::{RowConverter, SortField}; use criterion::{criterion_group, criterion_main, Criterion}; use datafusion::datasource::memory::MemorySourceConfig; diff --git a/native/shuffle/src/partitioners/immediate_mode.rs b/native/shuffle/src/partitioners/immediate_mode.rs index 8bad02fe27..c9dc24b754 100644 --- a/native/shuffle/src/partitioners/immediate_mode.rs +++ b/native/shuffle/src/partitioners/immediate_mode.rs @@ -346,11 +346,7 @@ impl PartitionOutputStream { } fn current_buffer_len(&self) -> usize { - let writer_len = self - .writer - .as_ref() - .map(|w| w.get_ref().len()) - .unwrap_or(0); + let writer_len = self.writer.as_ref().map(|w| w.get_ref().len()).unwrap_or(0); self.spilled_bytes.len() + writer_len } } @@ -404,9 +400,7 @@ impl ImmediateModePartitioner { .collect(); let streams = (0..num_output_partitions) - .map(|_| { - PartitionOutputStream::try_new(Arc::clone(&schema), write_options.clone()) - }) + .map(|_| PartitionOutputStream::try_new(Arc::clone(&schema), write_options.clone())) .collect::>>()?; let spill_files: Vec> = @@ -1055,8 +1049,8 @@ mod tests { // Each partition's data is one or more complete IPC streams. // Use StreamReader to decode them. let partition_data = &data[start..end]; - let mut reader = StreamReader::try_new(partition_data, None).unwrap(); - while let Some(batch_result) = reader.next() { + let reader = StreamReader::try_new(partition_data, None).unwrap(); + for batch_result in reader { let decoded = batch_result.unwrap(); assert_eq!(decoded.num_columns(), 1); assert!(decoded.num_rows() > 0); diff --git a/native/shuffle/src/shuffle_writer.rs b/native/shuffle/src/shuffle_writer.rs index 8afc709f05..bbf3fd432c 100644 --- a/native/shuffle/src/shuffle_writer.rs +++ b/native/shuffle/src/shuffle_writer.rs @@ -318,13 +318,9 @@ mod test { let write_time = Time::default(); { - let mut writer = BufBatchWriter::try_new( - &mut output, - batch.schema(), - write_options, - 8192, - ) - .unwrap(); + let mut writer = + BufBatchWriter::try_new(&mut output, batch.schema(), write_options, 8192) + .unwrap(); writer.write(&batch, &encode_time, &write_time).unwrap(); writer.flush(&encode_time, &write_time).unwrap(); } @@ -701,9 +697,9 @@ mod test { /// Read all record batches from an Arrow IPC stream, returning total row count. fn read_all_ipc_stream_rows(data: &[u8]) -> usize { - let mut reader = StreamReader::try_new(data, None).unwrap(); + let reader = StreamReader::try_new(data, None).unwrap(); let mut total_rows = 0; - while let Some(batch) = reader.next() { + for batch in reader { total_rows += batch.unwrap().num_rows(); } total_rows From 4ac38672ad6dbd4335de1dcd20dce62cc336970c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Apr 2026 10:12:51 -0600 Subject: [PATCH 41/46] feat: update ShuffleScanExec to use ShuffleStreamReader for Arrow IPC streams Replace the old CometShuffleBlockIterator-based read path in ShuffleScanExec with ShuffleStreamReader, which reads standard Arrow IPC streams directly from JVM InputStreams via JniInputStream. This eliminates the custom per-batch block format (8-byte length + 8-byte field count + 4-byte codec + compressed IPC) and the per-batch JNI calls (hasNext/getBuffer) in favor of streaming reads. Changes: - CometShuffledRowRDD: return raw InputStream instead of CometShuffleBlockIterator - CometExecIterator: accept Map[Int, InputStream] instead of Map[Int, CometShuffleBlockIterator] - ShuffleScanExec (Rust): lazily create ShuffleStreamReader from InputStream GlobalRef, read batches via reader.next_batch() instead of JNI block-by-block dance - Add Send+Sync impls for SharedJniStream/StreamReadAdapter to satisfy ExecutionPlan bounds --- .../src/execution/operators/shuffle_scan.rs | 186 ++++++++++-------- native/shuffle/src/bin/shuffle_bench.rs | 2 +- native/shuffle/src/ipc.rs | 117 +++++++++-- .../src/partitioners/multi_partition.rs | 6 +- .../shuffle/src/writers/buf_batch_writer.rs | 1 - native/shuffle/src/writers/spill.rs | 7 +- .../org/apache/comet/CometExecIterator.scala | 12 +- .../apache/spark/sql/comet/CometExecRDD.scala | 8 +- .../shuffle/CometShuffledRowRDD.scala | 12 +- 9 files changed, 231 insertions(+), 120 deletions(-) diff --git a/native/core/src/execution/operators/shuffle_scan.rs b/native/core/src/execution/operators/shuffle_scan.rs index f90e02eea6..385319c11a 100644 --- a/native/core/src/execution/operators/shuffle_scan.rs +++ b/native/core/src/execution/operators/shuffle_scan.rs @@ -18,9 +18,11 @@ use crate::{ errors::CometError, execution::{ - operators::ExecutionError, planner::TEST_EXEC_CONTEXT_ID, shuffle::ipc::read_ipc_compressed, + operators::ExecutionError, + planner::TEST_EXEC_CONTEXT_ID, + shuffle::ShuffleStreamReader, }, - jvm_bridge::{jni_call, JVMClasses}, + jvm_bridge::JVMClasses, }; use arrow::array::ArrayRef; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; @@ -35,7 +37,7 @@ use datafusion::{ physical_plan::{ExecutionPlan, *}, }; use futures::Stream; -use jni::objects::{GlobalRef, JByteBuffer, JObject}; +use jni::objects::GlobalRef; use std::{ any::Any, pin::Pin, @@ -45,14 +47,13 @@ use std::{ use super::scan::InputBatch; -/// ShuffleScanExec reads compressed shuffle blocks from JVM via JNI and decodes them natively. -/// Unlike ScanExec which receives Arrow arrays via FFI, ShuffleScanExec receives raw compressed -/// bytes from CometShuffleBlockIterator and decodes them using read_ipc_compressed(). -#[derive(Debug, Clone)] +/// ShuffleScanExec reads Arrow IPC streams from JVM via JNI and decodes them natively. +/// Unlike ScanExec which receives Arrow arrays via FFI, ShuffleScanExec receives a raw +/// InputStream from JVM and reads Arrow IPC streams using ShuffleStreamReader. pub struct ShuffleScanExec { /// The ID of the execution context that owns this subquery. pub exec_context_id: i64, - /// The input source: a global reference to a JVM CometShuffleBlockIterator object. + /// The input source: a global reference to a JVM InputStream object. pub input_source: Option>, /// The data types of columns in the shuffle output. pub data_types: Vec, @@ -60,16 +61,48 @@ pub struct ShuffleScanExec { pub schema: SchemaRef, /// The current input batch, populated by get_next_batch() before poll_next(). pub batch: Arc>>, + /// Cached ShuffleStreamReader, created lazily on first get_next call. + stream_reader: Option, /// Cache of plan properties. cache: PlanProperties, /// Metrics collector. metrics: ExecutionPlanMetricsSet, /// Baseline metrics. baseline_metrics: BaselineMetrics, - /// Time spent decoding compressed shuffle blocks. + /// Time spent decoding shuffle batches. decode_time: Time, } +impl std::fmt::Debug for ShuffleScanExec { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("ShuffleScanExec") + .field("exec_context_id", &self.exec_context_id) + .field("data_types", &self.data_types) + .field("schema", &self.schema) + .field("stream_reader", &self.stream_reader.is_some()) + .finish() + } +} + +impl Clone for ShuffleScanExec { + fn clone(&self) -> Self { + Self { + exec_context_id: self.exec_context_id, + input_source: self.input_source.clone(), + data_types: self.data_types.clone(), + schema: Arc::clone(&self.schema), + batch: Arc::clone(&self.batch), + // stream_reader is not cloneable; cloned instances start without one + // and will lazily create their own if needed. + stream_reader: None, + cache: self.cache.clone(), + metrics: self.metrics.clone(), + baseline_metrics: self.baseline_metrics.clone(), + decode_time: self.decode_time.clone(), + } + } +} + impl ShuffleScanExec { pub fn new( exec_context_id: i64, @@ -94,6 +127,7 @@ impl ShuffleScanExec { input_source, data_types, batch: Arc::new(Mutex::new(None)), + stream_reader: None, cache, metrics: metrics_set, baseline_metrics, @@ -114,90 +148,86 @@ impl ShuffleScanExec { // Unit test mode - no JNI calls needed. 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 = Self::get_next( - self.exec_context_id, - self.input_source.as_ref().unwrap().as_obj(), - &self.data_types, - &self.decode_time, - )?; + // Check if a batch is already pending without holding the lock during get_next + let needs_batch = { + let current_batch = self.batch.try_lock().unwrap(); + current_batch.is_none() + }; + + if needs_batch { + let start = std::time::Instant::now(); + let next_batch = self.get_next()?; + self.baseline_metrics + .elapsed_compute() + .add_duration(start.elapsed()); + let mut current_batch = self.batch.try_lock().unwrap(); *current_batch = Some(next_batch); } - timer.stop(); - Ok(()) } - /// Invokes JNI calls to get the next compressed shuffle block and decode it. - fn get_next( - exec_context_id: i64, - iter: &JObject, - data_types: &[DataType], - decode_time: &Time, - ) -> Result { - if exec_context_id == TEST_EXEC_CONTEXT_ID { + /// Reads the next batch from the ShuffleStreamReader, creating it lazily on first call. + fn get_next(&mut self) -> Result { + if self.exec_context_id == TEST_EXEC_CONTEXT_ID { return Ok(InputBatch::EOF); } - if iter.is_null() { - return Err(CometError::from(ExecutionError::GeneralError(format!( - "Null shuffle block iterator object. Plan id: {exec_context_id}" - )))); + // Lazily create the ShuffleStreamReader on first call + if self.stream_reader.is_none() { + let input_source = self.input_source.as_ref().ok_or_else(|| { + CometError::from(ExecutionError::GeneralError(format!( + "Null shuffle input source. Plan id: {}", + self.exec_context_id + ))) + })?; + let mut env = JVMClasses::get_env()?; + let reader = + ShuffleStreamReader::new(&mut env, input_source.as_obj()).map_err(|e| { + CometError::from(ExecutionError::GeneralError(format!( + "Failed to create ShuffleStreamReader: {e}" + ))) + })?; + self.stream_reader = Some(reader); } - let mut env = JVMClasses::get_env()?; + let reader = self.stream_reader.as_mut().unwrap(); + + let mut decode_timer = self.decode_time.timer(); + let batch_opt = reader.next_batch().map_err(|e| { + CometError::from(ExecutionError::GeneralError(format!( + "Failed to read shuffle batch: {e}" + ))) + })?; + decode_timer.stop(); + + match batch_opt { + None => Ok(InputBatch::EOF), + Some(batch) => { + let num_rows = batch.num_rows(); + + // Extract column arrays, unpacking any dictionary-encoded columns. + // Native shuffle may dictionary-encode string/binary columns for efficiency, + // but downstream DataFusion operators expect the value types declared in the + // schema (e.g. Utf8, not Dictionary). + let columns: Vec = batch + .columns() + .iter() + .map(|col| unpack_dictionary(col)) + .collect(); - // has_next() reads the next block and returns its length, or -1 if EOF - let block_length: i32 = unsafe { - jni_call!(&mut env, - comet_shuffle_block_iterator(iter).has_next() -> i32)? - }; + debug_assert_eq!( + columns.len(), + self.data_types.len(), + "Shuffle block column count mismatch: got {} but expected {}", + columns.len(), + self.data_types.len() + ); - if block_length == -1 { - return Ok(InputBatch::EOF); + Ok(InputBatch::new(columns, Some(num_rows))) + } } - - // Get the DirectByteBuffer containing the compressed shuffle block - let buffer: JObject = unsafe { - jni_call!(&mut env, - comet_shuffle_block_iterator(iter).get_buffer() -> JObject)? - }; - - let byte_buffer = JByteBuffer::from(buffer); - let raw_pointer = env.get_direct_buffer_address(&byte_buffer)?; - let length = block_length as usize; - let slice: &[u8] = unsafe { std::slice::from_raw_parts(raw_pointer, length) }; - - // Decode the compressed IPC data - let mut timer = decode_time.timer(); - let batch = read_ipc_compressed(slice)?; - timer.stop(); - - let num_rows = batch.num_rows(); - - // Extract column arrays, unpacking any dictionary-encoded columns. - // Native shuffle may dictionary-encode string/binary columns for efficiency, - // but downstream DataFusion operators expect the value types declared in the - // schema (e.g. Utf8, not Dictionary). - let columns: Vec = batch - .columns() - .iter() - .map(|col| unpack_dictionary(col)) - .collect(); - - debug_assert_eq!( - columns.len(), - data_types.len(), - "Shuffle block column count mismatch: got {} but expected {}", - columns.len(), - data_types.len() - ); - - Ok(InputBatch::new(columns, Some(num_rows))) } } diff --git a/native/shuffle/src/bin/shuffle_bench.rs b/native/shuffle/src/bin/shuffle_bench.rs index 00dd0ee4cd..78d4072d22 100644 --- a/native/shuffle/src/bin/shuffle_bench.rs +++ b/native/shuffle/src/bin/shuffle_bench.rs @@ -37,6 +37,7 @@ //! ``` use arrow::datatypes::{DataType, SchemaRef}; +use arrow::ipc::reader::StreamReader; use clap::Parser; use datafusion::execution::config::SessionConfig; use datafusion::execution::runtime_env::RuntimeEnvBuilder; @@ -46,7 +47,6 @@ use datafusion::physical_plan::common::collect; use datafusion::physical_plan::metrics::{MetricValue, MetricsSet}; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::{ParquetReadOptions, SessionContext}; -use arrow::ipc::reader::StreamReader; use datafusion_comet_shuffle::{CometPartitioning, CompressionCodec, ShuffleWriterExec}; use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; use std::fs; diff --git a/native/shuffle/src/ipc.rs b/native/shuffle/src/ipc.rs index e573dd4694..192bcc9746 100644 --- a/native/shuffle/src/ipc.rs +++ b/native/shuffle/src/ipc.rs @@ -140,37 +140,128 @@ impl Read for JniInputStream { } } -/// Manages a `StreamReader` for reading Arrow IPC streams -/// from a JVM InputStream. +/// A wrapper around `JniInputStream` that allows `StreamReader` to borrow +/// it while still being able to create new `StreamReader` instances for +/// concatenated IPC streams. +/// +/// Uses a raw pointer to the `JniInputStream` stored in a `Box` so that +/// the `StreamReader` can take a `Read` impl without taking ownership. +struct SharedJniStream { + inner: *mut JniInputStream, +} + +impl SharedJniStream { + fn new(stream: JniInputStream) -> Self { + Self { + inner: Box::into_raw(Box::new(stream)), + } + } + + /// Create a Read adapter that delegates to the inner stream. + fn reader(&self) -> StreamReadAdapter { + StreamReadAdapter { inner: self.inner } + } +} + +impl Drop for SharedJniStream { + fn drop(&mut self) { + unsafe { drop(Box::from_raw(self.inner)) }; + } +} + +// SAFETY: SharedJniStream owns the JniInputStream exclusively via a raw pointer. +// It is only accessed from a single thread at a time (the JNI thread that calls +// get_next_batch). The raw pointer is used to allow multiple sequential StreamReader +// instances to borrow the same underlying stream. +unsafe impl Send for SharedJniStream {} +unsafe impl Sync for SharedJniStream {} + +// SAFETY: StreamReadAdapter borrows from the same raw pointer as SharedJniStream. +// Same single-threaded access guarantees apply. +unsafe impl Send for StreamReadAdapter {} +unsafe impl Sync for StreamReadAdapter {} + +/// A Read adapter that delegates to a raw pointer to JniInputStream. +/// Multiple StreamReader instances can be created from this adapter +/// (sequentially, not concurrently). +struct StreamReadAdapter { + inner: *mut JniInputStream, +} + +impl Read for StreamReadAdapter { + fn read(&mut self, buf: &mut [u8]) -> std::io::Result { + unsafe { (*self.inner).read(buf) } + } +} + +/// Manages reading potentially concatenated Arrow IPC streams from a JVM +/// InputStream. A single partition's data may contain multiple IPC streams +/// (e.g., from spills), so when one stream reaches EOS we attempt to open +/// the next one from the same underlying InputStream. pub struct ShuffleStreamReader { - reader: StreamReader, + /// Shared ownership of the JniInputStream. + jni_stream: SharedJniStream, + /// Current Arrow IPC stream reader. `None` when all streams are exhausted. + reader: Option>, + num_fields: usize, } impl ShuffleStreamReader { /// Create a new `ShuffleStreamReader` over a JVM InputStream. pub fn new(env: &mut jni::JNIEnv, input_stream: &JObject) -> Result { - let jni_stream = - JniInputStream::new(env, input_stream).map_err(|e| format!("JNI error: {e}"))?; + let jni_stream = SharedJniStream::new( + JniInputStream::new(env, input_stream).map_err(|e| format!("JNI error: {e}"))?, + ); let reader = unsafe { - StreamReader::try_new(jni_stream, None) + StreamReader::try_new(jni_stream.reader(), None) .map_err(|e| format!("Arrow IPC error: {e}"))? .with_skip_validation(true) }; - Ok(Self { reader }) + let num_fields = reader.schema().fields().len(); + Ok(Self { + jni_stream, + reader: Some(reader), + num_fields, + }) } - /// Read the next batch from the stream. Returns `None` when exhausted. + /// Read the next batch from the stream. Returns `None` when all + /// concatenated IPC streams are exhausted. pub fn next_batch(&mut self) -> Result, String> { - match self.reader.next() { - Some(Ok(batch)) => Ok(Some(batch)), - Some(Err(e)) => Err(format!("Arrow IPC read error: {e}")), - None => Ok(None), + loop { + let reader = match &mut self.reader { + Some(r) => r, + None => return Ok(None), + }; + + match reader.next() { + Some(Ok(batch)) => return Ok(Some(batch)), + Some(Err(e)) => return Err(format!("Arrow IPC read error: {e}")), + None => { + // Current IPC stream exhausted. Drop the old reader and try + // to open the next concatenated stream. + self.reader = None; + + match StreamReader::try_new(self.jni_stream.reader(), None) { + Ok(new_reader) => { + self.reader = Some(unsafe { + new_reader.with_skip_validation(true) + }); + // Loop back to read from the new reader + } + Err(_) => { + // No more streams — the InputStream is exhausted + return Ok(None); + } + } + } + } } } /// Return the number of fields in the stream's schema. pub fn num_fields(&self) -> usize { - self.reader.schema().fields().len() + self.num_fields } } diff --git a/native/shuffle/src/partitioners/multi_partition.rs b/native/shuffle/src/partitioners/multi_partition.rs index a5077cf60f..c7db1a00f9 100644 --- a/native/shuffle/src/partitioners/multi_partition.rs +++ b/native/shuffle/src/partitioners/multi_partition.rs @@ -509,11 +509,7 @@ impl MultiPartitionShuffleRepartitioner { for partition_id in 0..num_output_partitions { let partition_writer = &mut self.partition_writers[partition_id]; let mut iter = partitioned_batches.produce(partition_id); - spilled_bytes += partition_writer.spill( - &mut iter, - &self.runtime, - &self.metrics, - )?; + spilled_bytes += partition_writer.spill(&mut iter, &self.runtime, &self.metrics)?; } self.reservation.free(); diff --git a/native/shuffle/src/writers/buf_batch_writer.rs b/native/shuffle/src/writers/buf_batch_writer.rs index 0ffedaa504..4c8d32f25d 100644 --- a/native/shuffle/src/writers/buf_batch_writer.rs +++ b/native/shuffle/src/writers/buf_batch_writer.rs @@ -81,5 +81,4 @@ impl BufBatchWriter { self.writer.finish()?; Ok(()) } - } diff --git a/native/shuffle/src/writers/spill.rs b/native/shuffle/src/writers/spill.rs index 9155798d23..c6feb34764 100644 --- a/native/shuffle/src/writers/spill.rs +++ b/native/shuffle/src/writers/spill.rs @@ -91,11 +91,8 @@ impl PartitionWriter { let file = &mut self.spill_file.as_mut().unwrap().file; let start_pos = file.metadata().map(|m| m.len()).unwrap_or(0); - let mut writer = StreamWriter::try_new_with_options( - file, - &self.schema, - self.write_options.clone(), - )?; + let mut writer = + StreamWriter::try_new_with_options(file, &self.schema, self.write_options.clone())?; let batch = batch?; let mut encode_timer = metrics.encode_time.timer(); diff --git a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala index e198ac99ff..e9e1968ccd 100644 --- a/spark/src/main/scala/org/apache/comet/CometExecIterator.scala +++ b/spark/src/main/scala/org/apache/comet/CometExecIterator.scala @@ -68,7 +68,7 @@ class CometExecIterator( partitionIndex: Int, broadcastedHadoopConfForEncryption: Option[Broadcast[SerializableConfiguration]] = None, encryptedFilePaths: Seq[String] = Seq.empty, - shuffleBlockIterators: Map[Int, CometShuffleBlockIterator] = Map.empty) + shuffleInputStreams: Map[Int, java.io.InputStream] = Map.empty) extends Iterator[ColumnarBatch] with Logging { @@ -79,11 +79,11 @@ class CometExecIterator( private val taskAttemptId = TaskContext.get().taskAttemptId private val taskCPUs = TaskContext.get().cpus() private val cometTaskMemoryManager = new CometTaskMemoryManager(id, taskAttemptId) - // Build a mixed array of iterators: CometShuffleBlockIterator for shuffle - // scan indices, CometBatchIterator for regular scan indices. + // Build a mixed array of iterators: InputStream 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 (_, idx) if shuffleInputStreams.contains(idx) => + shuffleInputStreams(idx).asInstanceOf[Object] case (iterator, _) => new CometBatchIterator(iterator, nativeUtil).asInstanceOf[Object] }.toArray @@ -235,7 +235,7 @@ class CometExecIterator( currentBatch = null } nativeUtil.close() - shuffleBlockIterators.values.foreach(_.close()) + shuffleInputStreams.values.foreach(_.close()) nativeLib.releasePlan(plan) if (tracingEnabled) { diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometExecRDD.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometExecRDD.scala index c5014818c4..963505dcf2 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometExecRDD.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometExecRDD.scala @@ -111,11 +111,11 @@ private[spark] class CometExecRDD( serializedPlan } - // Create shuffle block iterators for inputs that are CometShuffledBatchRDD - val shuffleBlockIters = shuffleScanIndices.flatMap { idx => + // Create raw InputStreams for inputs that are CometShuffledBatchRDD + val shuffleStreams = shuffleScanIndices.flatMap { idx => inputRDDs(idx) match { case rdd: CometShuffledBatchRDD => - Some(idx -> rdd.computeAsShuffleBlockIterator(partition.inputPartitions(idx), context)) + Some(idx -> rdd.computeAsRawStream(partition.inputPartitions(idx), context)) case _ => None } }.toMap @@ -130,7 +130,7 @@ private[spark] class CometExecRDD( partition.index, broadcastedHadoopConfForEncryption, encryptedFilePaths, - shuffleBlockIters) + shuffleStreams) // Register ScalarSubqueries so native code can look them up subqueries.foreach(sub => CometScalarSubquery.setSubquery(it.id, sub)) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala index 7604910b06..470f18a829 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala @@ -27,8 +27,6 @@ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsRe import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.comet.CometShuffleBlockIterator - /** * Different from [[org.apache.spark.sql.execution.ShuffledRowRDD]], this RDD is specialized for * reading shuffled data through [[CometBlockStoreShuffleReader]]. The shuffled data is read in an @@ -149,14 +147,14 @@ class CometShuffledBatchRDD( } /** - * Creates a CometShuffleBlockIterator that provides raw compressed shuffle blocks for direct - * consumption by native code, bypassing Arrow FFI. + * Returns the raw InputStream of concatenated Arrow IPC streams for direct consumption by + * native code via ShuffleStreamReader. */ - def computeAsShuffleBlockIterator( + def computeAsRawStream( split: Partition, - context: TaskContext): CometShuffleBlockIterator = { + context: TaskContext): java.io.InputStream = { val reader = createReader(split, context) - new CometShuffleBlockIterator(reader.readAsRawStream()) + reader.readAsRawStream() } override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { From ef35a427023bed0509dc3de2a0b67c545761b62d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Apr 2026 10:14:01 -0600 Subject: [PATCH 42/46] fix: apply spotless formatting --- .../logs/comet-offheap4g-q1.log | 76 + .../logs/comet-offheap4g-q1.time | 1628 ++++ .../logs/comet-offheap4g-q5.log | 317 + .../logs/comet-offheap4g-q5.time | 5639 ++++++++++++++ .../logs/comet-offheap4g-q9.log | 281 + .../logs/comet-offheap4g-q9.time | 6872 +++++++++++++++++ .../logs/comet-offheap8g-q1.log | 76 + .../logs/comet-offheap8g-q1.time | 1629 ++++ .../logs/comet-offheap8g-q5.log | 317 + .../logs/comet-offheap8g-q5.time | 5639 ++++++++++++++ .../logs/comet-offheap8g-q9.log | 281 + .../logs/comet-offheap8g-q9.time | 6868 ++++++++++++++++ .../logs/spark-offheap4g-q1.log | 80 + .../logs/spark-offheap4g-q1.time | 1650 ++++ .../logs/spark-offheap4g-q5.log | 330 + .../logs/spark-offheap4g-q5.time | 4441 +++++++++++ .../logs/spark-offheap4g-q9.log | 290 + .../logs/spark-offheap4g-q9.time | 5067 ++++++++++++ .../tpc/memory-profile-results/results.csv | 4 + .../plans/2026-04-01-spill-callback.md | 782 ++ .../shuffle/CometShuffledRowRDD.scala | 4 +- 21 files changed, 42268 insertions(+), 3 deletions(-) create mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.log create mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.time create mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.log create mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.time create mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.log create mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.time create mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.log create mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.time create mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.log create mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.time create mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.log create mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.time create mode 100644 benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.log create mode 100644 benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.time create mode 100644 benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.log create mode 100644 benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.time create mode 100644 benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.log create mode 100644 benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.time create mode 100644 benchmarks/tpc/memory-profile-results/results.csv create mode 100644 docs/superpowers/plans/2026-04-01-spill-callback.md diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.log b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.log new file mode 100644 index 0000000000..1aab0a482d --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.log @@ -0,0 +1,76 @@ +Registering table customer from /opt/tpch/sf100/customer +Registering table lineitem from /opt/tpch/sf100/lineitem +Registering table nation from /opt/tpch/sf100/nation +Registering table orders from /opt/tpch/sf100/orders +Registering table part from /opt/tpch/sf100/part +Registering table partsupp from /opt/tpch/sf100/partsupp +Registering table region from /opt/tpch/sf100/region +Registering table supplier from /opt/tpch/sf100/supplier + +============================================================ +Starting iteration 1 of 1 +============================================================ + +Running query 1 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q1.sql +Executing: -- CometBench-H query 1 derived from TPC-H query 1 under the terms of the TPC Fair Use Policy. +-- TP... +== Physical Plan == +AdaptiveSparkPlan (9) ++- CometSort (8) + +- CometExchange (7) + +- CometHashAggregate (6) + +- CometExchange (5) + +- CometHashAggregate (4) + +- CometProject (3) + +- CometFilter (2) + +- CometNativeScan parquet (1) + + +(1) CometNativeScan parquet +Output [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] +PushedFilters: [IsNotNull(l_shipdate), LessThanOrEqual(l_shipdate,1998-09-24)] +ReadSchema: struct + +(2) CometFilter +Input [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] +Condition : (isnotnull(l_shipdate#26) AND (l_shipdate#26 <= 1998-09-24)) + +(3) CometProject +Input [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] +Arguments: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25], [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25] + +(4) CometHashAggregate +Input [6]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25] +Keys [2]: [l_returnflag#24, l_linestatus#25] +Functions [8]: [partial_sum(l_quantity#20), partial_sum(l_extendedprice#21), partial_sum((l_extendedprice#21 * (1 - l_discount#22))), partial_sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23))), partial_avg(l_quantity#20), partial_avg(l_extendedprice#21), partial_avg(l_discount#22), partial_count(1)] + +(5) CometExchange +Input [17]: [l_returnflag#24, l_linestatus#25, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, count#172L, sum#173, count#174L, sum#175, count#176L, count#177L] +Arguments: hashpartitioning(l_returnflag#24, l_linestatus#25, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=83] + +(6) CometHashAggregate +Input [17]: [l_returnflag#24, l_linestatus#25, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, count#172L, sum#173, count#174L, sum#175, count#176L, count#177L] +Keys [2]: [l_returnflag#24, l_linestatus#25] +Functions [8]: [sum(l_quantity#20), sum(l_extendedprice#21), sum((l_extendedprice#21 * (1 - l_discount#22))), sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23))), avg(l_quantity#20), avg(l_extendedprice#21), avg(l_discount#22), count(1)] + +(7) CometExchange +Input [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] +Arguments: rangepartitioning(l_returnflag#24 ASC NULLS FIRST, l_linestatus#25 ASC NULLS FIRST, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=85] + +(8) CometSort +Input [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] +Arguments: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L], [l_returnflag#24 ASC NULLS FIRST, l_linestatus#25 ASC NULLS FIRST] + +(9) AdaptiveSparkPlan +Output [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] +Arguments: isFinalPlan=false + + +Query 1 returned 4 rows, hash=6839cc802df6220ed2efdaaaa48373a6 +Query 1 took 326.87 seconds + +Iteration 1 took 326.87 seconds + +Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/comet-offheap4g-q1-tpch-1775045118244.json diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.time b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.time new file mode 100644 index 0000000000..8eef79be16 --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.time @@ -0,0 +1,1628 @@ +26/04/01 05:59:46 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) +26/04/01 05:59:46 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address +26/04/01 05:59:46 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +26/04/01 05:59:47 INFO SparkContext: Running Spark version 3.5.8 +26/04/01 05:59:47 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 05:59:47 INFO SparkContext: Java version 17.0.17 +26/04/01 05:59:47 INFO ResourceUtils: ============================================================== +26/04/01 05:59:47 INFO ResourceUtils: No custom resources configured for spark.driver. +26/04/01 05:59:47 INFO ResourceUtils: ============================================================== +26/04/01 05:59:47 INFO SparkContext: Submitted application: comet-offheap4g-q1 benchmark derived from tpch +26/04/01 05:59:47 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 4096, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) +26/04/01 05:59:47 INFO ResourceProfile: Limiting resource is cpu +26/04/01 05:59:47 INFO ResourceProfileManager: Added ResourceProfile id: 0 +26/04/01 05:59:47 INFO SecurityManager: Changing view acls to: andy +26/04/01 05:59:47 INFO SecurityManager: Changing modify acls to: andy +26/04/01 05:59:47 INFO SecurityManager: Changing view acls groups to: +26/04/01 05:59:47 INFO SecurityManager: Changing modify acls groups to: +26/04/01 05:59:47 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY +26/04/01 05:59:47 INFO Utils: Successfully started service 'sparkDriver' on port 58233. +26/04/01 05:59:47 INFO SparkEnv: Registering MapOutputTracker +26/04/01 05:59:47 INFO SparkEnv: Registering BlockManagerMaster +26/04/01 05:59:47 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information +26/04/01 05:59:47 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up +26/04/01 05:59:47 INFO SparkEnv: Registering BlockManagerMasterHeartbeat +26/04/01 05:59:47 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-c8f0d8ba-25f5-42f4-99d1-8b3178ae04da +26/04/01 05:59:47 INFO MemoryStore: MemoryStore started with capacity 8.6 GiB +26/04/01 05:59:47 INFO SparkEnv: Registering OutputCommitCoordinator +26/04/01 05:59:47 INFO SparkContext: Added JAR file:///Users/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar at spark://10.0.0.133:58233/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775044787060 +26/04/01 05:59:47 INFO CometDriverPlugin: CometDriverPlugin init +26/04/01 05:59:47 INFO CometDriverPlugin: Setting spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions +26/04/01 05:59:47 INFO CometDriverPlugin: Comet metrics reporting is disabled. Set spark.comet.metrics.enabled=true to enable. +26/04/01 05:59:47 INFO CometDriverPlugin: Comet is running in unified memory mode and sharing off-heap memory with Spark +26/04/01 05:59:47 INFO DriverPluginContainer: Initialized driver component for plugin org.apache.spark.CometPlugin. +26/04/01 05:59:47 INFO Executor: Starting executor ID driver on host 10.0.0.133 +26/04/01 05:59:47 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 05:59:47 INFO Executor: Java version 17.0.17 +26/04/01 05:59:47 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' +26/04/01 05:59:47 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@1306f9cd for default. +26/04/01 05:59:47 INFO Executor: Fetching spark://10.0.0.133:58233/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775044787060 +26/04/01 05:59:47 INFO TransportClientFactory: Successfully created connection to /10.0.0.133:58233 after 9 ms (0 ms spent in bootstraps) +26/04/01 05:59:47 INFO Utils: Fetching spark://10.0.0.133:58233/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-8b7018bf-f101-4273-93fa-9cf7eb6d820a/userFiles-45b7fef5-97ad-40d7-be54-b4c80c409c73/fetchFileTemp7788925620064686833.tmp +26/04/01 05:59:47 INFO Executor: Adding file:/private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-8b7018bf-f101-4273-93fa-9cf7eb6d820a/userFiles-45b7fef5-97ad-40d7-be54-b4c80c409c73/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to class loader default +26/04/01 05:59:47 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 58235. +26/04/01 05:59:47 INFO NettyBlockTransferService: Server created on 10.0.0.133:58235 +26/04/01 05:59:47 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy +26/04/01 05:59:47 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 58235, None) +26/04/01 05:59:47 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:58235 with 8.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 58235, None) +26/04/01 05:59:47 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 58235, None) +26/04/01 05:59:47 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 58235, None) +26/04/01 05:59:47 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. +26/04/01 05:59:47 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. +26/04/01 05:59:47 INFO InMemoryFileIndex: It took 18 ms to list leaf files for 1 paths. +26/04/01 05:59:47 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:59:47 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:59:47 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:59:47 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:59:47 INFO DAGScheduler: Missing parents: List() +26/04/01 05:59:47 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:59:47 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 05:59:48 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 05:59:48 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:48 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:59:48 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:59:48 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 +26/04/01 05:59:48 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 05:59:48 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) +26/04/01 05:59:48 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 2041 bytes result sent to driver +26/04/01 05:59:48 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 127 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:59:48 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool +26/04/01 05:59:48 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.379 s +26/04/01 05:59:48 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:59:48 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished +26/04/01 05:59:48 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.389787 s +26/04/01 05:59:48 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:50 INFO NativeBase: Couldn't locate log file from either COMET_CONF_DIR or comet.log.file.path. Using default log configuration with INFO log level which emits to stderr +26/04/01 05:59:50 INFO core/src/lib.rs: Comet native library version 0.15.0 initialized +26/04/01 05:59:50 INFO NativeBase: Setting system property arrow.enable_unsafe_memory_access to true +26/04/01 05:59:50 INFO NativeBase: Setting system property arrow.enable_null_check_for_get to false +26/04/01 05:59:51 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. +26/04/01 05:59:51 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:59:51 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:59:51 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() +26/04/01 05:59:51 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:59:51 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 +26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) +26/04/01 05:59:51 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2344 bytes result sent to driver +26/04/01 05:59:51 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 15 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:59:51 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool +26/04/01 05:59:51 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.024 s +26/04/01 05:59:51 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:59:51 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished +26/04/01 05:59:51 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.024978 s +26/04/01 05:59:51 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 05:59:51 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:59:51 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:59:51 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() +26/04/01 05:59:51 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:59:51 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 +26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) +26/04/01 05:59:51 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver +26/04/01 05:59:51 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 8 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:59:51 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool +26/04/01 05:59:51 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.016 s +26/04/01 05:59:51 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:59:51 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished +26/04/01 05:59:51 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.017191 s +26/04/01 05:59:51 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. +26/04/01 05:59:51 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:59:51 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:59:51 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() +26/04/01 05:59:51 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:59:51 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 +26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) +26/04/01 05:59:51 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver +26/04/01 05:59:51 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:59:51 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool +26/04/01 05:59:51 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s +26/04/01 05:59:51 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:59:51 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished +26/04/01 05:59:51 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.015922 s +26/04/01 05:59:51 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. +26/04/01 05:59:51 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:59:51 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:59:51 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() +26/04/01 05:59:51 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:59:51 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 +26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9299 bytes) +26/04/01 05:59:51 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) +26/04/01 05:59:51 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver +26/04/01 05:59:51 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:59:51 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool +26/04/01 05:59:51 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s +26/04/01 05:59:51 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:59:51 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished +26/04/01 05:59:51 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.016757 s +26/04/01 05:59:51 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 05:59:51 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:59:51 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:59:51 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() +26/04/01 05:59:51 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:59:51 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 +26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) +26/04/01 05:59:51 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver +26/04/01 05:59:51 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 7 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:59:51 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool +26/04/01 05:59:51 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s +26/04/01 05:59:51 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:59:51 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished +26/04/01 05:59:51 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.016127 s +26/04/01 05:59:51 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. +26/04/01 05:59:51 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:59:51 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:59:51 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() +26/04/01 05:59:51 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:59:51 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 +26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) +26/04/01 05:59:51 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver +26/04/01 05:59:51 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:59:51 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool +26/04/01 05:59:51 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 05:59:51 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:59:51 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished +26/04/01 05:59:51 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.015107 s +26/04/01 05:59:51 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 05:59:51 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:59:51 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:59:51 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() +26/04/01 05:59:51 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:59:51 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 +26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) +26/04/01 05:59:51 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver +26/04/01 05:59:51 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:59:51 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool +26/04/01 05:59:51 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 05:59:51 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:59:51 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished +26/04/01 05:59:51 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.014655 s +26/04/01 05:59:51 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_shipdate),LessThanOrEqual(l_shipdate,1998-09-24) +26/04/01 05:59:51 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_shipdate#26),(l_shipdate#26 <= 1998-09-24) +26/04/01 05:59:51 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 05:59:51 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.5 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:58235 (size: 35.5 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:59:51 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 05:59:51 INFO DAGScheduler: Registering RDD 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 +26/04/01 05:59:51 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions +26/04/01 05:59:51 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() +26/04/01 05:59:51 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 46.8 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 10.1 KiB, free 8.6 GiB) +26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:58235 (size: 10.1 KiB, free: 8.6 GiB) +26/04/01 05:59:51 INFO SparkContext: Created broadcast 9 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:59:51 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 8.0 with 208 tasks resource profile 0 +26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) +26/04/01 05:59:51 INFO TaskSetManager: Starting task 1.0 in stage 8.0 (TID 9) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9274 bytes) +26/04/01 05:59:51 INFO TaskSetManager: Starting task 2.0 in stage 8.0 (TID 10) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9275 bytes) +26/04/01 05:59:51 INFO TaskSetManager: Starting task 3.0 in stage 8.0 (TID 11) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9275 bytes) +26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) +26/04/01 05:59:51 INFO Executor: Running task 1.0 in stage 8.0 (TID 9) +26/04/01 05:59:51 INFO Executor: Running task 3.0 in stage 8.0 (TID 11) +26/04/01 05:59:51 INFO Executor: Running task 2.0 in stage 8.0 (TID 10) +26/04/01 05:59:51 INFO BaseAllocator: Debug mode disabled. Enable with the VM option -Darrow.memory.debug.allocator=true. +26/04/01 05:59:51 INFO DefaultAllocationManagerOption: allocation manager type not specified, using netty as the default type +26/04/01 05:59:51 INFO CheckAllocator: Using DefaultAllocationManager at memory/unsafe/DefaultAllocationManagerFactory.class +26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 05:59:51 INFO core/src/execution/jni_api.rs: Comet tokio runtime: using spark.executor.cores=4 worker threads +26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 05:59:58 INFO Executor: Finished task 1.0 in stage 8.0 (TID 9). 6830 bytes result sent to driver +26/04/01 05:59:58 INFO Executor: Finished task 2.0 in stage 8.0 (TID 10). 6830 bytes result sent to driver +26/04/01 05:59:58 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 6830 bytes result sent to driver +26/04/01 05:59:58 INFO Executor: Finished task 3.0 in stage 8.0 (TID 11). 6830 bytes result sent to driver +26/04/01 05:59:58 INFO TaskSetManager: Starting task 4.0 in stage 8.0 (TID 12) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9275 bytes) +26/04/01 05:59:58 INFO Executor: Running task 4.0 in stage 8.0 (TID 12) +26/04/01 05:59:58 INFO TaskSetManager: Starting task 5.0 in stage 8.0 (TID 13) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9275 bytes) +26/04/01 05:59:58 INFO TaskSetManager: Finished task 1.0 in stage 8.0 (TID 9) in 6754 ms on 10.0.0.133 (executor driver) (1/208) +26/04/01 05:59:58 INFO TaskSetManager: Finished task 2.0 in stage 8.0 (TID 10) in 6754 ms on 10.0.0.133 (executor driver) (2/208) +26/04/01 05:59:58 INFO Executor: Running task 5.0 in stage 8.0 (TID 13) +26/04/01 05:59:58 INFO TaskSetManager: Starting task 6.0 in stage 8.0 (TID 14) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9269 bytes) +26/04/01 05:59:58 INFO Executor: Running task 6.0 in stage 8.0 (TID 14) +26/04/01 05:59:58 INFO TaskSetManager: Starting task 7.0 in stage 8.0 (TID 15) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9274 bytes) +26/04/01 05:59:58 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 6756 ms on 10.0.0.133 (executor driver) (3/208) +26/04/01 05:59:58 INFO Executor: Running task 7.0 in stage 8.0 (TID 15) +26/04/01 05:59:58 INFO TaskSetManager: Finished task 3.0 in stage 8.0 (TID 11) in 6756 ms on 10.0.0.133 (executor driver) (4/208) +26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:05 INFO Executor: Finished task 4.0 in stage 8.0 (TID 12). 6830 bytes result sent to driver +26/04/01 06:00:05 INFO Executor: Finished task 6.0 in stage 8.0 (TID 14). 6830 bytes result sent to driver +26/04/01 06:00:05 INFO Executor: Finished task 7.0 in stage 8.0 (TID 15). 6830 bytes result sent to driver +26/04/01 06:00:05 INFO TaskSetManager: Starting task 8.0 in stage 8.0 (TID 16) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:05 INFO Executor: Finished task 5.0 in stage 8.0 (TID 13). 6830 bytes result sent to driver +26/04/01 06:00:05 INFO Executor: Running task 8.0 in stage 8.0 (TID 16) +26/04/01 06:00:05 INFO TaskSetManager: Starting task 9.0 in stage 8.0 (TID 17) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:05 INFO Executor: Running task 9.0 in stage 8.0 (TID 17) +26/04/01 06:00:05 INFO TaskSetManager: Starting task 10.0 in stage 8.0 (TID 18) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:05 INFO TaskSetManager: Finished task 4.0 in stage 8.0 (TID 12) in 6489 ms on 10.0.0.133 (executor driver) (5/208) +26/04/01 06:00:05 INFO Executor: Running task 10.0 in stage 8.0 (TID 18) +26/04/01 06:00:05 INFO TaskSetManager: Starting task 11.0 in stage 8.0 (TID 19) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:05 INFO TaskSetManager: Finished task 6.0 in stage 8.0 (TID 14) in 6489 ms on 10.0.0.133 (executor driver) (6/208) +26/04/01 06:00:05 INFO Executor: Running task 11.0 in stage 8.0 (TID 19) +26/04/01 06:00:05 INFO TaskSetManager: Finished task 7.0 in stage 8.0 (TID 15) in 6488 ms on 10.0.0.133 (executor driver) (7/208) +26/04/01 06:00:05 INFO TaskSetManager: Finished task 5.0 in stage 8.0 (TID 13) in 6490 ms on 10.0.0.133 (executor driver) (8/208) +26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:11 INFO Executor: Finished task 9.0 in stage 8.0 (TID 17). 6830 bytes result sent to driver +26/04/01 06:00:11 INFO TaskSetManager: Starting task 12.0 in stage 8.0 (TID 20) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:00:11 INFO Executor: Running task 12.0 in stage 8.0 (TID 20) +26/04/01 06:00:11 INFO TaskSetManager: Finished task 9.0 in stage 8.0 (TID 17) in 6468 ms on 10.0.0.133 (executor driver) (9/208) +26/04/01 06:00:11 INFO Executor: Finished task 8.0 in stage 8.0 (TID 16). 6830 bytes result sent to driver +26/04/01 06:00:11 INFO TaskSetManager: Starting task 13.0 in stage 8.0 (TID 21) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:00:11 INFO TaskSetManager: Finished task 8.0 in stage 8.0 (TID 16) in 6470 ms on 10.0.0.133 (executor driver) (10/208) +26/04/01 06:00:11 INFO Executor: Running task 13.0 in stage 8.0 (TID 21) +26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:11 INFO Executor: Finished task 10.0 in stage 8.0 (TID 18). 6830 bytes result sent to driver +26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:11 INFO TaskSetManager: Starting task 14.0 in stage 8.0 (TID 22) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:11 INFO Executor: Running task 14.0 in stage 8.0 (TID 22) +26/04/01 06:00:11 INFO TaskSetManager: Finished task 10.0 in stage 8.0 (TID 18) in 6472 ms on 10.0.0.133 (executor driver) (11/208) +26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:11 INFO Executor: Finished task 11.0 in stage 8.0 (TID 19). 6830 bytes result sent to driver +26/04/01 06:00:11 INFO TaskSetManager: Starting task 15.0 in stage 8.0 (TID 23) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:11 INFO Executor: Running task 15.0 in stage 8.0 (TID 23) +26/04/01 06:00:11 INFO TaskSetManager: Finished task 11.0 in stage 8.0 (TID 19) in 6473 ms on 10.0.0.133 (executor driver) (12/208) +26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:17 INFO Executor: Finished task 14.0 in stage 8.0 (TID 22). 6787 bytes result sent to driver +26/04/01 06:00:17 INFO TaskSetManager: Starting task 16.0 in stage 8.0 (TID 24) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:17 INFO Executor: Running task 16.0 in stage 8.0 (TID 24) +26/04/01 06:00:17 INFO TaskSetManager: Finished task 14.0 in stage 8.0 (TID 22) in 6440 ms on 10.0.0.133 (executor driver) (13/208) +26/04/01 06:00:17 INFO Executor: Finished task 15.0 in stage 8.0 (TID 23). 6830 bytes result sent to driver +26/04/01 06:00:17 INFO TaskSetManager: Starting task 17.0 in stage 8.0 (TID 25) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:17 INFO Executor: Running task 17.0 in stage 8.0 (TID 25) +26/04/01 06:00:17 INFO TaskSetManager: Finished task 15.0 in stage 8.0 (TID 23) in 6439 ms on 10.0.0.133 (executor driver) (14/208) +26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:17 INFO Executor: Finished task 13.0 in stage 8.0 (TID 21). 6787 bytes result sent to driver +26/04/01 06:00:17 INFO TaskSetManager: Starting task 18.0 in stage 8.0 (TID 26) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:17 INFO TaskSetManager: Finished task 13.0 in stage 8.0 (TID 21) in 6447 ms on 10.0.0.133 (executor driver) (15/208) +26/04/01 06:00:17 INFO Executor: Running task 18.0 in stage 8.0 (TID 26) +26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:17 INFO Executor: Finished task 12.0 in stage 8.0 (TID 20). 6787 bytes result sent to driver +26/04/01 06:00:17 INFO TaskSetManager: Starting task 19.0 in stage 8.0 (TID 27) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:00:17 INFO Executor: Running task 19.0 in stage 8.0 (TID 27) +26/04/01 06:00:17 INFO TaskSetManager: Finished task 12.0 in stage 8.0 (TID 20) in 6454 ms on 10.0.0.133 (executor driver) (16/208) +26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:24 INFO Executor: Finished task 18.0 in stage 8.0 (TID 26). 6787 bytes result sent to driver +26/04/01 06:00:24 INFO TaskSetManager: Starting task 20.0 in stage 8.0 (TID 28) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:00:24 INFO Executor: Running task 20.0 in stage 8.0 (TID 28) +26/04/01 06:00:24 INFO TaskSetManager: Finished task 18.0 in stage 8.0 (TID 26) in 6430 ms on 10.0.0.133 (executor driver) (17/208) +26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:24 INFO Executor: Finished task 19.0 in stage 8.0 (TID 27). 6787 bytes result sent to driver +26/04/01 06:00:24 INFO TaskSetManager: Starting task 21.0 in stage 8.0 (TID 29) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:00:24 INFO Executor: Running task 21.0 in stage 8.0 (TID 29) +26/04/01 06:00:24 INFO TaskSetManager: Finished task 19.0 in stage 8.0 (TID 27) in 6431 ms on 10.0.0.133 (executor driver) (18/208) +26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:24 INFO Executor: Finished task 17.0 in stage 8.0 (TID 25). 6787 bytes result sent to driver +26/04/01 06:00:24 INFO TaskSetManager: Starting task 22.0 in stage 8.0 (TID 30) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:00:24 INFO TaskSetManager: Finished task 17.0 in stage 8.0 (TID 25) in 6442 ms on 10.0.0.133 (executor driver) (19/208) +26/04/01 06:00:24 INFO Executor: Running task 22.0 in stage 8.0 (TID 30) +26/04/01 06:00:24 INFO Executor: Finished task 16.0 in stage 8.0 (TID 24). 6787 bytes result sent to driver +26/04/01 06:00:24 INFO TaskSetManager: Starting task 23.0 in stage 8.0 (TID 31) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:00:24 INFO Executor: Running task 23.0 in stage 8.0 (TID 31) +26/04/01 06:00:24 INFO TaskSetManager: Finished task 16.0 in stage 8.0 (TID 24) in 6446 ms on 10.0.0.133 (executor driver) (20/208) +26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:30 INFO Executor: Finished task 23.0 in stage 8.0 (TID 31). 6787 bytes result sent to driver +26/04/01 06:00:30 INFO TaskSetManager: Starting task 24.0 in stage 8.0 (TID 32) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:00:30 INFO Executor: Running task 24.0 in stage 8.0 (TID 32) +26/04/01 06:00:30 INFO TaskSetManager: Finished task 23.0 in stage 8.0 (TID 31) in 6471 ms on 10.0.0.133 (executor driver) (21/208) +26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:30 INFO Executor: Finished task 22.0 in stage 8.0 (TID 30). 6873 bytes result sent to driver +26/04/01 06:00:30 INFO TaskSetManager: Starting task 25.0 in stage 8.0 (TID 33) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:00:30 INFO Executor: Running task 25.0 in stage 8.0 (TID 33) +26/04/01 06:00:30 INFO Executor: Finished task 21.0 in stage 8.0 (TID 29). 6830 bytes result sent to driver +26/04/01 06:00:30 INFO TaskSetManager: Finished task 22.0 in stage 8.0 (TID 30) in 6478 ms on 10.0.0.133 (executor driver) (22/208) +26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:30 INFO Executor: Finished task 20.0 in stage 8.0 (TID 28). 6830 bytes result sent to driver +26/04/01 06:00:30 INFO TaskSetManager: Starting task 26.0 in stage 8.0 (TID 34) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:30 INFO Executor: Running task 26.0 in stage 8.0 (TID 34) +26/04/01 06:00:30 INFO TaskSetManager: Starting task 27.0 in stage 8.0 (TID 35) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:30 INFO TaskSetManager: Finished task 21.0 in stage 8.0 (TID 29) in 6486 ms on 10.0.0.133 (executor driver) (23/208) +26/04/01 06:00:30 INFO Executor: Running task 27.0 in stage 8.0 (TID 35) +26/04/01 06:00:30 INFO TaskSetManager: Finished task 20.0 in stage 8.0 (TID 28) in 6494 ms on 10.0.0.133 (executor driver) (24/208) +26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:37 INFO Executor: Finished task 24.0 in stage 8.0 (TID 32). 6830 bytes result sent to driver +26/04/01 06:00:37 INFO TaskSetManager: Starting task 28.0 in stage 8.0 (TID 36) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:37 INFO Executor: Running task 28.0 in stage 8.0 (TID 36) +26/04/01 06:00:37 INFO TaskSetManager: Finished task 24.0 in stage 8.0 (TID 32) in 6421 ms on 10.0.0.133 (executor driver) (25/208) +26/04/01 06:00:37 INFO Executor: Finished task 27.0 in stage 8.0 (TID 35). 6787 bytes result sent to driver +26/04/01 06:00:37 INFO TaskSetManager: Starting task 29.0 in stage 8.0 (TID 37) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:37 INFO Executor: Running task 29.0 in stage 8.0 (TID 37) +26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:37 INFO TaskSetManager: Finished task 27.0 in stage 8.0 (TID 35) in 6417 ms on 10.0.0.133 (executor driver) (26/208) +26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:37 INFO Executor: Finished task 26.0 in stage 8.0 (TID 34). 6873 bytes result sent to driver +26/04/01 06:00:37 INFO TaskSetManager: Starting task 30.0 in stage 8.0 (TID 38) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:00:37 INFO Executor: Running task 30.0 in stage 8.0 (TID 38) +26/04/01 06:00:37 INFO TaskSetManager: Finished task 26.0 in stage 8.0 (TID 34) in 6424 ms on 10.0.0.133 (executor driver) (27/208) +26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:37 INFO Executor: Finished task 25.0 in stage 8.0 (TID 33). 6873 bytes result sent to driver +26/04/01 06:00:37 INFO TaskSetManager: Starting task 31.0 in stage 8.0 (TID 39) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:00:37 INFO TaskSetManager: Finished task 25.0 in stage 8.0 (TID 33) in 6429 ms on 10.0.0.133 (executor driver) (28/208) +26/04/01 06:00:37 INFO Executor: Running task 31.0 in stage 8.0 (TID 39) +26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:43 INFO Executor: Finished task 28.0 in stage 8.0 (TID 36). 6787 bytes result sent to driver +26/04/01 06:00:43 INFO TaskSetManager: Starting task 32.0 in stage 8.0 (TID 40) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:43 INFO Executor: Running task 32.0 in stage 8.0 (TID 40) +26/04/01 06:00:43 INFO TaskSetManager: Finished task 28.0 in stage 8.0 (TID 36) in 6372 ms on 10.0.0.133 (executor driver) (29/208) +26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:43 INFO Executor: Finished task 30.0 in stage 8.0 (TID 38). 6787 bytes result sent to driver +26/04/01 06:00:43 INFO TaskSetManager: Starting task 33.0 in stage 8.0 (TID 41) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:43 INFO Executor: Running task 33.0 in stage 8.0 (TID 41) +26/04/01 06:00:43 INFO TaskSetManager: Finished task 30.0 in stage 8.0 (TID 38) in 6367 ms on 10.0.0.133 (executor driver) (30/208) +26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:43 INFO Executor: Finished task 31.0 in stage 8.0 (TID 39). 6787 bytes result sent to driver +26/04/01 06:00:43 INFO TaskSetManager: Starting task 34.0 in stage 8.0 (TID 42) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:43 INFO TaskSetManager: Finished task 31.0 in stage 8.0 (TID 39) in 6372 ms on 10.0.0.133 (executor driver) (31/208) +26/04/01 06:00:43 INFO Executor: Running task 34.0 in stage 8.0 (TID 42) +26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:43 INFO Executor: Finished task 29.0 in stage 8.0 (TID 37). 6787 bytes result sent to driver +26/04/01 06:00:43 INFO TaskSetManager: Starting task 35.0 in stage 8.0 (TID 43) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:43 INFO Executor: Running task 35.0 in stage 8.0 (TID 43) +26/04/01 06:00:43 INFO TaskSetManager: Finished task 29.0 in stage 8.0 (TID 37) in 6388 ms on 10.0.0.133 (executor driver) (32/208) +26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:50 INFO Executor: Finished task 33.0 in stage 8.0 (TID 41). 6787 bytes result sent to driver +26/04/01 06:00:50 INFO TaskSetManager: Starting task 36.0 in stage 8.0 (TID 44) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:00:50 INFO TaskSetManager: Finished task 33.0 in stage 8.0 (TID 41) in 6375 ms on 10.0.0.133 (executor driver) (33/208) +26/04/01 06:00:50 INFO Executor: Running task 36.0 in stage 8.0 (TID 44) +26/04/01 06:00:50 INFO Executor: Finished task 32.0 in stage 8.0 (TID 40). 6787 bytes result sent to driver +26/04/01 06:00:50 INFO TaskSetManager: Starting task 37.0 in stage 8.0 (TID 45) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:00:50 INFO Executor: Running task 37.0 in stage 8.0 (TID 45) +26/04/01 06:00:50 INFO TaskSetManager: Finished task 32.0 in stage 8.0 (TID 40) in 6379 ms on 10.0.0.133 (executor driver) (34/208) +26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:50 INFO Executor: Finished task 34.0 in stage 8.0 (TID 42). 6787 bytes result sent to driver +26/04/01 06:00:50 INFO TaskSetManager: Starting task 38.0 in stage 8.0 (TID 46) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:50 INFO Executor: Running task 38.0 in stage 8.0 (TID 46) +26/04/01 06:00:50 INFO TaskSetManager: Finished task 34.0 in stage 8.0 (TID 42) in 6380 ms on 10.0.0.133 (executor driver) (35/208) +26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:50 INFO Executor: Finished task 35.0 in stage 8.0 (TID 43). 6787 bytes result sent to driver +26/04/01 06:00:50 INFO TaskSetManager: Starting task 39.0 in stage 8.0 (TID 47) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:50 INFO TaskSetManager: Finished task 35.0 in stage 8.0 (TID 43) in 6377 ms on 10.0.0.133 (executor driver) (36/208) +26/04/01 06:00:50 INFO Executor: Running task 39.0 in stage 8.0 (TID 47) +26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:56 INFO Executor: Finished task 36.0 in stage 8.0 (TID 44). 6787 bytes result sent to driver +26/04/01 06:00:56 INFO TaskSetManager: Starting task 40.0 in stage 8.0 (TID 48) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:56 INFO Executor: Running task 40.0 in stage 8.0 (TID 48) +26/04/01 06:00:56 INFO TaskSetManager: Finished task 36.0 in stage 8.0 (TID 44) in 6459 ms on 10.0.0.133 (executor driver) (37/208) +26/04/01 06:00:56 INFO Executor: Finished task 37.0 in stage 8.0 (TID 45). 6787 bytes result sent to driver +26/04/01 06:00:56 INFO TaskSetManager: Starting task 41.0 in stage 8.0 (TID 49) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:00:56 INFO TaskSetManager: Finished task 37.0 in stage 8.0 (TID 45) in 6459 ms on 10.0.0.133 (executor driver) (38/208) +26/04/01 06:00:56 INFO Executor: Running task 41.0 in stage 8.0 (TID 49) +26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:56 INFO Executor: Finished task 38.0 in stage 8.0 (TID 46). 6787 bytes result sent to driver +26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:56 INFO TaskSetManager: Starting task 42.0 in stage 8.0 (TID 50) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:00:56 INFO TaskSetManager: Finished task 38.0 in stage 8.0 (TID 46) in 6447 ms on 10.0.0.133 (executor driver) (39/208) +26/04/01 06:00:56 INFO Executor: Running task 42.0 in stage 8.0 (TID 50) +26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:56 INFO Executor: Finished task 39.0 in stage 8.0 (TID 47). 6787 bytes result sent to driver +26/04/01 06:00:56 INFO TaskSetManager: Starting task 43.0 in stage 8.0 (TID 51) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:00:56 INFO TaskSetManager: Finished task 39.0 in stage 8.0 (TID 47) in 6447 ms on 10.0.0.133 (executor driver) (40/208) +26/04/01 06:00:56 INFO Executor: Running task 43.0 in stage 8.0 (TID 51) +26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:02 INFO Executor: Finished task 42.0 in stage 8.0 (TID 50). 6787 bytes result sent to driver +26/04/01 06:01:02 INFO TaskSetManager: Starting task 44.0 in stage 8.0 (TID 52) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:02 INFO Executor: Running task 44.0 in stage 8.0 (TID 52) +26/04/01 06:01:02 INFO TaskSetManager: Finished task 42.0 in stage 8.0 (TID 50) in 6390 ms on 10.0.0.133 (executor driver) (41/208) +26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:02 INFO Executor: Finished task 43.0 in stage 8.0 (TID 51). 6787 bytes result sent to driver +26/04/01 06:01:02 INFO TaskSetManager: Starting task 45.0 in stage 8.0 (TID 53) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:02 INFO Executor: Running task 45.0 in stage 8.0 (TID 53) +26/04/01 06:01:02 INFO TaskSetManager: Finished task 43.0 in stage 8.0 (TID 51) in 6390 ms on 10.0.0.133 (executor driver) (42/208) +26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:02 INFO Executor: Finished task 41.0 in stage 8.0 (TID 49). 6787 bytes result sent to driver +26/04/01 06:01:02 INFO Executor: Finished task 40.0 in stage 8.0 (TID 48). 6787 bytes result sent to driver +26/04/01 06:01:02 INFO TaskSetManager: Starting task 46.0 in stage 8.0 (TID 54) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:02 INFO TaskSetManager: Finished task 41.0 in stage 8.0 (TID 49) in 6439 ms on 10.0.0.133 (executor driver) (43/208) +26/04/01 06:01:02 INFO Executor: Running task 46.0 in stage 8.0 (TID 54) +26/04/01 06:01:02 INFO TaskSetManager: Starting task 47.0 in stage 8.0 (TID 55) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:02 INFO Executor: Running task 47.0 in stage 8.0 (TID 55) +26/04/01 06:01:02 INFO TaskSetManager: Finished task 40.0 in stage 8.0 (TID 48) in 6440 ms on 10.0.0.133 (executor driver) (44/208) +26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:09 INFO Executor: Finished task 45.0 in stage 8.0 (TID 53). 6787 bytes result sent to driver +26/04/01 06:01:09 INFO TaskSetManager: Starting task 48.0 in stage 8.0 (TID 56) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:01:09 INFO TaskSetManager: Finished task 45.0 in stage 8.0 (TID 53) in 6151 ms on 10.0.0.133 (executor driver) (45/208) +26/04/01 06:01:09 INFO Executor: Running task 48.0 in stage 8.0 (TID 56) +26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:09 INFO Executor: Finished task 47.0 in stage 8.0 (TID 55). 6787 bytes result sent to driver +26/04/01 06:01:09 INFO TaskSetManager: Starting task 49.0 in stage 8.0 (TID 57) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:09 INFO TaskSetManager: Finished task 47.0 in stage 8.0 (TID 55) in 6395 ms on 10.0.0.133 (executor driver) (46/208) +26/04/01 06:01:09 INFO Executor: Running task 49.0 in stage 8.0 (TID 57) +26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:09 INFO Executor: Finished task 46.0 in stage 8.0 (TID 54). 6787 bytes result sent to driver +26/04/01 06:01:09 INFO TaskSetManager: Starting task 50.0 in stage 8.0 (TID 58) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:01:09 INFO TaskSetManager: Finished task 46.0 in stage 8.0 (TID 54) in 6403 ms on 10.0.0.133 (executor driver) (47/208) +26/04/01 06:01:09 INFO Executor: Running task 50.0 in stage 8.0 (TID 58) +26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:09 INFO Executor: Finished task 44.0 in stage 8.0 (TID 52). 6787 bytes result sent to driver +26/04/01 06:01:09 INFO TaskSetManager: Starting task 51.0 in stage 8.0 (TID 59) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:01:09 INFO Executor: Running task 51.0 in stage 8.0 (TID 59) +26/04/01 06:01:09 INFO TaskSetManager: Finished task 44.0 in stage 8.0 (TID 52) in 6648 ms on 10.0.0.133 (executor driver) (48/208) +26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:15 INFO Executor: Finished task 48.0 in stage 8.0 (TID 56). 6787 bytes result sent to driver +26/04/01 06:01:15 INFO TaskSetManager: Starting task 52.0 in stage 8.0 (TID 60) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:01:15 INFO TaskSetManager: Finished task 48.0 in stage 8.0 (TID 56) in 6440 ms on 10.0.0.133 (executor driver) (49/208) +26/04/01 06:01:15 INFO Executor: Running task 52.0 in stage 8.0 (TID 60) +26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:15 INFO Executor: Finished task 49.0 in stage 8.0 (TID 57). 6787 bytes result sent to driver +26/04/01 06:01:15 INFO TaskSetManager: Starting task 53.0 in stage 8.0 (TID 61) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:01:15 INFO Executor: Running task 53.0 in stage 8.0 (TID 61) +26/04/01 06:01:15 INFO TaskSetManager: Finished task 49.0 in stage 8.0 (TID 57) in 6413 ms on 10.0.0.133 (executor driver) (50/208) +26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:15 INFO Executor: Finished task 50.0 in stage 8.0 (TID 58). 6787 bytes result sent to driver +26/04/01 06:01:15 INFO TaskSetManager: Starting task 54.0 in stage 8.0 (TID 62) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:01:15 INFO TaskSetManager: Finished task 50.0 in stage 8.0 (TID 58) in 6422 ms on 10.0.0.133 (executor driver) (51/208) +26/04/01 06:01:15 INFO Executor: Running task 54.0 in stage 8.0 (TID 62) +26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:15 INFO Executor: Finished task 51.0 in stage 8.0 (TID 59). 6787 bytes result sent to driver +26/04/01 06:01:15 INFO TaskSetManager: Starting task 55.0 in stage 8.0 (TID 63) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:15 INFO Executor: Running task 55.0 in stage 8.0 (TID 63) +26/04/01 06:01:15 INFO TaskSetManager: Finished task 51.0 in stage 8.0 (TID 59) in 6444 ms on 10.0.0.133 (executor driver) (52/208) +26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:21 INFO Executor: Finished task 52.0 in stage 8.0 (TID 60). 6787 bytes result sent to driver +26/04/01 06:01:21 INFO TaskSetManager: Starting task 56.0 in stage 8.0 (TID 64) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:01:21 INFO TaskSetManager: Finished task 52.0 in stage 8.0 (TID 60) in 6472 ms on 10.0.0.133 (executor driver) (53/208) +26/04/01 06:01:21 INFO Executor: Running task 56.0 in stage 8.0 (TID 64) +26/04/01 06:01:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:22 INFO Executor: Finished task 53.0 in stage 8.0 (TID 61). 6787 bytes result sent to driver +26/04/01 06:01:22 INFO TaskSetManager: Starting task 57.0 in stage 8.0 (TID 65) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:01:22 INFO TaskSetManager: Finished task 53.0 in stage 8.0 (TID 61) in 6457 ms on 10.0.0.133 (executor driver) (54/208) +26/04/01 06:01:22 INFO Executor: Running task 57.0 in stage 8.0 (TID 65) +26/04/01 06:01:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:22 INFO Executor: Finished task 54.0 in stage 8.0 (TID 62). 6787 bytes result sent to driver +26/04/01 06:01:22 INFO TaskSetManager: Starting task 58.0 in stage 8.0 (TID 66) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:01:22 INFO TaskSetManager: Finished task 54.0 in stage 8.0 (TID 62) in 6461 ms on 10.0.0.133 (executor driver) (55/208) +26/04/01 06:01:22 INFO Executor: Running task 58.0 in stage 8.0 (TID 66) +26/04/01 06:01:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:22 INFO Executor: Finished task 55.0 in stage 8.0 (TID 63). 6787 bytes result sent to driver +26/04/01 06:01:22 INFO TaskSetManager: Starting task 59.0 in stage 8.0 (TID 67) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:01:22 INFO Executor: Running task 59.0 in stage 8.0 (TID 67) +26/04/01 06:01:22 INFO TaskSetManager: Finished task 55.0 in stage 8.0 (TID 63) in 6469 ms on 10.0.0.133 (executor driver) (56/208) +26/04/01 06:01:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:28 INFO Executor: Finished task 56.0 in stage 8.0 (TID 64). 6787 bytes result sent to driver +26/04/01 06:01:28 INFO TaskSetManager: Starting task 60.0 in stage 8.0 (TID 68) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:01:28 INFO Executor: Running task 60.0 in stage 8.0 (TID 68) +26/04/01 06:01:28 INFO TaskSetManager: Finished task 56.0 in stage 8.0 (TID 64) in 6478 ms on 10.0.0.133 (executor driver) (57/208) +26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:28 INFO Executor: Finished task 57.0 in stage 8.0 (TID 65). 6787 bytes result sent to driver +26/04/01 06:01:28 INFO TaskSetManager: Starting task 61.0 in stage 8.0 (TID 69) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:28 INFO TaskSetManager: Finished task 57.0 in stage 8.0 (TID 65) in 6477 ms on 10.0.0.133 (executor driver) (58/208) +26/04/01 06:01:28 INFO Executor: Running task 61.0 in stage 8.0 (TID 69) +26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:28 INFO Executor: Finished task 58.0 in stage 8.0 (TID 66). 6787 bytes result sent to driver +26/04/01 06:01:28 INFO TaskSetManager: Starting task 62.0 in stage 8.0 (TID 70) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:01:28 INFO Executor: Running task 62.0 in stage 8.0 (TID 70) +26/04/01 06:01:28 INFO TaskSetManager: Finished task 58.0 in stage 8.0 (TID 66) in 6458 ms on 10.0.0.133 (executor driver) (59/208) +26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:28 INFO Executor: Finished task 59.0 in stage 8.0 (TID 67). 6787 bytes result sent to driver +26/04/01 06:01:28 INFO TaskSetManager: Starting task 63.0 in stage 8.0 (TID 71) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:01:28 INFO TaskSetManager: Finished task 59.0 in stage 8.0 (TID 67) in 6477 ms on 10.0.0.133 (executor driver) (60/208) +26/04/01 06:01:28 INFO Executor: Running task 63.0 in stage 8.0 (TID 71) +26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:34 INFO Executor: Finished task 60.0 in stage 8.0 (TID 68). 6787 bytes result sent to driver +26/04/01 06:01:34 INFO TaskSetManager: Starting task 64.0 in stage 8.0 (TID 72) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:01:34 INFO TaskSetManager: Finished task 60.0 in stage 8.0 (TID 68) in 6400 ms on 10.0.0.133 (executor driver) (61/208) +26/04/01 06:01:34 INFO Executor: Running task 64.0 in stage 8.0 (TID 72) +26/04/01 06:01:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:35 INFO Executor: Finished task 61.0 in stage 8.0 (TID 69). 6787 bytes result sent to driver +26/04/01 06:01:35 INFO TaskSetManager: Starting task 65.0 in stage 8.0 (TID 73) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:01:35 INFO Executor: Running task 65.0 in stage 8.0 (TID 73) +26/04/01 06:01:35 INFO TaskSetManager: Finished task 61.0 in stage 8.0 (TID 69) in 6392 ms on 10.0.0.133 (executor driver) (62/208) +26/04/01 06:01:35 INFO Executor: Finished task 62.0 in stage 8.0 (TID 70). 6787 bytes result sent to driver +26/04/01 06:01:35 INFO TaskSetManager: Starting task 66.0 in stage 8.0 (TID 74) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:01:35 INFO Executor: Running task 66.0 in stage 8.0 (TID 74) +26/04/01 06:01:35 INFO TaskSetManager: Finished task 62.0 in stage 8.0 (TID 70) in 6391 ms on 10.0.0.133 (executor driver) (63/208) +26/04/01 06:01:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:35 INFO Executor: Finished task 63.0 in stage 8.0 (TID 71). 6787 bytes result sent to driver +26/04/01 06:01:35 INFO TaskSetManager: Starting task 67.0 in stage 8.0 (TID 75) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:01:35 INFO TaskSetManager: Finished task 63.0 in stage 8.0 (TID 71) in 6392 ms on 10.0.0.133 (executor driver) (64/208) +26/04/01 06:01:35 INFO Executor: Running task 67.0 in stage 8.0 (TID 75) +26/04/01 06:01:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:41 INFO Executor: Finished task 64.0 in stage 8.0 (TID 72). 6787 bytes result sent to driver +26/04/01 06:01:41 INFO TaskSetManager: Starting task 68.0 in stage 8.0 (TID 76) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:41 INFO Executor: Running task 68.0 in stage 8.0 (TID 76) +26/04/01 06:01:41 INFO TaskSetManager: Finished task 64.0 in stage 8.0 (TID 72) in 6387 ms on 10.0.0.133 (executor driver) (65/208) +26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:41 INFO Executor: Finished task 65.0 in stage 8.0 (TID 73). 6787 bytes result sent to driver +26/04/01 06:01:41 INFO Executor: Finished task 66.0 in stage 8.0 (TID 74). 6787 bytes result sent to driver +26/04/01 06:01:41 INFO TaskSetManager: Starting task 69.0 in stage 8.0 (TID 77) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:41 INFO Executor: Running task 69.0 in stage 8.0 (TID 77) +26/04/01 06:01:41 INFO TaskSetManager: Starting task 70.0 in stage 8.0 (TID 78) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:41 INFO TaskSetManager: Finished task 65.0 in stage 8.0 (TID 73) in 6410 ms on 10.0.0.133 (executor driver) (66/208) +26/04/01 06:01:41 INFO Executor: Running task 70.0 in stage 8.0 (TID 78) +26/04/01 06:01:41 INFO TaskSetManager: Finished task 66.0 in stage 8.0 (TID 74) in 6411 ms on 10.0.0.133 (executor driver) (67/208) +26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:41 INFO Executor: Finished task 67.0 in stage 8.0 (TID 75). 6787 bytes result sent to driver +26/04/01 06:01:41 INFO TaskSetManager: Starting task 71.0 in stage 8.0 (TID 79) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:41 INFO Executor: Running task 71.0 in stage 8.0 (TID 79) +26/04/01 06:01:41 INFO TaskSetManager: Finished task 67.0 in stage 8.0 (TID 75) in 6389 ms on 10.0.0.133 (executor driver) (68/208) +26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:47 INFO Executor: Finished task 68.0 in stage 8.0 (TID 76). 6787 bytes result sent to driver +26/04/01 06:01:47 INFO TaskSetManager: Starting task 72.0 in stage 8.0 (TID 80) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:01:47 INFO Executor: Running task 72.0 in stage 8.0 (TID 80) +26/04/01 06:01:47 INFO TaskSetManager: Finished task 68.0 in stage 8.0 (TID 76) in 6396 ms on 10.0.0.133 (executor driver) (69/208) +26/04/01 06:01:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:47 INFO Executor: Finished task 69.0 in stage 8.0 (TID 77). 6787 bytes result sent to driver +26/04/01 06:01:47 INFO TaskSetManager: Starting task 73.0 in stage 8.0 (TID 81) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:01:47 INFO TaskSetManager: Finished task 69.0 in stage 8.0 (TID 77) in 6392 ms on 10.0.0.133 (executor driver) (70/208) +26/04/01 06:01:47 INFO Executor: Running task 73.0 in stage 8.0 (TID 81) +26/04/01 06:01:47 INFO Executor: Finished task 70.0 in stage 8.0 (TID 78). 6787 bytes result sent to driver +26/04/01 06:01:47 INFO TaskSetManager: Starting task 74.0 in stage 8.0 (TID 82) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:47 INFO Executor: Running task 74.0 in stage 8.0 (TID 82) +26/04/01 06:01:47 INFO TaskSetManager: Finished task 70.0 in stage 8.0 (TID 78) in 6392 ms on 10.0.0.133 (executor driver) (71/208) +26/04/01 06:01:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:48 INFO Executor: Finished task 71.0 in stage 8.0 (TID 79). 6830 bytes result sent to driver +26/04/01 06:01:48 INFO TaskSetManager: Starting task 75.0 in stage 8.0 (TID 83) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:48 INFO TaskSetManager: Finished task 71.0 in stage 8.0 (TID 79) in 6390 ms on 10.0.0.133 (executor driver) (72/208) +26/04/01 06:01:48 INFO Executor: Running task 75.0 in stage 8.0 (TID 83) +26/04/01 06:01:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:54 INFO Executor: Finished task 72.0 in stage 8.0 (TID 80). 6787 bytes result sent to driver +26/04/01 06:01:54 INFO TaskSetManager: Starting task 76.0 in stage 8.0 (TID 84) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:54 INFO Executor: Running task 76.0 in stage 8.0 (TID 84) +26/04/01 06:01:54 INFO TaskSetManager: Finished task 72.0 in stage 8.0 (TID 80) in 6449 ms on 10.0.0.133 (executor driver) (73/208) +26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:54 INFO Executor: Finished task 73.0 in stage 8.0 (TID 81). 6787 bytes result sent to driver +26/04/01 06:01:54 INFO TaskSetManager: Starting task 77.0 in stage 8.0 (TID 85) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:54 INFO TaskSetManager: Finished task 73.0 in stage 8.0 (TID 81) in 6437 ms on 10.0.0.133 (executor driver) (74/208) +26/04/01 06:01:54 INFO Executor: Running task 77.0 in stage 8.0 (TID 85) +26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:54 INFO Executor: Finished task 74.0 in stage 8.0 (TID 82). 6787 bytes result sent to driver +26/04/01 06:01:54 INFO TaskSetManager: Starting task 78.0 in stage 8.0 (TID 86) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:01:54 INFO TaskSetManager: Finished task 74.0 in stage 8.0 (TID 82) in 6446 ms on 10.0.0.133 (executor driver) (75/208) +26/04/01 06:01:54 INFO Executor: Running task 78.0 in stage 8.0 (TID 86) +26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:54 INFO Executor: Finished task 75.0 in stage 8.0 (TID 83). 6787 bytes result sent to driver +26/04/01 06:01:54 INFO TaskSetManager: Starting task 79.0 in stage 8.0 (TID 87) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:01:54 INFO TaskSetManager: Finished task 75.0 in stage 8.0 (TID 83) in 6464 ms on 10.0.0.133 (executor driver) (76/208) +26/04/01 06:01:54 INFO Executor: Running task 79.0 in stage 8.0 (TID 87) +26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:00 INFO Executor: Finished task 76.0 in stage 8.0 (TID 84). 6787 bytes result sent to driver +26/04/01 06:02:00 INFO TaskSetManager: Starting task 80.0 in stage 8.0 (TID 88) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:00 INFO TaskSetManager: Finished task 76.0 in stage 8.0 (TID 84) in 6447 ms on 10.0.0.133 (executor driver) (77/208) +26/04/01 06:02:00 INFO Executor: Running task 80.0 in stage 8.0 (TID 88) +26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:00 INFO Executor: Finished task 77.0 in stage 8.0 (TID 85). 6787 bytes result sent to driver +26/04/01 06:02:00 INFO TaskSetManager: Starting task 81.0 in stage 8.0 (TID 89) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:00 INFO TaskSetManager: Finished task 77.0 in stage 8.0 (TID 85) in 6447 ms on 10.0.0.133 (executor driver) (78/208) +26/04/01 06:02:00 INFO Executor: Running task 81.0 in stage 8.0 (TID 89) +26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:00 INFO Executor: Finished task 78.0 in stage 8.0 (TID 86). 6787 bytes result sent to driver +26/04/01 06:02:00 INFO TaskSetManager: Starting task 82.0 in stage 8.0 (TID 90) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:00 INFO Executor: Running task 82.0 in stage 8.0 (TID 90) +26/04/01 06:02:00 INFO TaskSetManager: Finished task 78.0 in stage 8.0 (TID 86) in 6440 ms on 10.0.0.133 (executor driver) (79/208) +26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:00 INFO Executor: Finished task 79.0 in stage 8.0 (TID 87). 6787 bytes result sent to driver +26/04/01 06:02:00 INFO TaskSetManager: Starting task 83.0 in stage 8.0 (TID 91) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:00 INFO TaskSetManager: Finished task 79.0 in stage 8.0 (TID 87) in 6444 ms on 10.0.0.133 (executor driver) (80/208) +26/04/01 06:02:00 INFO Executor: Running task 83.0 in stage 8.0 (TID 91) +26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:06 INFO Executor: Finished task 80.0 in stage 8.0 (TID 88). 6787 bytes result sent to driver +26/04/01 06:02:06 INFO TaskSetManager: Starting task 84.0 in stage 8.0 (TID 92) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:02:06 INFO TaskSetManager: Finished task 80.0 in stage 8.0 (TID 88) in 6451 ms on 10.0.0.133 (executor driver) (81/208) +26/04/01 06:02:06 INFO Executor: Running task 84.0 in stage 8.0 (TID 92) +26/04/01 06:02:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:07 INFO Executor: Finished task 81.0 in stage 8.0 (TID 89). 6787 bytes result sent to driver +26/04/01 06:02:07 INFO Executor: Finished task 82.0 in stage 8.0 (TID 90). 6787 bytes result sent to driver +26/04/01 06:02:07 INFO TaskSetManager: Starting task 85.0 in stage 8.0 (TID 93) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:02:07 INFO Executor: Running task 85.0 in stage 8.0 (TID 93) +26/04/01 06:02:07 INFO TaskSetManager: Starting task 86.0 in stage 8.0 (TID 94) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:07 INFO TaskSetManager: Finished task 81.0 in stage 8.0 (TID 89) in 6451 ms on 10.0.0.133 (executor driver) (82/208) +26/04/01 06:02:07 INFO Executor: Running task 86.0 in stage 8.0 (TID 94) +26/04/01 06:02:07 INFO TaskSetManager: Finished task 82.0 in stage 8.0 (TID 90) in 6447 ms on 10.0.0.133 (executor driver) (83/208) +26/04/01 06:02:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:07 INFO Executor: Finished task 83.0 in stage 8.0 (TID 91). 6787 bytes result sent to driver +26/04/01 06:02:07 INFO TaskSetManager: Starting task 87.0 in stage 8.0 (TID 95) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:07 INFO TaskSetManager: Finished task 83.0 in stage 8.0 (TID 91) in 6452 ms on 10.0.0.133 (executor driver) (84/208) +26/04/01 06:02:07 INFO Executor: Running task 87.0 in stage 8.0 (TID 95) +26/04/01 06:02:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:13 INFO Executor: Finished task 84.0 in stage 8.0 (TID 92). 6787 bytes result sent to driver +26/04/01 06:02:13 INFO TaskSetManager: Starting task 88.0 in stage 8.0 (TID 96) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:13 INFO TaskSetManager: Finished task 84.0 in stage 8.0 (TID 92) in 6455 ms on 10.0.0.133 (executor driver) (85/208) +26/04/01 06:02:13 INFO Executor: Running task 88.0 in stage 8.0 (TID 96) +26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:13 INFO Executor: Finished task 86.0 in stage 8.0 (TID 94). 6787 bytes result sent to driver +26/04/01 06:02:13 INFO TaskSetManager: Starting task 89.0 in stage 8.0 (TID 97) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:13 INFO Executor: Running task 89.0 in stage 8.0 (TID 97) +26/04/01 06:02:13 INFO TaskSetManager: Finished task 86.0 in stage 8.0 (TID 94) in 6453 ms on 10.0.0.133 (executor driver) (86/208) +26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:13 INFO Executor: Finished task 85.0 in stage 8.0 (TID 93). 6787 bytes result sent to driver +26/04/01 06:02:13 INFO TaskSetManager: Starting task 90.0 in stage 8.0 (TID 98) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:02:13 INFO TaskSetManager: Finished task 85.0 in stage 8.0 (TID 93) in 6460 ms on 10.0.0.133 (executor driver) (87/208) +26/04/01 06:02:13 INFO Executor: Running task 90.0 in stage 8.0 (TID 98) +26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:13 INFO Executor: Finished task 87.0 in stage 8.0 (TID 95). 6787 bytes result sent to driver +26/04/01 06:02:13 INFO TaskSetManager: Starting task 91.0 in stage 8.0 (TID 99) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:02:13 INFO Executor: Running task 91.0 in stage 8.0 (TID 99) +26/04/01 06:02:13 INFO TaskSetManager: Finished task 87.0 in stage 8.0 (TID 95) in 6456 ms on 10.0.0.133 (executor driver) (88/208) +26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:19 INFO Executor: Finished task 88.0 in stage 8.0 (TID 96). 6787 bytes result sent to driver +26/04/01 06:02:19 INFO TaskSetManager: Starting task 92.0 in stage 8.0 (TID 100) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:19 INFO TaskSetManager: Finished task 88.0 in stage 8.0 (TID 96) in 6448 ms on 10.0.0.133 (executor driver) (89/208) +26/04/01 06:02:19 INFO Executor: Running task 92.0 in stage 8.0 (TID 100) +26/04/01 06:02:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:20 INFO Executor: Finished task 91.0 in stage 8.0 (TID 99). 6787 bytes result sent to driver +26/04/01 06:02:20 INFO TaskSetManager: Starting task 93.0 in stage 8.0 (TID 101) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:20 INFO TaskSetManager: Finished task 91.0 in stage 8.0 (TID 99) in 6190 ms on 10.0.0.133 (executor driver) (90/208) +26/04/01 06:02:20 INFO Executor: Running task 93.0 in stage 8.0 (TID 101) +26/04/01 06:02:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:20 INFO Executor: Finished task 89.0 in stage 8.0 (TID 97). 6787 bytes result sent to driver +26/04/01 06:02:20 INFO TaskSetManager: Starting task 94.0 in stage 8.0 (TID 102) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:20 INFO TaskSetManager: Finished task 89.0 in stage 8.0 (TID 97) in 6446 ms on 10.0.0.133 (executor driver) (91/208) +26/04/01 06:02:20 INFO Executor: Running task 94.0 in stage 8.0 (TID 102) +26/04/01 06:02:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:20 INFO Executor: Finished task 90.0 in stage 8.0 (TID 98). 6787 bytes result sent to driver +26/04/01 06:02:20 INFO TaskSetManager: Starting task 95.0 in stage 8.0 (TID 103) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:20 INFO Executor: Running task 95.0 in stage 8.0 (TID 103) +26/04/01 06:02:20 INFO TaskSetManager: Finished task 90.0 in stage 8.0 (TID 98) in 6689 ms on 10.0.0.133 (executor driver) (92/208) +26/04/01 06:02:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:26 INFO Executor: Finished task 92.0 in stage 8.0 (TID 100). 6787 bytes result sent to driver +26/04/01 06:02:26 INFO TaskSetManager: Starting task 96.0 in stage 8.0 (TID 104) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:02:26 INFO TaskSetManager: Finished task 92.0 in stage 8.0 (TID 100) in 6427 ms on 10.0.0.133 (executor driver) (93/208) +26/04/01 06:02:26 INFO Executor: Running task 96.0 in stage 8.0 (TID 104) +26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:26 INFO Executor: Finished task 93.0 in stage 8.0 (TID 101). 6787 bytes result sent to driver +26/04/01 06:02:26 INFO TaskSetManager: Starting task 97.0 in stage 8.0 (TID 105) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:02:26 INFO TaskSetManager: Finished task 93.0 in stage 8.0 (TID 101) in 6430 ms on 10.0.0.133 (executor driver) (94/208) +26/04/01 06:02:26 INFO Executor: Running task 97.0 in stage 8.0 (TID 105) +26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:26 INFO Executor: Finished task 94.0 in stage 8.0 (TID 102). 6787 bytes result sent to driver +26/04/01 06:02:26 INFO TaskSetManager: Starting task 98.0 in stage 8.0 (TID 106) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:02:26 INFO Executor: Running task 98.0 in stage 8.0 (TID 106) +26/04/01 06:02:26 INFO TaskSetManager: Finished task 94.0 in stage 8.0 (TID 102) in 6433 ms on 10.0.0.133 (executor driver) (95/208) +26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:26 INFO Executor: Finished task 95.0 in stage 8.0 (TID 103). 6787 bytes result sent to driver +26/04/01 06:02:26 INFO TaskSetManager: Starting task 99.0 in stage 8.0 (TID 107) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:02:26 INFO TaskSetManager: Finished task 95.0 in stage 8.0 (TID 103) in 6428 ms on 10.0.0.133 (executor driver) (96/208) +26/04/01 06:02:26 INFO Executor: Running task 99.0 in stage 8.0 (TID 107) +26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:32 INFO Executor: Finished task 96.0 in stage 8.0 (TID 104). 6787 bytes result sent to driver +26/04/01 06:02:32 INFO TaskSetManager: Starting task 100.0 in stage 8.0 (TID 108) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:02:32 INFO TaskSetManager: Finished task 96.0 in stage 8.0 (TID 104) in 6449 ms on 10.0.0.133 (executor driver) (97/208) +26/04/01 06:02:32 INFO Executor: Running task 100.0 in stage 8.0 (TID 108) +26/04/01 06:02:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:32 INFO Executor: Finished task 97.0 in stage 8.0 (TID 105). 6787 bytes result sent to driver +26/04/01 06:02:32 INFO TaskSetManager: Starting task 101.0 in stage 8.0 (TID 109) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:02:32 INFO TaskSetManager: Finished task 97.0 in stage 8.0 (TID 105) in 6449 ms on 10.0.0.133 (executor driver) (98/208) +26/04/01 06:02:32 INFO Executor: Running task 101.0 in stage 8.0 (TID 109) +26/04/01 06:02:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:32 INFO Executor: Finished task 98.0 in stage 8.0 (TID 106). 6787 bytes result sent to driver +26/04/01 06:02:32 INFO TaskSetManager: Starting task 102.0 in stage 8.0 (TID 110) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:02:32 INFO TaskSetManager: Finished task 98.0 in stage 8.0 (TID 106) in 6446 ms on 10.0.0.133 (executor driver) (99/208) +26/04/01 06:02:32 INFO Executor: Running task 102.0 in stage 8.0 (TID 110) +26/04/01 06:02:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:33 INFO Executor: Finished task 99.0 in stage 8.0 (TID 107). 6787 bytes result sent to driver +26/04/01 06:02:33 INFO TaskSetManager: Starting task 103.0 in stage 8.0 (TID 111) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:02:33 INFO Executor: Running task 103.0 in stage 8.0 (TID 111) +26/04/01 06:02:33 INFO TaskSetManager: Finished task 99.0 in stage 8.0 (TID 107) in 6450 ms on 10.0.0.133 (executor driver) (100/208) +26/04/01 06:02:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:39 INFO Executor: Finished task 100.0 in stage 8.0 (TID 108). 6787 bytes result sent to driver +26/04/01 06:02:39 INFO TaskSetManager: Starting task 104.0 in stage 8.0 (TID 112) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:39 INFO TaskSetManager: Finished task 100.0 in stage 8.0 (TID 108) in 6450 ms on 10.0.0.133 (executor driver) (101/208) +26/04/01 06:02:39 INFO Executor: Running task 104.0 in stage 8.0 (TID 112) +26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:39 INFO Executor: Finished task 102.0 in stage 8.0 (TID 110). 6787 bytes result sent to driver +26/04/01 06:02:39 INFO Executor: Finished task 101.0 in stage 8.0 (TID 109). 6787 bytes result sent to driver +26/04/01 06:02:39 INFO TaskSetManager: Starting task 105.0 in stage 8.0 (TID 113) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:39 INFO TaskSetManager: Starting task 106.0 in stage 8.0 (TID 114) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:39 INFO Executor: Running task 105.0 in stage 8.0 (TID 113) +26/04/01 06:02:39 INFO TaskSetManager: Finished task 102.0 in stage 8.0 (TID 110) in 6451 ms on 10.0.0.133 (executor driver) (102/208) +26/04/01 06:02:39 INFO Executor: Running task 106.0 in stage 8.0 (TID 114) +26/04/01 06:02:39 INFO TaskSetManager: Finished task 101.0 in stage 8.0 (TID 109) in 6454 ms on 10.0.0.133 (executor driver) (103/208) +26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:39 INFO Executor: Finished task 103.0 in stage 8.0 (TID 111). 6787 bytes result sent to driver +26/04/01 06:02:39 INFO TaskSetManager: Starting task 107.0 in stage 8.0 (TID 115) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:39 INFO TaskSetManager: Finished task 103.0 in stage 8.0 (TID 111) in 6454 ms on 10.0.0.133 (executor driver) (104/208) +26/04/01 06:02:39 INFO Executor: Running task 107.0 in stage 8.0 (TID 115) +26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:45 INFO Executor: Finished task 104.0 in stage 8.0 (TID 112). 6787 bytes result sent to driver +26/04/01 06:02:45 INFO TaskSetManager: Starting task 108.0 in stage 8.0 (TID 116) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:02:45 INFO TaskSetManager: Finished task 104.0 in stage 8.0 (TID 112) in 6464 ms on 10.0.0.133 (executor driver) (105/208) +26/04/01 06:02:45 INFO Executor: Running task 108.0 in stage 8.0 (TID 116) +26/04/01 06:02:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:45 INFO Executor: Finished task 105.0 in stage 8.0 (TID 113). 6787 bytes result sent to driver +26/04/01 06:02:45 INFO Executor: Finished task 106.0 in stage 8.0 (TID 114). 6787 bytes result sent to driver +26/04/01 06:02:45 INFO TaskSetManager: Starting task 109.0 in stage 8.0 (TID 117) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:02:45 INFO TaskSetManager: Starting task 110.0 in stage 8.0 (TID 118) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:02:45 INFO Executor: Running task 109.0 in stage 8.0 (TID 117) +26/04/01 06:02:45 INFO TaskSetManager: Finished task 105.0 in stage 8.0 (TID 113) in 6467 ms on 10.0.0.133 (executor driver) (106/208) +26/04/01 06:02:45 INFO Executor: Running task 110.0 in stage 8.0 (TID 118) +26/04/01 06:02:45 INFO TaskSetManager: Finished task 106.0 in stage 8.0 (TID 114) in 6466 ms on 10.0.0.133 (executor driver) (107/208) +26/04/01 06:02:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:46 INFO Executor: Finished task 107.0 in stage 8.0 (TID 115). 6787 bytes result sent to driver +26/04/01 06:02:46 INFO TaskSetManager: Starting task 111.0 in stage 8.0 (TID 119) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:02:46 INFO TaskSetManager: Finished task 107.0 in stage 8.0 (TID 115) in 6461 ms on 10.0.0.133 (executor driver) (108/208) +26/04/01 06:02:46 INFO Executor: Running task 111.0 in stage 8.0 (TID 119) +26/04/01 06:02:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:52 INFO Executor: Finished task 108.0 in stage 8.0 (TID 116). 6787 bytes result sent to driver +26/04/01 06:02:52 INFO TaskSetManager: Starting task 112.0 in stage 8.0 (TID 120) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:02:52 INFO Executor: Running task 112.0 in stage 8.0 (TID 120) +26/04/01 06:02:52 INFO TaskSetManager: Finished task 108.0 in stage 8.0 (TID 116) in 6471 ms on 10.0.0.133 (executor driver) (109/208) +26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:52 INFO Executor: Finished task 109.0 in stage 8.0 (TID 117). 6787 bytes result sent to driver +26/04/01 06:02:52 INFO TaskSetManager: Starting task 113.0 in stage 8.0 (TID 121) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:02:52 INFO Executor: Running task 113.0 in stage 8.0 (TID 121) +26/04/01 06:02:52 INFO TaskSetManager: Finished task 109.0 in stage 8.0 (TID 117) in 6464 ms on 10.0.0.133 (executor driver) (110/208) +26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:52 INFO Executor: Finished task 110.0 in stage 8.0 (TID 118). 6787 bytes result sent to driver +26/04/01 06:02:52 INFO TaskSetManager: Starting task 114.0 in stage 8.0 (TID 122) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:02:52 INFO TaskSetManager: Finished task 110.0 in stage 8.0 (TID 118) in 6468 ms on 10.0.0.133 (executor driver) (111/208) +26/04/01 06:02:52 INFO Executor: Running task 114.0 in stage 8.0 (TID 122) +26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:52 INFO Executor: Finished task 111.0 in stage 8.0 (TID 119). 6787 bytes result sent to driver +26/04/01 06:02:52 INFO TaskSetManager: Starting task 115.0 in stage 8.0 (TID 123) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:02:52 INFO TaskSetManager: Finished task 111.0 in stage 8.0 (TID 119) in 6473 ms on 10.0.0.133 (executor driver) (112/208) +26/04/01 06:02:52 INFO Executor: Running task 115.0 in stage 8.0 (TID 123) +26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:58 INFO Executor: Finished task 112.0 in stage 8.0 (TID 120). 6787 bytes result sent to driver +26/04/01 06:02:58 INFO TaskSetManager: Starting task 116.0 in stage 8.0 (TID 124) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:58 INFO TaskSetManager: Finished task 112.0 in stage 8.0 (TID 120) in 6456 ms on 10.0.0.133 (executor driver) (113/208) +26/04/01 06:02:58 INFO Executor: Running task 116.0 in stage 8.0 (TID 124) +26/04/01 06:02:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:58 INFO Executor: Finished task 113.0 in stage 8.0 (TID 121). 6787 bytes result sent to driver +26/04/01 06:02:58 INFO TaskSetManager: Starting task 117.0 in stage 8.0 (TID 125) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:58 INFO Executor: Running task 117.0 in stage 8.0 (TID 125) +26/04/01 06:02:58 INFO TaskSetManager: Finished task 113.0 in stage 8.0 (TID 121) in 6457 ms on 10.0.0.133 (executor driver) (114/208) +26/04/01 06:02:58 INFO Executor: Finished task 114.0 in stage 8.0 (TID 122). 6787 bytes result sent to driver +26/04/01 06:02:58 INFO TaskSetManager: Starting task 118.0 in stage 8.0 (TID 126) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:58 INFO TaskSetManager: Finished task 114.0 in stage 8.0 (TID 122) in 6454 ms on 10.0.0.133 (executor driver) (115/208) +26/04/01 06:02:58 INFO Executor: Running task 118.0 in stage 8.0 (TID 126) +26/04/01 06:02:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:59 INFO Executor: Finished task 115.0 in stage 8.0 (TID 123). 6830 bytes result sent to driver +26/04/01 06:02:59 INFO TaskSetManager: Starting task 119.0 in stage 8.0 (TID 127) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:02:59 INFO TaskSetManager: Finished task 115.0 in stage 8.0 (TID 123) in 6447 ms on 10.0.0.133 (executor driver) (116/208) +26/04/01 06:02:59 INFO Executor: Running task 119.0 in stage 8.0 (TID 127) +26/04/01 06:02:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:02:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:05 INFO Executor: Finished task 116.0 in stage 8.0 (TID 124). 6830 bytes result sent to driver +26/04/01 06:03:05 INFO TaskSetManager: Starting task 120.0 in stage 8.0 (TID 128) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:03:05 INFO TaskSetManager: Finished task 116.0 in stage 8.0 (TID 124) in 6456 ms on 10.0.0.133 (executor driver) (117/208) +26/04/01 06:03:05 INFO Executor: Running task 120.0 in stage 8.0 (TID 128) +26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:05 INFO Executor: Finished task 118.0 in stage 8.0 (TID 126). 6873 bytes result sent to driver +26/04/01 06:03:05 INFO TaskSetManager: Starting task 121.0 in stage 8.0 (TID 129) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:03:05 INFO Executor: Running task 121.0 in stage 8.0 (TID 129) +26/04/01 06:03:05 INFO TaskSetManager: Finished task 118.0 in stage 8.0 (TID 126) in 6462 ms on 10.0.0.133 (executor driver) (118/208) +26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:05 INFO Executor: Finished task 117.0 in stage 8.0 (TID 125). 6830 bytes result sent to driver +26/04/01 06:03:05 INFO TaskSetManager: Starting task 122.0 in stage 8.0 (TID 130) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:05 INFO Executor: Running task 122.0 in stage 8.0 (TID 130) +26/04/01 06:03:05 INFO TaskSetManager: Finished task 117.0 in stage 8.0 (TID 125) in 6466 ms on 10.0.0.133 (executor driver) (119/208) +26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:05 INFO Executor: Finished task 119.0 in stage 8.0 (TID 127). 6787 bytes result sent to driver +26/04/01 06:03:05 INFO TaskSetManager: Starting task 123.0 in stage 8.0 (TID 131) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:05 INFO Executor: Running task 123.0 in stage 8.0 (TID 131) +26/04/01 06:03:05 INFO TaskSetManager: Finished task 119.0 in stage 8.0 (TID 127) in 6469 ms on 10.0.0.133 (executor driver) (120/208) +26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:11 INFO Executor: Finished task 120.0 in stage 8.0 (TID 128). 6787 bytes result sent to driver +26/04/01 06:03:11 INFO TaskSetManager: Starting task 124.0 in stage 8.0 (TID 132) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:11 INFO TaskSetManager: Finished task 120.0 in stage 8.0 (TID 128) in 6474 ms on 10.0.0.133 (executor driver) (121/208) +26/04/01 06:03:11 INFO Executor: Running task 124.0 in stage 8.0 (TID 132) +26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:11 INFO Executor: Finished task 122.0 in stage 8.0 (TID 130). 6787 bytes result sent to driver +26/04/01 06:03:11 INFO TaskSetManager: Starting task 125.0 in stage 8.0 (TID 133) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:11 INFO TaskSetManager: Finished task 122.0 in stage 8.0 (TID 130) in 6453 ms on 10.0.0.133 (executor driver) (122/208) +26/04/01 06:03:11 INFO Executor: Running task 125.0 in stage 8.0 (TID 133) +26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:11 INFO Executor: Finished task 121.0 in stage 8.0 (TID 129). 6787 bytes result sent to driver +26/04/01 06:03:11 INFO TaskSetManager: Starting task 126.0 in stage 8.0 (TID 134) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:03:11 INFO TaskSetManager: Finished task 121.0 in stage 8.0 (TID 129) in 6466 ms on 10.0.0.133 (executor driver) (123/208) +26/04/01 06:03:11 INFO Executor: Running task 126.0 in stage 8.0 (TID 134) +26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:11 INFO Executor: Finished task 123.0 in stage 8.0 (TID 131). 6787 bytes result sent to driver +26/04/01 06:03:11 INFO TaskSetManager: Starting task 127.0 in stage 8.0 (TID 135) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:03:11 INFO TaskSetManager: Finished task 123.0 in stage 8.0 (TID 131) in 6453 ms on 10.0.0.133 (executor driver) (124/208) +26/04/01 06:03:11 INFO Executor: Running task 127.0 in stage 8.0 (TID 135) +26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:17 INFO Executor: Finished task 124.0 in stage 8.0 (TID 132). 6787 bytes result sent to driver +26/04/01 06:03:17 INFO TaskSetManager: Starting task 128.0 in stage 8.0 (TID 136) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:17 INFO Executor: Running task 128.0 in stage 8.0 (TID 136) +26/04/01 06:03:17 INFO TaskSetManager: Finished task 124.0 in stage 8.0 (TID 132) in 6445 ms on 10.0.0.133 (executor driver) (125/208) +26/04/01 06:03:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:18 INFO Executor: Finished task 126.0 in stage 8.0 (TID 134). 6787 bytes result sent to driver +26/04/01 06:03:18 INFO Executor: Finished task 125.0 in stage 8.0 (TID 133). 6787 bytes result sent to driver +26/04/01 06:03:18 INFO TaskSetManager: Starting task 129.0 in stage 8.0 (TID 137) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:18 INFO TaskSetManager: Starting task 130.0 in stage 8.0 (TID 138) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:18 INFO Executor: Running task 129.0 in stage 8.0 (TID 137) +26/04/01 06:03:18 INFO TaskSetManager: Finished task 126.0 in stage 8.0 (TID 134) in 6448 ms on 10.0.0.133 (executor driver) (126/208) +26/04/01 06:03:18 INFO Executor: Running task 130.0 in stage 8.0 (TID 138) +26/04/01 06:03:18 INFO TaskSetManager: Finished task 125.0 in stage 8.0 (TID 133) in 6458 ms on 10.0.0.133 (executor driver) (127/208) +26/04/01 06:03:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:18 INFO Executor: Finished task 127.0 in stage 8.0 (TID 135). 6787 bytes result sent to driver +26/04/01 06:03:18 INFO TaskSetManager: Starting task 131.0 in stage 8.0 (TID 139) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:18 INFO TaskSetManager: Finished task 127.0 in stage 8.0 (TID 135) in 6458 ms on 10.0.0.133 (executor driver) (128/208) +26/04/01 06:03:18 INFO Executor: Running task 131.0 in stage 8.0 (TID 139) +26/04/01 06:03:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:24 INFO Executor: Finished task 128.0 in stage 8.0 (TID 136). 6787 bytes result sent to driver +26/04/01 06:03:24 INFO TaskSetManager: Starting task 132.0 in stage 8.0 (TID 140) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:03:24 INFO TaskSetManager: Finished task 128.0 in stage 8.0 (TID 136) in 6468 ms on 10.0.0.133 (executor driver) (129/208) +26/04/01 06:03:24 INFO Executor: Running task 132.0 in stage 8.0 (TID 140) +26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:24 INFO Executor: Finished task 130.0 in stage 8.0 (TID 138). 6787 bytes result sent to driver +26/04/01 06:03:24 INFO Executor: Finished task 129.0 in stage 8.0 (TID 137). 6787 bytes result sent to driver +26/04/01 06:03:24 INFO TaskSetManager: Starting task 133.0 in stage 8.0 (TID 141) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:03:24 INFO Executor: Running task 133.0 in stage 8.0 (TID 141) +26/04/01 06:03:24 INFO TaskSetManager: Starting task 134.0 in stage 8.0 (TID 142) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:24 INFO TaskSetManager: Finished task 130.0 in stage 8.0 (TID 138) in 6469 ms on 10.0.0.133 (executor driver) (130/208) +26/04/01 06:03:24 INFO Executor: Running task 134.0 in stage 8.0 (TID 142) +26/04/01 06:03:24 INFO TaskSetManager: Finished task 129.0 in stage 8.0 (TID 137) in 6471 ms on 10.0.0.133 (executor driver) (131/208) +26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:24 INFO Executor: Finished task 131.0 in stage 8.0 (TID 139). 6787 bytes result sent to driver +26/04/01 06:03:24 INFO TaskSetManager: Starting task 135.0 in stage 8.0 (TID 143) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:24 INFO TaskSetManager: Finished task 131.0 in stage 8.0 (TID 139) in 6465 ms on 10.0.0.133 (executor driver) (132/208) +26/04/01 06:03:24 INFO Executor: Running task 135.0 in stage 8.0 (TID 143) +26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:30 INFO Executor: Finished task 132.0 in stage 8.0 (TID 140). 6787 bytes result sent to driver +26/04/01 06:03:30 INFO TaskSetManager: Starting task 136.0 in stage 8.0 (TID 144) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:30 INFO Executor: Running task 136.0 in stage 8.0 (TID 144) +26/04/01 06:03:30 INFO TaskSetManager: Finished task 132.0 in stage 8.0 (TID 140) in 6406 ms on 10.0.0.133 (executor driver) (133/208) +26/04/01 06:03:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:31 INFO Executor: Finished task 134.0 in stage 8.0 (TID 142). 6787 bytes result sent to driver +26/04/01 06:03:31 INFO Executor: Finished task 133.0 in stage 8.0 (TID 141). 6787 bytes result sent to driver +26/04/01 06:03:31 INFO TaskSetManager: Starting task 137.0 in stage 8.0 (TID 145) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:31 INFO TaskSetManager: Finished task 134.0 in stage 8.0 (TID 142) in 6396 ms on 10.0.0.133 (executor driver) (134/208) +26/04/01 06:03:31 INFO Executor: Running task 137.0 in stage 8.0 (TID 145) +26/04/01 06:03:31 INFO TaskSetManager: Starting task 138.0 in stage 8.0 (TID 146) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:03:31 INFO TaskSetManager: Finished task 133.0 in stage 8.0 (TID 141) in 6396 ms on 10.0.0.133 (executor driver) (135/208) +26/04/01 06:03:31 INFO Executor: Running task 138.0 in stage 8.0 (TID 146) +26/04/01 06:03:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:31 INFO Executor: Finished task 135.0 in stage 8.0 (TID 143). 6787 bytes result sent to driver +26/04/01 06:03:31 INFO TaskSetManager: Starting task 139.0 in stage 8.0 (TID 147) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:03:31 INFO TaskSetManager: Finished task 135.0 in stage 8.0 (TID 143) in 6403 ms on 10.0.0.133 (executor driver) (136/208) +26/04/01 06:03:31 INFO Executor: Running task 139.0 in stage 8.0 (TID 147) +26/04/01 06:03:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:37 INFO Executor: Finished task 136.0 in stage 8.0 (TID 144). 6787 bytes result sent to driver +26/04/01 06:03:37 INFO TaskSetManager: Starting task 140.0 in stage 8.0 (TID 148) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:37 INFO TaskSetManager: Finished task 136.0 in stage 8.0 (TID 144) in 6393 ms on 10.0.0.133 (executor driver) (137/208) +26/04/01 06:03:37 INFO Executor: Running task 140.0 in stage 8.0 (TID 148) +26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:37 INFO Executor: Finished task 137.0 in stage 8.0 (TID 145). 6787 bytes result sent to driver +26/04/01 06:03:37 INFO TaskSetManager: Starting task 141.0 in stage 8.0 (TID 149) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:37 INFO Executor: Running task 141.0 in stage 8.0 (TID 149) +26/04/01 06:03:37 INFO TaskSetManager: Finished task 137.0 in stage 8.0 (TID 145) in 6396 ms on 10.0.0.133 (executor driver) (138/208) +26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:37 INFO Executor: Finished task 138.0 in stage 8.0 (TID 146). 6787 bytes result sent to driver +26/04/01 06:03:37 INFO TaskSetManager: Starting task 142.0 in stage 8.0 (TID 150) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:37 INFO TaskSetManager: Finished task 138.0 in stage 8.0 (TID 146) in 6411 ms on 10.0.0.133 (executor driver) (139/208) +26/04/01 06:03:37 INFO Executor: Running task 142.0 in stage 8.0 (TID 150) +26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:37 INFO Executor: Finished task 139.0 in stage 8.0 (TID 147). 6787 bytes result sent to driver +26/04/01 06:03:37 INFO TaskSetManager: Starting task 143.0 in stage 8.0 (TID 151) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:37 INFO TaskSetManager: Finished task 139.0 in stage 8.0 (TID 147) in 6390 ms on 10.0.0.133 (executor driver) (140/208) +26/04/01 06:03:37 INFO Executor: Running task 143.0 in stage 8.0 (TID 151) +26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:43 INFO Executor: Finished task 140.0 in stage 8.0 (TID 148). 6787 bytes result sent to driver +26/04/01 06:03:43 INFO TaskSetManager: Starting task 144.0 in stage 8.0 (TID 152) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:03:43 INFO TaskSetManager: Finished task 140.0 in stage 8.0 (TID 148) in 6385 ms on 10.0.0.133 (executor driver) (141/208) +26/04/01 06:03:43 INFO Executor: Running task 144.0 in stage 8.0 (TID 152) +26/04/01 06:03:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:43 INFO Executor: Finished task 141.0 in stage 8.0 (TID 149). 6787 bytes result sent to driver +26/04/01 06:03:43 INFO TaskSetManager: Starting task 145.0 in stage 8.0 (TID 153) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:03:43 INFO Executor: Running task 145.0 in stage 8.0 (TID 153) +26/04/01 06:03:43 INFO TaskSetManager: Finished task 141.0 in stage 8.0 (TID 149) in 6382 ms on 10.0.0.133 (executor driver) (142/208) +26/04/01 06:03:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:43 INFO Executor: Finished task 142.0 in stage 8.0 (TID 150). 6787 bytes result sent to driver +26/04/01 06:03:43 INFO TaskSetManager: Starting task 146.0 in stage 8.0 (TID 154) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:03:43 INFO Executor: Running task 146.0 in stage 8.0 (TID 154) +26/04/01 06:03:43 INFO TaskSetManager: Finished task 142.0 in stage 8.0 (TID 150) in 6388 ms on 10.0.0.133 (executor driver) (143/208) +26/04/01 06:03:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:44 INFO Executor: Finished task 143.0 in stage 8.0 (TID 151). 6787 bytes result sent to driver +26/04/01 06:03:44 INFO TaskSetManager: Starting task 147.0 in stage 8.0 (TID 155) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:03:44 INFO Executor: Running task 147.0 in stage 8.0 (TID 155) +26/04/01 06:03:44 INFO TaskSetManager: Finished task 143.0 in stage 8.0 (TID 151) in 6383 ms on 10.0.0.133 (executor driver) (144/208) +26/04/01 06:03:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:50 INFO Executor: Finished task 144.0 in stage 8.0 (TID 152). 6787 bytes result sent to driver +26/04/01 06:03:50 INFO TaskSetManager: Starting task 148.0 in stage 8.0 (TID 156) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:03:50 INFO Executor: Running task 148.0 in stage 8.0 (TID 156) +26/04/01 06:03:50 INFO TaskSetManager: Finished task 144.0 in stage 8.0 (TID 152) in 6448 ms on 10.0.0.133 (executor driver) (145/208) +26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:50 INFO Executor: Finished task 146.0 in stage 8.0 (TID 154). 6787 bytes result sent to driver +26/04/01 06:03:50 INFO TaskSetManager: Starting task 149.0 in stage 8.0 (TID 157) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:03:50 INFO TaskSetManager: Finished task 146.0 in stage 8.0 (TID 154) in 6436 ms on 10.0.0.133 (executor driver) (146/208) +26/04/01 06:03:50 INFO Executor: Running task 149.0 in stage 8.0 (TID 157) +26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:50 INFO Executor: Finished task 145.0 in stage 8.0 (TID 153). 6787 bytes result sent to driver +26/04/01 06:03:50 INFO TaskSetManager: Starting task 150.0 in stage 8.0 (TID 158) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:03:50 INFO TaskSetManager: Finished task 145.0 in stage 8.0 (TID 153) in 6459 ms on 10.0.0.133 (executor driver) (147/208) +26/04/01 06:03:50 INFO Executor: Running task 150.0 in stage 8.0 (TID 158) +26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:50 INFO Executor: Finished task 147.0 in stage 8.0 (TID 155). 6787 bytes result sent to driver +26/04/01 06:03:50 INFO TaskSetManager: Starting task 151.0 in stage 8.0 (TID 159) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:03:50 INFO TaskSetManager: Finished task 147.0 in stage 8.0 (TID 155) in 6460 ms on 10.0.0.133 (executor driver) (148/208) +26/04/01 06:03:50 INFO Executor: Running task 151.0 in stage 8.0 (TID 159) +26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:56 INFO Executor: Finished task 148.0 in stage 8.0 (TID 156). 6787 bytes result sent to driver +26/04/01 06:03:56 INFO TaskSetManager: Starting task 152.0 in stage 8.0 (TID 160) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:56 INFO TaskSetManager: Finished task 148.0 in stage 8.0 (TID 156) in 6463 ms on 10.0.0.133 (executor driver) (149/208) +26/04/01 06:03:56 INFO Executor: Running task 152.0 in stage 8.0 (TID 160) +26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:56 INFO Executor: Finished task 149.0 in stage 8.0 (TID 157). 6787 bytes result sent to driver +26/04/01 06:03:56 INFO TaskSetManager: Starting task 153.0 in stage 8.0 (TID 161) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:56 INFO TaskSetManager: Finished task 149.0 in stage 8.0 (TID 157) in 6462 ms on 10.0.0.133 (executor driver) (150/208) +26/04/01 06:03:56 INFO Executor: Running task 153.0 in stage 8.0 (TID 161) +26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:56 INFO Executor: Finished task 150.0 in stage 8.0 (TID 158). 6787 bytes result sent to driver +26/04/01 06:03:56 INFO TaskSetManager: Starting task 154.0 in stage 8.0 (TID 162) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:56 INFO TaskSetManager: Finished task 150.0 in stage 8.0 (TID 158) in 6464 ms on 10.0.0.133 (executor driver) (151/208) +26/04/01 06:03:56 INFO Executor: Running task 154.0 in stage 8.0 (TID 162) +26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:56 INFO Executor: Finished task 151.0 in stage 8.0 (TID 159). 6787 bytes result sent to driver +26/04/01 06:03:56 INFO TaskSetManager: Starting task 155.0 in stage 8.0 (TID 163) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:03:56 INFO TaskSetManager: Finished task 151.0 in stage 8.0 (TID 159) in 6462 ms on 10.0.0.133 (executor driver) (152/208) +26/04/01 06:03:56 INFO Executor: Running task 155.0 in stage 8.0 (TID 163) +26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:02 INFO Executor: Finished task 152.0 in stage 8.0 (TID 160). 6787 bytes result sent to driver +26/04/01 06:04:02 INFO TaskSetManager: Starting task 156.0 in stage 8.0 (TID 164) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:04:02 INFO TaskSetManager: Finished task 152.0 in stage 8.0 (TID 160) in 6447 ms on 10.0.0.133 (executor driver) (153/208) +26/04/01 06:04:02 INFO Executor: Running task 156.0 in stage 8.0 (TID 164) +26/04/01 06:04:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:03 INFO Executor: Finished task 153.0 in stage 8.0 (TID 161). 6787 bytes result sent to driver +26/04/01 06:04:03 INFO TaskSetManager: Starting task 157.0 in stage 8.0 (TID 165) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:04:03 INFO Executor: Running task 157.0 in stage 8.0 (TID 165) +26/04/01 06:04:03 INFO TaskSetManager: Finished task 153.0 in stage 8.0 (TID 161) in 6447 ms on 10.0.0.133 (executor driver) (154/208) +26/04/01 06:04:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:03 INFO Executor: Finished task 154.0 in stage 8.0 (TID 162). 6787 bytes result sent to driver +26/04/01 06:04:03 INFO TaskSetManager: Starting task 158.0 in stage 8.0 (TID 166) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:03 INFO Executor: Running task 158.0 in stage 8.0 (TID 166) +26/04/01 06:04:03 INFO TaskSetManager: Finished task 154.0 in stage 8.0 (TID 162) in 6447 ms on 10.0.0.133 (executor driver) (155/208) +26/04/01 06:04:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:03 INFO Executor: Finished task 155.0 in stage 8.0 (TID 163). 6787 bytes result sent to driver +26/04/01 06:04:03 INFO TaskSetManager: Starting task 159.0 in stage 8.0 (TID 167) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:03 INFO Executor: Running task 159.0 in stage 8.0 (TID 167) +26/04/01 06:04:03 INFO TaskSetManager: Finished task 155.0 in stage 8.0 (TID 163) in 6450 ms on 10.0.0.133 (executor driver) (156/208) +26/04/01 06:04:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:09 INFO Executor: Finished task 156.0 in stage 8.0 (TID 164). 6787 bytes result sent to driver +26/04/01 06:04:09 INFO TaskSetManager: Starting task 160.0 in stage 8.0 (TID 168) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:09 INFO TaskSetManager: Finished task 156.0 in stage 8.0 (TID 164) in 6408 ms on 10.0.0.133 (executor driver) (157/208) +26/04/01 06:04:09 INFO Executor: Running task 160.0 in stage 8.0 (TID 168) +26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:09 INFO Executor: Finished task 158.0 in stage 8.0 (TID 166). 6787 bytes result sent to driver +26/04/01 06:04:09 INFO Executor: Finished task 157.0 in stage 8.0 (TID 165). 6787 bytes result sent to driver +26/04/01 06:04:09 INFO TaskSetManager: Starting task 161.0 in stage 8.0 (TID 169) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:09 INFO Executor: Running task 161.0 in stage 8.0 (TID 169) +26/04/01 06:04:09 INFO TaskSetManager: Starting task 162.0 in stage 8.0 (TID 170) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:04:09 INFO TaskSetManager: Finished task 158.0 in stage 8.0 (TID 166) in 6406 ms on 10.0.0.133 (executor driver) (158/208) +26/04/01 06:04:09 INFO Executor: Running task 162.0 in stage 8.0 (TID 170) +26/04/01 06:04:09 INFO TaskSetManager: Finished task 157.0 in stage 8.0 (TID 165) in 6410 ms on 10.0.0.133 (executor driver) (159/208) +26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:09 INFO Executor: Finished task 159.0 in stage 8.0 (TID 167). 6787 bytes result sent to driver +26/04/01 06:04:09 INFO TaskSetManager: Starting task 163.0 in stage 8.0 (TID 171) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:04:09 INFO TaskSetManager: Finished task 159.0 in stage 8.0 (TID 167) in 6410 ms on 10.0.0.133 (executor driver) (160/208) +26/04/01 06:04:09 INFO Executor: Running task 163.0 in stage 8.0 (TID 171) +26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:15 INFO Executor: Finished task 160.0 in stage 8.0 (TID 168). 6787 bytes result sent to driver +26/04/01 06:04:15 INFO TaskSetManager: Starting task 164.0 in stage 8.0 (TID 172) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:04:15 INFO Executor: Running task 164.0 in stage 8.0 (TID 172) +26/04/01 06:04:15 INFO TaskSetManager: Finished task 160.0 in stage 8.0 (TID 168) in 6422 ms on 10.0.0.133 (executor driver) (161/208) +26/04/01 06:04:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:16 INFO Executor: Finished task 162.0 in stage 8.0 (TID 170). 6787 bytes result sent to driver +26/04/01 06:04:16 INFO TaskSetManager: Starting task 165.0 in stage 8.0 (TID 173) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:04:16 INFO Executor: Finished task 161.0 in stage 8.0 (TID 169). 6787 bytes result sent to driver +26/04/01 06:04:16 INFO TaskSetManager: Finished task 162.0 in stage 8.0 (TID 170) in 6422 ms on 10.0.0.133 (executor driver) (162/208) +26/04/01 06:04:16 INFO Executor: Running task 165.0 in stage 8.0 (TID 173) +26/04/01 06:04:16 INFO TaskSetManager: Starting task 166.0 in stage 8.0 (TID 174) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:04:16 INFO Executor: Running task 166.0 in stage 8.0 (TID 174) +26/04/01 06:04:16 INFO TaskSetManager: Finished task 161.0 in stage 8.0 (TID 169) in 6423 ms on 10.0.0.133 (executor driver) (163/208) +26/04/01 06:04:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:16 INFO Executor: Finished task 163.0 in stage 8.0 (TID 171). 6830 bytes result sent to driver +26/04/01 06:04:16 INFO TaskSetManager: Starting task 167.0 in stage 8.0 (TID 175) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:04:16 INFO Executor: Running task 167.0 in stage 8.0 (TID 175) +26/04/01 06:04:16 INFO TaskSetManager: Finished task 163.0 in stage 8.0 (TID 171) in 6417 ms on 10.0.0.133 (executor driver) (164/208) +26/04/01 06:04:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:22 INFO Executor: Finished task 164.0 in stage 8.0 (TID 172). 6830 bytes result sent to driver +26/04/01 06:04:22 INFO TaskSetManager: Starting task 168.0 in stage 8.0 (TID 176) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:04:22 INFO Executor: Running task 168.0 in stage 8.0 (TID 176) +26/04/01 06:04:22 INFO TaskSetManager: Finished task 164.0 in stage 8.0 (TID 172) in 6422 ms on 10.0.0.133 (executor driver) (165/208) +26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:22 INFO Executor: Finished task 165.0 in stage 8.0 (TID 173). 6830 bytes result sent to driver +26/04/01 06:04:22 INFO TaskSetManager: Starting task 169.0 in stage 8.0 (TID 177) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:04:22 INFO Executor: Running task 169.0 in stage 8.0 (TID 177) +26/04/01 06:04:22 INFO TaskSetManager: Finished task 165.0 in stage 8.0 (TID 173) in 6418 ms on 10.0.0.133 (executor driver) (166/208) +26/04/01 06:04:22 INFO Executor: Finished task 166.0 in stage 8.0 (TID 174). 6830 bytes result sent to driver +26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:22 INFO TaskSetManager: Starting task 170.0 in stage 8.0 (TID 178) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:22 INFO TaskSetManager: Finished task 166.0 in stage 8.0 (TID 174) in 6420 ms on 10.0.0.133 (executor driver) (167/208) +26/04/01 06:04:22 INFO Executor: Running task 170.0 in stage 8.0 (TID 178) +26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:22 INFO Executor: Finished task 167.0 in stage 8.0 (TID 175). 6787 bytes result sent to driver +26/04/01 06:04:22 INFO TaskSetManager: Starting task 171.0 in stage 8.0 (TID 179) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:22 INFO TaskSetManager: Finished task 167.0 in stage 8.0 (TID 175) in 6428 ms on 10.0.0.133 (executor driver) (168/208) +26/04/01 06:04:22 INFO Executor: Running task 171.0 in stage 8.0 (TID 179) +26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:28 INFO Executor: Finished task 168.0 in stage 8.0 (TID 176). 6787 bytes result sent to driver +26/04/01 06:04:28 INFO TaskSetManager: Starting task 172.0 in stage 8.0 (TID 180) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:28 INFO TaskSetManager: Finished task 168.0 in stage 8.0 (TID 176) in 6424 ms on 10.0.0.133 (executor driver) (169/208) +26/04/01 06:04:28 INFO Executor: Running task 172.0 in stage 8.0 (TID 180) +26/04/01 06:04:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:28 INFO Executor: Finished task 169.0 in stage 8.0 (TID 177). 6787 bytes result sent to driver +26/04/01 06:04:28 INFO Executor: Finished task 170.0 in stage 8.0 (TID 178). 6787 bytes result sent to driver +26/04/01 06:04:28 INFO TaskSetManager: Starting task 173.0 in stage 8.0 (TID 181) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:28 INFO TaskSetManager: Finished task 169.0 in stage 8.0 (TID 177) in 6426 ms on 10.0.0.133 (executor driver) (170/208) +26/04/01 06:04:28 INFO Executor: Running task 173.0 in stage 8.0 (TID 181) +26/04/01 06:04:28 INFO TaskSetManager: Starting task 174.0 in stage 8.0 (TID 182) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:04:28 INFO TaskSetManager: Finished task 170.0 in stage 8.0 (TID 178) in 6424 ms on 10.0.0.133 (executor driver) (171/208) +26/04/01 06:04:28 INFO Executor: Running task 174.0 in stage 8.0 (TID 182) +26/04/01 06:04:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:29 INFO Executor: Finished task 171.0 in stage 8.0 (TID 179). 6787 bytes result sent to driver +26/04/01 06:04:29 INFO TaskSetManager: Starting task 175.0 in stage 8.0 (TID 183) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:04:29 INFO TaskSetManager: Finished task 171.0 in stage 8.0 (TID 179) in 6416 ms on 10.0.0.133 (executor driver) (172/208) +26/04/01 06:04:29 INFO Executor: Running task 175.0 in stage 8.0 (TID 183) +26/04/01 06:04:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:35 INFO Executor: Finished task 172.0 in stage 8.0 (TID 180). 6787 bytes result sent to driver +26/04/01 06:04:35 INFO TaskSetManager: Starting task 176.0 in stage 8.0 (TID 184) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:04:35 INFO Executor: Running task 176.0 in stage 8.0 (TID 184) +26/04/01 06:04:35 INFO TaskSetManager: Finished task 172.0 in stage 8.0 (TID 180) in 6420 ms on 10.0.0.133 (executor driver) (173/208) +26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:35 INFO Executor: Finished task 173.0 in stage 8.0 (TID 181). 6787 bytes result sent to driver +26/04/01 06:04:35 INFO TaskSetManager: Starting task 177.0 in stage 8.0 (TID 185) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:04:35 INFO TaskSetManager: Finished task 173.0 in stage 8.0 (TID 181) in 6419 ms on 10.0.0.133 (executor driver) (174/208) +26/04/01 06:04:35 INFO Executor: Running task 177.0 in stage 8.0 (TID 185) +26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:35 INFO Executor: Finished task 174.0 in stage 8.0 (TID 182). 6787 bytes result sent to driver +26/04/01 06:04:35 INFO TaskSetManager: Starting task 178.0 in stage 8.0 (TID 186) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:04:35 INFO Executor: Running task 178.0 in stage 8.0 (TID 186) +26/04/01 06:04:35 INFO TaskSetManager: Finished task 174.0 in stage 8.0 (TID 182) in 6434 ms on 10.0.0.133 (executor driver) (175/208) +26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:35 INFO Executor: Finished task 175.0 in stage 8.0 (TID 183). 6787 bytes result sent to driver +26/04/01 06:04:35 INFO TaskSetManager: Starting task 179.0 in stage 8.0 (TID 187) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:04:35 INFO TaskSetManager: Finished task 175.0 in stage 8.0 (TID 183) in 6430 ms on 10.0.0.133 (executor driver) (176/208) +26/04/01 06:04:35 INFO Executor: Running task 179.0 in stage 8.0 (TID 187) +26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:41 INFO Executor: Finished task 176.0 in stage 8.0 (TID 184). 6787 bytes result sent to driver +26/04/01 06:04:41 INFO TaskSetManager: Starting task 180.0 in stage 8.0 (TID 188) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:04:41 INFO Executor: Running task 180.0 in stage 8.0 (TID 188) +26/04/01 06:04:41 INFO TaskSetManager: Finished task 176.0 in stage 8.0 (TID 184) in 6454 ms on 10.0.0.133 (executor driver) (177/208) +26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:41 INFO Executor: Finished task 177.0 in stage 8.0 (TID 185). 6787 bytes result sent to driver +26/04/01 06:04:41 INFO Executor: Finished task 178.0 in stage 8.0 (TID 186). 6787 bytes result sent to driver +26/04/01 06:04:41 INFO TaskSetManager: Starting task 181.0 in stage 8.0 (TID 189) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:04:41 INFO TaskSetManager: Finished task 177.0 in stage 8.0 (TID 185) in 6452 ms on 10.0.0.133 (executor driver) (178/208) +26/04/01 06:04:41 INFO Executor: Running task 181.0 in stage 8.0 (TID 189) +26/04/01 06:04:41 INFO TaskSetManager: Starting task 182.0 in stage 8.0 (TID 190) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:41 INFO TaskSetManager: Finished task 178.0 in stage 8.0 (TID 186) in 6438 ms on 10.0.0.133 (executor driver) (179/208) +26/04/01 06:04:41 INFO Executor: Running task 182.0 in stage 8.0 (TID 190) +26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:41 INFO Executor: Finished task 179.0 in stage 8.0 (TID 187). 6787 bytes result sent to driver +26/04/01 06:04:41 INFO TaskSetManager: Starting task 183.0 in stage 8.0 (TID 191) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:41 INFO TaskSetManager: Finished task 179.0 in stage 8.0 (TID 187) in 6446 ms on 10.0.0.133 (executor driver) (180/208) +26/04/01 06:04:41 INFO Executor: Running task 183.0 in stage 8.0 (TID 191) +26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:47 INFO Executor: Finished task 180.0 in stage 8.0 (TID 188). 6787 bytes result sent to driver +26/04/01 06:04:47 INFO TaskSetManager: Starting task 184.0 in stage 8.0 (TID 192) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:47 INFO TaskSetManager: Finished task 180.0 in stage 8.0 (TID 188) in 6482 ms on 10.0.0.133 (executor driver) (181/208) +26/04/01 06:04:47 INFO Executor: Running task 184.0 in stage 8.0 (TID 192) +26/04/01 06:04:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:48 INFO Executor: Finished task 181.0 in stage 8.0 (TID 189). 6787 bytes result sent to driver +26/04/01 06:04:48 INFO Executor: Finished task 182.0 in stage 8.0 (TID 190). 6787 bytes result sent to driver +26/04/01 06:04:48 INFO TaskSetManager: Starting task 185.0 in stage 8.0 (TID 193) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:48 INFO TaskSetManager: Starting task 186.0 in stage 8.0 (TID 194) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:04:48 INFO Executor: Running task 185.0 in stage 8.0 (TID 193) +26/04/01 06:04:48 INFO TaskSetManager: Finished task 181.0 in stage 8.0 (TID 189) in 6489 ms on 10.0.0.133 (executor driver) (182/208) +26/04/01 06:04:48 INFO Executor: Running task 186.0 in stage 8.0 (TID 194) +26/04/01 06:04:48 INFO TaskSetManager: Finished task 182.0 in stage 8.0 (TID 190) in 6488 ms on 10.0.0.133 (executor driver) (183/208) +26/04/01 06:04:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:48 INFO Executor: Finished task 183.0 in stage 8.0 (TID 191). 6787 bytes result sent to driver +26/04/01 06:04:48 INFO TaskSetManager: Starting task 187.0 in stage 8.0 (TID 195) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:04:48 INFO TaskSetManager: Finished task 183.0 in stage 8.0 (TID 191) in 6484 ms on 10.0.0.133 (executor driver) (184/208) +26/04/01 06:04:48 INFO Executor: Running task 187.0 in stage 8.0 (TID 195) +26/04/01 06:04:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:54 INFO Executor: Finished task 184.0 in stage 8.0 (TID 192). 6787 bytes result sent to driver +26/04/01 06:04:54 INFO TaskSetManager: Starting task 188.0 in stage 8.0 (TID 196) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:54 INFO Executor: Running task 188.0 in stage 8.0 (TID 196) +26/04/01 06:04:54 INFO TaskSetManager: Finished task 184.0 in stage 8.0 (TID 192) in 6490 ms on 10.0.0.133 (executor driver) (185/208) +26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:54 INFO Executor: Finished task 186.0 in stage 8.0 (TID 194). 6787 bytes result sent to driver +26/04/01 06:04:54 INFO TaskSetManager: Starting task 189.0 in stage 8.0 (TID 197) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:54 INFO TaskSetManager: Finished task 186.0 in stage 8.0 (TID 194) in 6482 ms on 10.0.0.133 (executor driver) (186/208) +26/04/01 06:04:54 INFO Executor: Running task 189.0 in stage 8.0 (TID 197) +26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:54 INFO Executor: Finished task 185.0 in stage 8.0 (TID 193). 6787 bytes result sent to driver +26/04/01 06:04:54 INFO TaskSetManager: Starting task 190.0 in stage 8.0 (TID 198) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:54 INFO Executor: Running task 190.0 in stage 8.0 (TID 198) +26/04/01 06:04:54 INFO TaskSetManager: Finished task 185.0 in stage 8.0 (TID 193) in 6485 ms on 10.0.0.133 (executor driver) (187/208) +26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:54 INFO Executor: Finished task 187.0 in stage 8.0 (TID 195). 6787 bytes result sent to driver +26/04/01 06:04:54 INFO TaskSetManager: Starting task 191.0 in stage 8.0 (TID 199) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:04:54 INFO Executor: Running task 191.0 in stage 8.0 (TID 199) +26/04/01 06:04:54 INFO TaskSetManager: Finished task 187.0 in stage 8.0 (TID 195) in 6484 ms on 10.0.0.133 (executor driver) (188/208) +26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:00 INFO Executor: Finished task 188.0 in stage 8.0 (TID 196). 6787 bytes result sent to driver +26/04/01 06:05:00 INFO TaskSetManager: Starting task 192.0 in stage 8.0 (TID 200) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:05:00 INFO TaskSetManager: Finished task 188.0 in stage 8.0 (TID 196) in 6477 ms on 10.0.0.133 (executor driver) (189/208) +26/04/01 06:05:00 INFO Executor: Running task 192.0 in stage 8.0 (TID 200) +26/04/01 06:05:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:01 INFO Executor: Finished task 189.0 in stage 8.0 (TID 197). 6787 bytes result sent to driver +26/04/01 06:05:01 INFO Executor: Finished task 190.0 in stage 8.0 (TID 198). 6787 bytes result sent to driver +26/04/01 06:05:01 INFO TaskSetManager: Starting task 193.0 in stage 8.0 (TID 201) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:05:01 INFO Executor: Running task 193.0 in stage 8.0 (TID 201) +26/04/01 06:05:01 INFO TaskSetManager: Starting task 194.0 in stage 8.0 (TID 202) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:05:01 INFO TaskSetManager: Finished task 189.0 in stage 8.0 (TID 197) in 6481 ms on 10.0.0.133 (executor driver) (190/208) +26/04/01 06:05:01 INFO Executor: Running task 194.0 in stage 8.0 (TID 202) +26/04/01 06:05:01 INFO TaskSetManager: Finished task 190.0 in stage 8.0 (TID 198) in 6478 ms on 10.0.0.133 (executor driver) (191/208) +26/04/01 06:05:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:01 INFO Executor: Finished task 191.0 in stage 8.0 (TID 199). 6787 bytes result sent to driver +26/04/01 06:05:01 INFO TaskSetManager: Starting task 195.0 in stage 8.0 (TID 203) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:05:01 INFO TaskSetManager: Finished task 191.0 in stage 8.0 (TID 199) in 6478 ms on 10.0.0.133 (executor driver) (192/208) +26/04/01 06:05:01 INFO Executor: Running task 195.0 in stage 8.0 (TID 203) +26/04/01 06:05:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:04 INFO Executor: Finished task 192.0 in stage 8.0 (TID 200). 6787 bytes result sent to driver +26/04/01 06:05:04 INFO TaskSetManager: Starting task 196.0 in stage 8.0 (TID 204) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:05:04 INFO Executor: Running task 196.0 in stage 8.0 (TID 204) +26/04/01 06:05:04 INFO TaskSetManager: Finished task 192.0 in stage 8.0 (TID 200) in 4019 ms on 10.0.0.133 (executor driver) (193/208) +26/04/01 06:05:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:05 INFO Executor: Finished task 193.0 in stage 8.0 (TID 201). 6787 bytes result sent to driver +26/04/01 06:05:05 INFO TaskSetManager: Starting task 197.0 in stage 8.0 (TID 205) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:05:05 INFO TaskSetManager: Finished task 193.0 in stage 8.0 (TID 201) in 4016 ms on 10.0.0.133 (executor driver) (194/208) +26/04/01 06:05:05 INFO Executor: Running task 197.0 in stage 8.0 (TID 205) +26/04/01 06:05:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:05 INFO Executor: Finished task 194.0 in stage 8.0 (TID 202). 6787 bytes result sent to driver +26/04/01 06:05:05 INFO TaskSetManager: Starting task 198.0 in stage 8.0 (TID 206) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:05:05 INFO TaskSetManager: Finished task 194.0 in stage 8.0 (TID 202) in 4016 ms on 10.0.0.133 (executor driver) (195/208) +26/04/01 06:05:05 INFO Executor: Running task 198.0 in stage 8.0 (TID 206) +26/04/01 06:05:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:05 INFO Executor: Finished task 195.0 in stage 8.0 (TID 203). 6787 bytes result sent to driver +26/04/01 06:05:05 INFO TaskSetManager: Starting task 199.0 in stage 8.0 (TID 207) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:05:05 INFO TaskSetManager: Finished task 195.0 in stage 8.0 (TID 203) in 4022 ms on 10.0.0.133 (executor driver) (196/208) +26/04/01 06:05:05 INFO Executor: Running task 199.0 in stage 8.0 (TID 207) +26/04/01 06:05:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:08 INFO Executor: Finished task 196.0 in stage 8.0 (TID 204). 6787 bytes result sent to driver +26/04/01 06:05:08 INFO TaskSetManager: Starting task 200.0 in stage 8.0 (TID 208) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9346 bytes) +26/04/01 06:05:08 INFO Executor: Running task 200.0 in stage 8.0 (TID 208) +26/04/01 06:05:08 INFO TaskSetManager: Finished task 196.0 in stage 8.0 (TID 204) in 4014 ms on 10.0.0.133 (executor driver) (197/208) +26/04/01 06:05:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:09 INFO Executor: Finished task 198.0 in stage 8.0 (TID 206). 6787 bytes result sent to driver +26/04/01 06:05:09 INFO Executor: Finished task 197.0 in stage 8.0 (TID 205). 6787 bytes result sent to driver +26/04/01 06:05:09 INFO TaskSetManager: Starting task 201.0 in stage 8.0 (TID 209) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:05:09 INFO Executor: Running task 201.0 in stage 8.0 (TID 209) +26/04/01 06:05:09 INFO TaskSetManager: Starting task 202.0 in stage 8.0 (TID 210) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:05:09 INFO TaskSetManager: Finished task 198.0 in stage 8.0 (TID 206) in 4016 ms on 10.0.0.133 (executor driver) (198/208) +26/04/01 06:05:09 INFO Executor: Running task 202.0 in stage 8.0 (TID 210) +26/04/01 06:05:09 INFO TaskSetManager: Finished task 197.0 in stage 8.0 (TID 205) in 4017 ms on 10.0.0.133 (executor driver) (199/208) +26/04/01 06:05:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:09 INFO Executor: Finished task 199.0 in stage 8.0 (TID 207). 6787 bytes result sent to driver +26/04/01 06:05:09 INFO TaskSetManager: Starting task 203.0 in stage 8.0 (TID 211) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:05:09 INFO TaskSetManager: Finished task 199.0 in stage 8.0 (TID 207) in 4016 ms on 10.0.0.133 (executor driver) (200/208) +26/04/01 06:05:09 INFO Executor: Running task 203.0 in stage 8.0 (TID 211) +26/04/01 06:05:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:12 INFO Executor: Finished task 200.0 in stage 8.0 (TID 208). 6787 bytes result sent to driver +26/04/01 06:05:12 INFO TaskSetManager: Starting task 204.0 in stage 8.0 (TID 212) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:05:12 INFO TaskSetManager: Finished task 200.0 in stage 8.0 (TID 208) in 3996 ms on 10.0.0.133 (executor driver) (201/208) +26/04/01 06:05:12 INFO Executor: Running task 204.0 in stage 8.0 (TID 212) +26/04/01 06:05:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:13 INFO Executor: Finished task 202.0 in stage 8.0 (TID 210). 6787 bytes result sent to driver +26/04/01 06:05:13 INFO TaskSetManager: Starting task 205.0 in stage 8.0 (TID 213) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:05:13 INFO TaskSetManager: Finished task 202.0 in stage 8.0 (TID 210) in 3997 ms on 10.0.0.133 (executor driver) (202/208) +26/04/01 06:05:13 INFO Executor: Running task 205.0 in stage 8.0 (TID 213) +26/04/01 06:05:13 INFO Executor: Finished task 201.0 in stage 8.0 (TID 209). 6787 bytes result sent to driver +26/04/01 06:05:13 INFO TaskSetManager: Starting task 206.0 in stage 8.0 (TID 214) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:05:13 INFO Executor: Running task 206.0 in stage 8.0 (TID 214) +26/04/01 06:05:13 INFO TaskSetManager: Finished task 201.0 in stage 8.0 (TID 209) in 3998 ms on 10.0.0.133 (executor driver) (203/208) +26/04/01 06:05:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:13 INFO Executor: Finished task 203.0 in stage 8.0 (TID 211). 6787 bytes result sent to driver +26/04/01 06:05:13 INFO TaskSetManager: Starting task 207.0 in stage 8.0 (TID 215) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:05:13 INFO TaskSetManager: Finished task 203.0 in stage 8.0 (TID 211) in 3996 ms on 10.0.0.133 (executor driver) (204/208) +26/04/01 06:05:13 INFO Executor: Running task 207.0 in stage 8.0 (TID 215) +26/04/01 06:05:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:16 INFO Executor: Finished task 204.0 in stage 8.0 (TID 212). 6787 bytes result sent to driver +26/04/01 06:05:16 INFO TaskSetManager: Finished task 204.0 in stage 8.0 (TID 212) in 3997 ms on 10.0.0.133 (executor driver) (205/208) +26/04/01 06:05:17 INFO Executor: Finished task 205.0 in stage 8.0 (TID 213). 6787 bytes result sent to driver +26/04/01 06:05:17 INFO Executor: Finished task 206.0 in stage 8.0 (TID 214). 6787 bytes result sent to driver +26/04/01 06:05:17 INFO TaskSetManager: Finished task 205.0 in stage 8.0 (TID 213) in 3994 ms on 10.0.0.133 (executor driver) (206/208) +26/04/01 06:05:17 INFO TaskSetManager: Finished task 206.0 in stage 8.0 (TID 214) in 3994 ms on 10.0.0.133 (executor driver) (207/208) +26/04/01 06:05:17 INFO Executor: Finished task 207.0 in stage 8.0 (TID 215). 6787 bytes result sent to driver +26/04/01 06:05:17 INFO TaskSetManager: Finished task 207.0 in stage 8.0 (TID 215) in 3967 ms on 10.0.0.133 (executor driver) (208/208) +26/04/01 06:05:17 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool +26/04/01 06:05:17 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 325.669 s +26/04/01 06:05:17 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:05:17 INFO DAGScheduler: running: Set() +26/04/01 06:05:17 INFO DAGScheduler: waiting: Set() +26/04/01 06:05:17 INFO DAGScheduler: failed: Set() +26/04/01 06:05:17 INFO ShufflePartitionsUtil: For shuffle(0), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 06:05:17 INFO CodeGenerator: Code generated in 61.275208 ms +26/04/01 06:05:17 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:05:17 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:05:17 INFO DAGScheduler: Final stage: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:05:17 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 9) +26/04/01 06:05:17 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:17 INFO DAGScheduler: Submitting ResultStage 10 (MapPartitionsRDD[23] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:05:17 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 49.9 KiB, free 8.6 GiB) +26/04/01 06:05:17 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 10.4 KiB, free 8.6 GiB) +26/04/01 06:05:17 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:58235 (size: 10.4 KiB, free: 8.6 GiB) +26/04/01 06:05:17 INFO SparkContext: Created broadcast 10 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:17 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 10 (MapPartitionsRDD[23] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:05:17 INFO TaskSchedulerImpl: Adding task set 10.0 with 1 tasks resource profile 0 +26/04/01 06:05:17 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 216) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) +26/04/01 06:05:17 INFO Executor: Running task 0.0 in stage 10.0 (TID 216) +26/04/01 06:05:17 INFO ShuffleBlockFetcherIterator: Getting 208 (853.9 KiB) non-empty blocks including 208 (853.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:05:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms +26/04/01 06:05:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:17 INFO CodeGenerator: Code generated in 5.76175 ms +26/04/01 06:05:17 INFO Executor: Finished task 0.0 in stage 10.0 (TID 216). 5160 bytes result sent to driver +26/04/01 06:05:17 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 216) in 342 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:05:17 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool +26/04/01 06:05:17 INFO DAGScheduler: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.345 s +26/04/01 06:05:17 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:05:17 INFO TaskSchedulerImpl: Killing all running tasks in stage 10: Stage finished +26/04/01 06:05:17 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.349708 s +26/04/01 06:05:17 INFO DAGScheduler: Registering RDD 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 +26/04/01 06:05:17 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:05:17 INFO DAGScheduler: Final stage: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:05:17 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 11) +26/04/01 06:05:17 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:17 INFO DAGScheduler: Submitting ShuffleMapStage 12 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:05:17 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 51.7 KiB, free 8.6 GiB) +26/04/01 06:05:17 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 11.2 KiB, free 8.6 GiB) +26/04/01 06:05:17 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:58235 (size: 11.2 KiB, free: 8.6 GiB) +26/04/01 06:05:17 INFO SparkContext: Created broadcast 11 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:17 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 12 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:05:17 INFO TaskSchedulerImpl: Adding task set 12.0 with 1 tasks resource profile 0 +26/04/01 06:05:17 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 217) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9404 bytes) +26/04/01 06:05:17 INFO Executor: Running task 0.0 in stage 12.0 (TID 217) +26/04/01 06:05:17 INFO CodeGenerator: Code generated in 3.546 ms +26/04/01 06:05:17 INFO ShuffleBlockFetcherIterator: Getting 208 (853.9 KiB) non-empty blocks including 208 (853.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:05:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:05:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:18 INFO BlockManagerInfo: Removed broadcast_10_piece0 on 10.0.0.133:58235 in memory (size: 10.4 KiB, free: 8.6 GiB) +26/04/01 06:05:18 INFO Executor: Finished task 0.0 in stage 12.0 (TID 217). 6609 bytes result sent to driver +26/04/01 06:05:18 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 217) in 313 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:05:18 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool +26/04/01 06:05:18 INFO DAGScheduler: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.316 s +26/04/01 06:05:18 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:05:18 INFO DAGScheduler: running: Set() +26/04/01 06:05:18 INFO DAGScheduler: waiting: Set() +26/04/01 06:05:18 INFO DAGScheduler: failed: Set() +26/04/01 06:05:18 INFO ShufflePartitionsUtil: For shuffle(1), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 06:05:18 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:05:18 INFO DAGScheduler: Got job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:05:18 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:05:18 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 14) +26/04/01 06:05:18 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:18 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:05:18 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 20.4 KiB, free 8.6 GiB) +26/04/01 06:05:18 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 9.0 KiB, free 8.6 GiB) +26/04/01 06:05:18 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:58235 (size: 9.0 KiB, free: 8.6 GiB) +26/04/01 06:05:18 INFO SparkContext: Created broadcast 12 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:18 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:05:18 INFO TaskSchedulerImpl: Adding task set 15.0 with 1 tasks resource profile 0 +26/04/01 06:05:18 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 218) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) +26/04/01 06:05:18 INFO Executor: Running task 0.0 in stage 15.0 (TID 218) +26/04/01 06:05:18 INFO ShuffleBlockFetcherIterator: Getting 1 (2.6 KiB) non-empty blocks including 1 (2.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:05:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:05:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:18 INFO Executor: Finished task 0.0 in stage 15.0 (TID 218). 5560 bytes result sent to driver +26/04/01 06:05:18 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 218) in 9 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:05:18 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool +26/04/01 06:05:18 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.012 s +26/04/01 06:05:18 INFO DAGScheduler: Job 11 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:05:18 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished +26/04/01 06:05:18 INFO DAGScheduler: Job 11 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.012342 s +26/04/01 06:05:18 INFO SparkContext: SparkContext is stopping with exitCode 0. +26/04/01 06:05:18 INFO CometDriverPlugin: CometDriverPlugin shutdown +26/04/01 06:05:18 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! +26/04/01 06:05:18 INFO MemoryStore: MemoryStore cleared +26/04/01 06:05:18 INFO BlockManager: BlockManager stopped +26/04/01 06:05:18 INFO BlockManagerMaster: BlockManagerMaster stopped +26/04/01 06:05:18 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! +26/04/01 06:05:18 INFO SparkContext: Successfully stopped SparkContext +26/04/01 06:05:18 INFO ShutdownHookManager: Shutdown hook called +26/04/01 06:05:18 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-8b7018bf-f101-4273-93fa-9cf7eb6d820a/pyspark-b3568a12-6421-40dc-b3fc-0213601717c5 +26/04/01 06:05:18 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-960962b7-b8cb-4781-a172-17193aefe80f +26/04/01 06:05:18 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-8b7018bf-f101-4273-93fa-9cf7eb6d820a + 332.80 real 1302.40 user 5.62 sys + 712359936 maximum resident set size + 0 average shared memory size + 0 average unshared data size + 0 average unshared stack size + 71057 page reclaims + 63 page faults + 0 swaps + 0 block input operations + 0 block output operations + 1275 messages sent + 2361 messages received + 17 signals received + 31134 voluntary context switches + 203718 involuntary context switches + 20342884625324 instructions retired + 4776772444264 cycles elapsed + 597542592 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.log b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.log new file mode 100644 index 0000000000..bea807a74f --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.log @@ -0,0 +1,317 @@ +Registering table customer from /opt/tpch/sf100/customer +Registering table lineitem from /opt/tpch/sf100/lineitem +Registering table nation from /opt/tpch/sf100/nation +Registering table orders from /opt/tpch/sf100/orders +Registering table part from /opt/tpch/sf100/part +Registering table partsupp from /opt/tpch/sf100/partsupp +Registering table region from /opt/tpch/sf100/region +Registering table supplier from /opt/tpch/sf100/supplier + +============================================================ +Starting iteration 1 of 1 +============================================================ + +Running query 5 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q5.sql +Executing: -- CometBench-H query 5 derived from TPC-H query 5 under the terms of the TPC Fair Use Policy. +-- TP... +== Physical Plan == +AdaptiveSparkPlan (44) ++- CometSort (43) + +- CometExchange (42) + +- CometHashAggregate (41) + +- CometExchange (40) + +- CometHashAggregate (39) + +- CometProject (38) + +- CometBroadcastHashJoin (37) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometProject (27) + : : +- CometSortMergeJoin (26) + : : :- CometSort (21) + : : : +- CometExchange (20) + : : : +- CometProject (19) + : : : +- CometSortMergeJoin (18) + : : : :- CometSort (13) + : : : : +- CometExchange (12) + : : : : +- CometProject (11) + : : : : +- CometSortMergeJoin (10) + : : : : :- CometSort (4) + : : : : : +- CometExchange (3) + : : : : : +- CometFilter (2) + : : : : : +- CometNativeScan parquet (1) + : : : : +- CometSort (9) + : : : : +- CometExchange (8) + : : : : +- CometProject (7) + : : : : +- CometFilter (6) + : : : : +- CometNativeScan parquet (5) + : : : +- CometSort (17) + : : : +- CometExchange (16) + : : : +- CometFilter (15) + : : : +- CometNativeScan parquet (14) + : : +- CometSort (25) + : : +- CometExchange (24) + : : +- CometFilter (23) + : : +- CometNativeScan parquet (22) + : +- CometBroadcastExchange (30) + : +- CometFilter (29) + : +- CometNativeScan parquet (28) + +- CometBroadcastExchange (36) + +- CometProject (35) + +- CometFilter (34) + +- CometNativeScan parquet (33) + + +(1) CometNativeScan parquet +Output [2]: [c_custkey#0L, c_nationkey#3L] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/customer] +PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] +ReadSchema: struct + +(2) CometFilter +Input [2]: [c_custkey#0L, c_nationkey#3L] +Condition : (isnotnull(c_custkey#0L) AND isnotnull(c_nationkey#3L)) + +(3) CometExchange +Input [2]: [c_custkey#0L, c_nationkey#3L] +Arguments: hashpartitioning(c_custkey#0L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=412] + +(4) CometSort +Input [2]: [c_custkey#0L, c_nationkey#3L] +Arguments: [c_custkey#0L, c_nationkey#3L], [c_custkey#0L ASC NULLS FIRST] + +(5) CometNativeScan parquet +Output [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/orders] +PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1994-01-01), LessThan(o_orderdate,1995-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] +ReadSchema: struct + +(6) CometFilter +Input [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] +Condition : ((((isnotnull(o_orderdate#60) AND (o_orderdate#60 >= 1994-01-01)) AND (o_orderdate#60 < 1995-01-01)) AND isnotnull(o_custkey#57L)) AND isnotnull(o_orderkey#56L)) + +(7) CometProject +Input [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] +Arguments: [o_orderkey#56L, o_custkey#57L], [o_orderkey#56L, o_custkey#57L] + +(8) CometExchange +Input [2]: [o_orderkey#56L, o_custkey#57L] +Arguments: hashpartitioning(o_custkey#57L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=416] + +(9) CometSort +Input [2]: [o_orderkey#56L, o_custkey#57L] +Arguments: [o_orderkey#56L, o_custkey#57L], [o_custkey#57L ASC NULLS FIRST] + +(10) CometSortMergeJoin +Left output [2]: [c_custkey#0L, c_nationkey#3L] +Right output [2]: [o_orderkey#56L, o_custkey#57L] +Arguments: [c_custkey#0L], [o_custkey#57L], Inner + +(11) CometProject +Input [4]: [c_custkey#0L, c_nationkey#3L, o_orderkey#56L, o_custkey#57L] +Arguments: [c_nationkey#3L, o_orderkey#56L], [c_nationkey#3L, o_orderkey#56L] + +(12) CometExchange +Input [2]: [c_nationkey#3L, o_orderkey#56L] +Arguments: hashpartitioning(o_orderkey#56L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=420] + +(13) CometSort +Input [2]: [c_nationkey#3L, o_orderkey#56L] +Arguments: [c_nationkey#3L, o_orderkey#56L], [o_orderkey#56L ASC NULLS FIRST] + +(14) CometNativeScan parquet +Output [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] +PushedFilters: [IsNotNull(l_orderkey), IsNotNull(l_suppkey)] +ReadSchema: struct + +(15) CometFilter +Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Condition : (isnotnull(l_orderkey#16L) AND isnotnull(l_suppkey#18L)) + +(16) CometExchange +Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: hashpartitioning(l_orderkey#16L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=424] + +(17) CometSort +Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22], [l_orderkey#16L ASC NULLS FIRST] + +(18) CometSortMergeJoin +Left output [2]: [c_nationkey#3L, o_orderkey#56L] +Right output [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: [o_orderkey#56L], [l_orderkey#16L], Inner + +(19) CometProject +Input [6]: [c_nationkey#3L, o_orderkey#56L, l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22], [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] + +(20) CometExchange +Input [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: hashpartitioning(l_suppkey#18L, c_nationkey#3L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=428] + +(21) CometSort +Input [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22], [l_suppkey#18L ASC NULLS FIRST, c_nationkey#3L ASC NULLS FIRST] + +(22) CometNativeScan parquet +Output [2]: [s_suppkey#108L, s_nationkey#111L] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/supplier] +PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] +ReadSchema: struct + +(23) CometFilter +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Condition : (isnotnull(s_suppkey#108L) AND isnotnull(s_nationkey#111L)) + +(24) CometExchange +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: hashpartitioning(s_suppkey#108L, s_nationkey#111L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=432] + +(25) CometSort +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: [s_suppkey#108L, s_nationkey#111L], [s_suppkey#108L ASC NULLS FIRST, s_nationkey#111L ASC NULLS FIRST] + +(26) CometSortMergeJoin +Left output [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Right output [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: [l_suppkey#18L, c_nationkey#3L], [s_suppkey#108L, s_nationkey#111L], Inner + +(27) CometProject +Input [6]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22, s_suppkey#108L, s_nationkey#111L] +Arguments: [l_extendedprice#21, l_discount#22, s_nationkey#111L], [l_extendedprice#21, l_discount#22, s_nationkey#111L] + +(28) CometNativeScan parquet +Output [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/nation] +PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] +ReadSchema: struct + +(29) CometFilter +Input [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] +Condition : ((isnotnull(n_nationkey#48L) AND isnotnull(n_regionkey#50L)) AND might_contain(Subquery subquery#128, [id=#161], xxhash64(n_regionkey#50L, 42))) + +(30) CometBroadcastExchange +Input [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] +Arguments: [n_nationkey#48L, n_name#49, n_regionkey#50L] + +(31) CometBroadcastHashJoin +Left output [3]: [l_extendedprice#21, l_discount#22, s_nationkey#111L] +Right output [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] +Arguments: [s_nationkey#111L], [n_nationkey#48L], Inner, BuildRight + +(32) CometProject +Input [6]: [l_extendedprice#21, l_discount#22, s_nationkey#111L, n_nationkey#48L, n_name#49, n_regionkey#50L] +Arguments: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L], [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L] + +(33) CometNativeScan parquet +Output [2]: [r_regionkey#102L, r_name#103] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/region] +PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AFRICA), IsNotNull(r_regionkey)] +ReadSchema: struct + +(34) CometFilter +Input [2]: [r_regionkey#102L, r_name#103] +Condition : ((isnotnull(r_name#103) AND (r_name#103 = AFRICA)) AND isnotnull(r_regionkey#102L)) + +(35) CometProject +Input [2]: [r_regionkey#102L, r_name#103] +Arguments: [r_regionkey#102L], [r_regionkey#102L] + +(36) CometBroadcastExchange +Input [1]: [r_regionkey#102L] +Arguments: [r_regionkey#102L] + +(37) CometBroadcastHashJoin +Left output [4]: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L] +Right output [1]: [r_regionkey#102L] +Arguments: [n_regionkey#50L], [r_regionkey#102L], Inner, BuildRight + +(38) CometProject +Input [5]: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L, r_regionkey#102L] +Arguments: [l_extendedprice#21, l_discount#22, n_name#49], [l_extendedprice#21, l_discount#22, n_name#49] + +(39) CometHashAggregate +Input [3]: [l_extendedprice#21, l_discount#22, n_name#49] +Keys [1]: [n_name#49] +Functions [1]: [partial_sum((l_extendedprice#21 * (1 - l_discount#22)))] + +(40) CometExchange +Input [3]: [n_name#49, sum#131, isEmpty#132] +Arguments: hashpartitioning(n_name#49, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=445] + +(41) CometHashAggregate +Input [3]: [n_name#49, sum#131, isEmpty#132] +Keys [1]: [n_name#49] +Functions [1]: [sum((l_extendedprice#21 * (1 - l_discount#22)))] + +(42) CometExchange +Input [2]: [n_name#49, revenue#122] +Arguments: rangepartitioning(revenue#122 DESC NULLS LAST, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=447] + +(43) CometSort +Input [2]: [n_name#49, revenue#122] +Arguments: [n_name#49, revenue#122], [revenue#122 DESC NULLS LAST] + +(44) AdaptiveSparkPlan +Output [2]: [n_name#49, revenue#122] +Arguments: isFinalPlan=false + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 29 Hosting Expression = Subquery subquery#128, [id=#161] +AdaptiveSparkPlan (51) ++- CometHashAggregate (50) + +- CometExchange (49) + +- CometHashAggregate (48) + +- CometProject (47) + +- CometFilter (46) + +- CometNativeScan parquet (45) + + +(45) CometNativeScan parquet +Output [2]: [r_regionkey#102L, r_name#103] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/region] +PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AFRICA), IsNotNull(r_regionkey)] +ReadSchema: struct + +(46) CometFilter +Input [2]: [r_regionkey#102L, r_name#103] +Condition : ((isnotnull(r_name#103) AND (r_name#103 = AFRICA)) AND isnotnull(r_regionkey#102L)) + +(47) CometProject +Input [2]: [r_regionkey#102L, r_name#103] +Arguments: [r_regionkey#102L], [r_regionkey#102L] + +(48) CometHashAggregate +Input [1]: [r_regionkey#102L] +Keys: [] +Functions [1]: [partial_bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)] + +(49) CometExchange +Input [1]: [buf#134] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=159] + +(50) CometHashAggregate +Input [1]: [buf#134] +Keys: [] +Functions [1]: [bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)] + +(51) AdaptiveSparkPlan +Output [1]: [bloomFilter#127] +Arguments: isFinalPlan=false + + + +Query 5 returned 5 rows, hash=f1e68aab4aa9d0988709357fc210d775 +Query 5 took 482.58 seconds + +Iteration 1 took 482.58 seconds + +Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/comet-offheap4g-q5-tpch-1775045606478.json diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.time b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.time new file mode 100644 index 0000000000..e5c7eae40a --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.time @@ -0,0 +1,5639 @@ +26/04/01 06:05:19 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) +26/04/01 06:05:19 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address +26/04/01 06:05:19 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +26/04/01 06:05:19 INFO SparkContext: Running Spark version 3.5.8 +26/04/01 06:05:19 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 06:05:19 INFO SparkContext: Java version 17.0.17 +26/04/01 06:05:19 INFO ResourceUtils: ============================================================== +26/04/01 06:05:19 INFO ResourceUtils: No custom resources configured for spark.driver. +26/04/01 06:05:19 INFO ResourceUtils: ============================================================== +26/04/01 06:05:19 INFO SparkContext: Submitted application: comet-offheap4g-q5 benchmark derived from tpch +26/04/01 06:05:19 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 4096, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) +26/04/01 06:05:19 INFO ResourceProfile: Limiting resource is cpu +26/04/01 06:05:19 INFO ResourceProfileManager: Added ResourceProfile id: 0 +26/04/01 06:05:19 INFO SecurityManager: Changing view acls to: andy +26/04/01 06:05:19 INFO SecurityManager: Changing modify acls to: andy +26/04/01 06:05:19 INFO SecurityManager: Changing view acls groups to: +26/04/01 06:05:19 INFO SecurityManager: Changing modify acls groups to: +26/04/01 06:05:19 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY +26/04/01 06:05:19 INFO Utils: Successfully started service 'sparkDriver' on port 58257. +26/04/01 06:05:19 INFO SparkEnv: Registering MapOutputTracker +26/04/01 06:05:19 INFO SparkEnv: Registering BlockManagerMaster +26/04/01 06:05:19 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information +26/04/01 06:05:19 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up +26/04/01 06:05:19 INFO SparkEnv: Registering BlockManagerMasterHeartbeat +26/04/01 06:05:19 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-833f884d-78cd-4dcb-b53f-b68f98c95068 +26/04/01 06:05:19 INFO MemoryStore: MemoryStore started with capacity 8.6 GiB +26/04/01 06:05:19 INFO SparkEnv: Registering OutputCommitCoordinator +26/04/01 06:05:19 INFO SparkContext: Added JAR file:///Users/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar at spark://10.0.0.133:58257/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775045119743 +26/04/01 06:05:19 INFO CometDriverPlugin: CometDriverPlugin init +26/04/01 06:05:19 INFO CometDriverPlugin: Setting spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions +26/04/01 06:05:19 INFO CometDriverPlugin: Comet metrics reporting is disabled. Set spark.comet.metrics.enabled=true to enable. +26/04/01 06:05:19 INFO CometDriverPlugin: Comet is running in unified memory mode and sharing off-heap memory with Spark +26/04/01 06:05:19 INFO DriverPluginContainer: Initialized driver component for plugin org.apache.spark.CometPlugin. +26/04/01 06:05:19 INFO Executor: Starting executor ID driver on host 10.0.0.133 +26/04/01 06:05:19 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 06:05:19 INFO Executor: Java version 17.0.17 +26/04/01 06:05:19 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' +26/04/01 06:05:19 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@656aa134 for default. +26/04/01 06:05:19 INFO Executor: Fetching spark://10.0.0.133:58257/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775045119743 +26/04/01 06:05:19 INFO TransportClientFactory: Successfully created connection to /10.0.0.133:58257 after 8 ms (0 ms spent in bootstraps) +26/04/01 06:05:19 INFO Utils: Fetching spark://10.0.0.133:58257/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-73a4ef89-b8ea-48f7-92a1-8593182866f3/userFiles-f8334189-3236-49b1-b1d8-d36506822e30/fetchFileTemp14484242616185578193.tmp +26/04/01 06:05:20 INFO Executor: Adding file:/private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-73a4ef89-b8ea-48f7-92a1-8593182866f3/userFiles-f8334189-3236-49b1-b1d8-d36506822e30/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to class loader default +26/04/01 06:05:20 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 58259. +26/04/01 06:05:20 INFO NettyBlockTransferService: Server created on 10.0.0.133:58259 +26/04/01 06:05:20 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy +26/04/01 06:05:20 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 58259, None) +26/04/01 06:05:20 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:58259 with 8.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 58259, None) +26/04/01 06:05:20 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 58259, None) +26/04/01 06:05:20 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 58259, None) +26/04/01 06:05:20 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. +26/04/01 06:05:20 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. +26/04/01 06:05:20 INFO InMemoryFileIndex: It took 17 ms to list leaf files for 1 paths. +26/04/01 06:05:20 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:05:20 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:05:20 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:05:20 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:05:20 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:20 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:05:20 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 06:05:20 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) +26/04/01 06:05:20 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:20 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:20 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:05:20 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 +26/04/01 06:05:20 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 06:05:20 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) +26/04/01 06:05:20 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 2041 bytes result sent to driver +26/04/01 06:05:20 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 112 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:05:20 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool +26/04/01 06:05:20 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.390 s +26/04/01 06:05:20 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:05:20 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished +26/04/01 06:05:20 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.401075 s +26/04/01 06:05:21 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:23 INFO NativeBase: Couldn't locate log file from either COMET_CONF_DIR or comet.log.file.path. Using default log configuration with INFO log level which emits to stderr +26/04/01 06:05:23 INFO core/src/lib.rs: Comet native library version 0.15.0 initialized +26/04/01 06:05:23 INFO NativeBase: Setting system property arrow.enable_unsafe_memory_access to true +26/04/01 06:05:23 INFO NativeBase: Setting system property arrow.enable_null_check_for_get to false +26/04/01 06:05:23 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. +26/04/01 06:05:23 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:05:23 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:05:23 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:05:23 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:05:23 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:23 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:05:23 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 06:05:23 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) +26/04/01 06:05:23 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:23 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:23 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:05:23 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 +26/04/01 06:05:23 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 06:05:23 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) +26/04/01 06:05:23 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2430 bytes result sent to driver +26/04/01 06:05:23 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 15 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:05:23 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool +26/04/01 06:05:23 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.024 s +26/04/01 06:05:23 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:05:23 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished +26/04/01 06:05:23 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.024869 s +26/04/01 06:05:23 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. +26/04/01 06:05:23 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:05:23 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:05:23 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:05:23 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:05:23 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:23 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:05:23 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 06:05:23 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) +26/04/01 06:05:23 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:23 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:23 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:05:23 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 +26/04/01 06:05:23 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 06:05:23 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) +26/04/01 06:05:23 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver +26/04/01 06:05:23 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:05:23 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool +26/04/01 06:05:23 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 06:05:23 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:05:23 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished +26/04/01 06:05:23 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.014269 s +26/04/01 06:05:23 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 06:05:23 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:05:23 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:05:23 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:05:23 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:05:23 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:23 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:05:23 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 06:05:23 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) +26/04/01 06:05:23 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:23 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:23 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:05:23 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 +26/04/01 06:05:23 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 06:05:23 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) +26/04/01 06:05:23 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver +26/04/01 06:05:23 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:05:23 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool +26/04/01 06:05:23 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 06:05:23 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:05:23 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished +26/04/01 06:05:23 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.014762 s +26/04/01 06:05:23 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 06:05:23 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:05:23 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:05:23 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:05:23 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:05:23 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:23 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:05:23 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 06:05:23 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) +26/04/01 06:05:23 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:23 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:23 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:23 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:05:23 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 +26/04/01 06:05:23 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9299 bytes) +26/04/01 06:05:23 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:23 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) +26/04/01 06:05:23 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:23 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver +26/04/01 06:05:23 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:05:23 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool +26/04/01 06:05:23 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 06:05:23 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:05:23 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished +26/04/01 06:05:23 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.014726 s +26/04/01 06:05:23 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 06:05:23 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:05:23 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:05:23 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:05:23 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:05:23 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:23 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:05:23 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 06:05:23 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) +26/04/01 06:05:23 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:23 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:23 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:23 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:05:23 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 +26/04/01 06:05:23 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 06:05:23 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) +26/04/01 06:05:23 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver +26/04/01 06:05:23 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:05:23 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool +26/04/01 06:05:23 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 06:05:23 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:05:23 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished +26/04/01 06:05:23 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.014210 s +26/04/01 06:05:23 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. +26/04/01 06:05:23 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:05:23 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:05:23 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:05:23 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:05:23 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:23 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:05:23 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 06:05:23 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) +26/04/01 06:05:23 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:23 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:23 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:05:23 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 +26/04/01 06:05:23 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 06:05:23 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) +26/04/01 06:05:23 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver +26/04/01 06:05:23 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 4 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:05:23 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool +26/04/01 06:05:23 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.010 s +26/04/01 06:05:23 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:05:23 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished +26/04/01 06:05:23 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.011783 s +26/04/01 06:05:23 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 06:05:23 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:05:23 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:05:23 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:05:23 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:05:23 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:23 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:05:23 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 06:05:23 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) +26/04/01 06:05:23 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:23 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:23 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:05:23 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 +26/04/01 06:05:23 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 06:05:23 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) +26/04/01 06:05:23 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver +26/04/01 06:05:23 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:05:23 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool +26/04/01 06:05:23 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.011 s +26/04/01 06:05:23 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:05:23 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished +26/04/01 06:05:23 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.012717 s +26/04/01 06:05:24 INFO FileSourceStrategy: Pushed Filters: IsNotNull(c_custkey),IsNotNull(c_nationkey) +26/04/01 06:05:24 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(c_custkey#0L),isnotnull(c_nationkey#3L) +26/04/01 06:05:24 INFO FileSourceStrategy: Pushed Filters: IsNotNull(o_orderdate),GreaterThanOrEqual(o_orderdate,1994-01-01),LessThan(o_orderdate,1995-01-01),IsNotNull(o_custkey),IsNotNull(o_orderkey) +26/04/01 06:05:24 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(o_orderdate#60),(o_orderdate#60 >= 1994-01-01),(o_orderdate#60 < 1995-01-01),isnotnull(o_custkey#57L),isnotnull(o_orderkey#56L) +26/04/01 06:05:24 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_orderkey),IsNotNull(l_suppkey) +26/04/01 06:05:24 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_orderkey#16L),isnotnull(l_suppkey#18L) +26/04/01 06:05:24 INFO FileSourceStrategy: Pushed Filters: IsNotNull(s_suppkey),IsNotNull(s_nationkey) +26/04/01 06:05:24 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(s_suppkey#108L),isnotnull(s_nationkey#111L) +26/04/01 06:05:24 INFO FileSourceStrategy: Pushed Filters: IsNotNull(n_nationkey),IsNotNull(n_regionkey) +26/04/01 06:05:24 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(n_nationkey#48L),isnotnull(n_regionkey#50L),might_contain(scalar-subquery#128 [], xxhash64(n_regionkey#50L, 42)) +26/04/01 06:05:24 INFO FileSourceStrategy: Pushed Filters: IsNotNull(r_name),EqualTo(r_name,AFRICA),IsNotNull(r_regionkey) +26/04/01 06:05:24 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(r_name#103),(r_name#103 = AFRICA),isnotnull(r_regionkey#102L) +26/04/01 06:05:24 INFO FileSourceStrategy: Pushed Filters: IsNotNull(r_name),EqualTo(r_name,AFRICA),IsNotNull(r_regionkey) +26/04/01 06:05:24 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(r_name#103),(r_name#103 = AFRICA),isnotnull(r_regionkey#102L) +26/04/01 06:05:24 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 55920126 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:58259 (size: 35.4 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:05:24 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO DAGScheduler: Registering RDD 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 +26/04/01 06:05:24 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:05:24 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:24 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 23.0 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 8.0 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:58259 (size: 8.0 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO SparkContext: Created broadcast 9 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:24 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 8.0 with 1 tasks resource profile 0 +26/04/01 06:05:24 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9260 bytes) +26/04/01 06:05:24 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) +26/04/01 06:05:24 INFO BaseAllocator: Debug mode disabled. Enable with the VM option -Darrow.memory.debug.allocator=true. +26/04/01 06:05:24 INFO DefaultAllocationManagerOption: allocation manager type not specified, using netty as the default type +26/04/01 06:05:24 INFO CheckAllocator: Using DefaultAllocationManager at memory/unsafe/DefaultAllocationManagerFactory.class +26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:24 INFO core/src/execution/jni_api.rs: Comet tokio runtime: using spark.executor.cores=4 worker threads +26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:24 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 6628 bytes result sent to driver +26/04/01 06:05:24 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 292 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:05:24 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool +26/04/01 06:05:24 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.306 s +26/04/01 06:05:24 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:05:24 INFO DAGScheduler: running: Set() +26/04/01 06:05:24 INFO DAGScheduler: waiting: Set() +26/04/01 06:05:24 INFO DAGScheduler: failed: Set() +26/04/01 06:05:24 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:05:24 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:05:24 INFO DAGScheduler: Final stage: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 9) +26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:24 INFO DAGScheduler: Submitting ResultStage 10 (MapPartitionsRDD[22] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 17.6 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 7.5 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:58259 (size: 7.5 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO SparkContext: Created broadcast 10 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:24 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 10 (MapPartitionsRDD[22] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 10.0 with 1 tasks resource profile 0 +26/04/01 06:05:24 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 9) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) +26/04/01 06:05:24 INFO Executor: Running task 0.0 in stage 10.0 (TID 9) +26/04/01 06:05:24 INFO ShuffleBlockFetcherIterator: Getting 1 (4.7 KiB) non-empty blocks including 1 (4.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:05:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 3 ms +26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:24 INFO Executor: Finished task 0.0 in stage 10.0 (TID 9). 9966 bytes result sent to driver +26/04/01 06:05:24 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 9) in 39 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:05:24 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool +26/04/01 06:05:24 INFO DAGScheduler: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.045 s +26/04/01 06:05:24 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:05:24 INFO TaskSchedulerImpl: Killing all running tasks in stage 10: Stage finished +26/04/01 06:05:24 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.049825 s +26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_13 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:58259 (size: 35.4 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO SparkContext: Created broadcast 11 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_13_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:58259 (size: 35.4 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO SparkContext: Created broadcast 12 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_13_piece0 in memory on 10.0.0.133:58259 (size: 35.4 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO SparkContext: Created broadcast 13 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:05:24 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO DAGScheduler: Registering RDD 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 +26/04/01 06:05:24 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 16 output partitions +26/04/01 06:05:24 INFO DAGScheduler: Final stage: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:24 INFO DAGScheduler: Submitting ShuffleMapStage 11 (MapPartitionsRDD[31] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_14 stored as values in memory (estimated size 17.6 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_14_piece0 stored as bytes in memory (estimated size 7.8 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_14_piece0 in memory on 10.0.0.133:58259 (size: 7.8 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO SparkContext: Created broadcast 14 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:24 INFO DAGScheduler: Submitting 16 missing tasks from ShuffleMapStage 11 (MapPartitionsRDD[31] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 11.0 with 16 tasks resource profile 0 +26/04/01 06:05:24 INFO DAGScheduler: Got job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:05:24 INFO DAGScheduler: Final stage: ResultStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:24 INFO TaskSetManager: Starting task 0.0 in stage 11.0 (TID 10) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:05:24 INFO DAGScheduler: Submitting ResultStage 12 (MapPartitionsRDD[26] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:05:24 INFO TaskSetManager: Starting task 1.0 in stage 11.0 (TID 11) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9334 bytes) +26/04/01 06:05:24 INFO TaskSetManager: Starting task 2.0 in stage 11.0 (TID 12) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:05:24 INFO TaskSetManager: Starting task 3.0 in stage 11.0 (TID 13) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9334 bytes) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_15 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO Executor: Running task 0.0 in stage 11.0 (TID 10) +26/04/01 06:05:24 INFO Executor: Running task 1.0 in stage 11.0 (TID 11) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_16 stored as values in memory (estimated size 14.0 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO Executor: Running task 2.0 in stage 11.0 (TID 12) +26/04/01 06:05:24 INFO Executor: Running task 3.0 in stage 11.0 (TID 13) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_16_piece0 stored as bytes in memory (estimated size 5.4 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_16_piece0 in memory on 10.0.0.133:58259 (size: 5.4 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO SparkContext: Created broadcast 16 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:24 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 12 (MapPartitionsRDD[26] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 12.0 with 1 tasks resource profile 0 +26/04/01 06:05:24 INFO BlockManagerInfo: Removed broadcast_10_piece0 on 10.0.0.133:58259 in memory (size: 7.5 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Removed broadcast_9_piece0 on 10.0.0.133:58259 in memory (size: 8.0 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_15_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_15_piece0 in memory on 10.0.0.133:58259 (size: 35.4 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO SparkContext: Created broadcast 15 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:05:24 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:24 INFO DAGScheduler: Registering RDD 34 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 2 +26/04/01 06:05:24 INFO DAGScheduler: Got map stage job 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 64 output partitions +26/04/01 06:05:24 INFO DAGScheduler: Final stage: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:24 INFO DAGScheduler: Submitting ShuffleMapStage 13 (MapPartitionsRDD[34] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_17 stored as values in memory (estimated size 23.3 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_17_piece0 stored as bytes in memory (estimated size 8.1 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_17_piece0 in memory on 10.0.0.133:58259 (size: 8.1 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO SparkContext: Created broadcast 17 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:24 INFO DAGScheduler: Submitting 64 missing tasks from ShuffleMapStage 13 (MapPartitionsRDD[34] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 13.0 with 64 tasks resource profile 0 +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_18 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_18_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_18_piece0 in memory on 10.0.0.133:58259 (size: 35.4 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO SparkContext: Created broadcast 18 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:05:24 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO DAGScheduler: Registering RDD 37 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 3 +26/04/01 06:05:24 INFO DAGScheduler: Got map stage job 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions +26/04/01 06:05:24 INFO DAGScheduler: Final stage: ShuffleMapStage 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:24 INFO DAGScheduler: Submitting ShuffleMapStage 14 (MapPartitionsRDD[37] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 55920126 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_19 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_20 stored as values in memory (estimated size 18.4 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_20_piece0 stored as bytes in memory (estimated size 8.0 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_20_piece0 in memory on 10.0.0.133:58259 (size: 8.0 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO SparkContext: Created broadcast 20 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:24 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 14 (MapPartitionsRDD[37] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 14.0 with 208 tasks resource profile 0 +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_19_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_19_piece0 in memory on 10.0.0.133:58259 (size: 35.4 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO SparkContext: Created broadcast 19 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:05:24 INFO FileSourceScanExec: Planning scan with bin packing, max size: 55920126 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:05:24 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:05:24 INFO DAGScheduler: Got job 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:05:24 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:24 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:05:24 WARN DAGScheduler: Broadcasting large task binary with size 1113.7 KiB +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_21 stored as values in memory (estimated size 1113.7 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_21_piece0 stored as bytes in memory (estimated size 30.7 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_21_piece0 in memory on 10.0.0.133:58259 (size: 30.7 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO SparkContext: Created broadcast 21 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:24 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 15.0 with 1 tasks resource profile 0 +26/04/01 06:05:24 INFO DAGScheduler: Registering RDD 40 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 4 +26/04/01 06:05:24 INFO DAGScheduler: Got map stage job 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 4 output partitions +26/04/01 06:05:24 INFO DAGScheduler: Final stage: ShuffleMapStage 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:24 INFO DAGScheduler: Submitting ShuffleMapStage 16 (MapPartitionsRDD[40] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_22 stored as values in memory (estimated size 17.6 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO MemoryStore: Block broadcast_22_piece0 stored as bytes in memory (estimated size 7.8 KiB, free 8.6 GiB) +26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_22_piece0 in memory on 10.0.0.133:58259 (size: 7.8 KiB, free: 8.6 GiB) +26/04/01 06:05:24 INFO SparkContext: Created broadcast 22 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:24 INFO DAGScheduler: Submitting 4 missing tasks from ShuffleMapStage 16 (MapPartitionsRDD[40] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3)) +26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 16.0 with 4 tasks resource profile 0 +26/04/01 06:05:25 INFO Executor: Finished task 3.0 in stage 11.0 (TID 13). 6649 bytes result sent to driver +26/04/01 06:05:25 INFO Executor: Finished task 1.0 in stage 11.0 (TID 11). 6649 bytes result sent to driver +26/04/01 06:05:25 INFO TaskSetManager: Starting task 4.0 in stage 11.0 (TID 14) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:05:25 INFO TaskSetManager: Finished task 3.0 in stage 11.0 (TID 13) in 746 ms on 10.0.0.133 (executor driver) (1/16) +26/04/01 06:05:25 INFO Executor: Running task 4.0 in stage 11.0 (TID 14) +26/04/01 06:05:25 INFO Executor: Finished task 2.0 in stage 11.0 (TID 12). 6649 bytes result sent to driver +26/04/01 06:05:25 INFO TaskSetManager: Starting task 5.0 in stage 11.0 (TID 15) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:05:25 INFO Executor: Running task 5.0 in stage 11.0 (TID 15) +26/04/01 06:05:25 INFO TaskSetManager: Starting task 6.0 in stage 11.0 (TID 16) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:05:25 INFO Executor: Running task 6.0 in stage 11.0 (TID 16) +26/04/01 06:05:25 INFO Executor: Finished task 0.0 in stage 11.0 (TID 10). 6649 bytes result sent to driver +26/04/01 06:05:25 INFO TaskSetManager: Starting task 7.0 in stage 11.0 (TID 17) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9334 bytes) +26/04/01 06:05:25 INFO TaskSetManager: Finished task 1.0 in stage 11.0 (TID 11) in 747 ms on 10.0.0.133 (executor driver) (2/16) +26/04/01 06:05:25 INFO Executor: Running task 7.0 in stage 11.0 (TID 17) +26/04/01 06:05:25 INFO TaskSetManager: Finished task 2.0 in stage 11.0 (TID 12) in 748 ms on 10.0.0.133 (executor driver) (3/16) +26/04/01 06:05:25 INFO TaskSetManager: Finished task 0.0 in stage 11.0 (TID 10) in 748 ms on 10.0.0.133 (executor driver) (4/16) +26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO Executor: Finished task 6.0 in stage 11.0 (TID 16). 6606 bytes result sent to driver +26/04/01 06:05:26 INFO TaskSetManager: Starting task 8.0 in stage 11.0 (TID 18) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:05:26 INFO Executor: Running task 8.0 in stage 11.0 (TID 18) +26/04/01 06:05:26 INFO TaskSetManager: Finished task 6.0 in stage 11.0 (TID 16) in 717 ms on 10.0.0.133 (executor driver) (5/16) +26/04/01 06:05:26 INFO Executor: Finished task 4.0 in stage 11.0 (TID 14). 6606 bytes result sent to driver +26/04/01 06:05:26 INFO TaskSetManager: Starting task 9.0 in stage 11.0 (TID 19) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:05:26 INFO Executor: Finished task 5.0 in stage 11.0 (TID 15). 6606 bytes result sent to driver +26/04/01 06:05:26 INFO Executor: Running task 9.0 in stage 11.0 (TID 19) +26/04/01 06:05:26 INFO TaskSetManager: Starting task 10.0 in stage 11.0 (TID 20) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9334 bytes) +26/04/01 06:05:26 INFO Executor: Running task 10.0 in stage 11.0 (TID 20) +26/04/01 06:05:26 INFO Executor: Finished task 7.0 in stage 11.0 (TID 17). 6606 bytes result sent to driver +26/04/01 06:05:26 INFO TaskSetManager: Finished task 5.0 in stage 11.0 (TID 15) in 718 ms on 10.0.0.133 (executor driver) (6/16) +26/04/01 06:05:26 INFO TaskSetManager: Finished task 4.0 in stage 11.0 (TID 14) in 719 ms on 10.0.0.133 (executor driver) (7/16) +26/04/01 06:05:26 INFO TaskSetManager: Starting task 11.0 in stage 11.0 (TID 21) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO Executor: Running task 11.0 in stage 11.0 (TID 21) +26/04/01 06:05:26 INFO TaskSetManager: Finished task 7.0 in stage 11.0 (TID 17) in 718 ms on 10.0.0.133 (executor driver) (8/16) +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO Executor: Finished task 11.0 in stage 11.0 (TID 21). 6563 bytes result sent to driver +26/04/01 06:05:26 INFO Executor: Finished task 9.0 in stage 11.0 (TID 19). 6606 bytes result sent to driver +26/04/01 06:05:26 INFO Executor: Finished task 8.0 in stage 11.0 (TID 18). 6606 bytes result sent to driver +26/04/01 06:05:26 INFO TaskSetManager: Starting task 12.0 in stage 11.0 (TID 22) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9334 bytes) +26/04/01 06:05:26 INFO Executor: Finished task 10.0 in stage 11.0 (TID 20). 6606 bytes result sent to driver +26/04/01 06:05:26 INFO Executor: Running task 12.0 in stage 11.0 (TID 22) +26/04/01 06:05:26 INFO TaskSetManager: Starting task 13.0 in stage 11.0 (TID 23) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:05:26 INFO Executor: Running task 13.0 in stage 11.0 (TID 23) +26/04/01 06:05:26 INFO TaskSetManager: Starting task 14.0 in stage 11.0 (TID 24) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9334 bytes) +26/04/01 06:05:26 INFO TaskSetManager: Finished task 11.0 in stage 11.0 (TID 21) in 717 ms on 10.0.0.133 (executor driver) (9/16) +26/04/01 06:05:26 INFO Executor: Running task 14.0 in stage 11.0 (TID 24) +26/04/01 06:05:26 INFO TaskSetManager: Finished task 8.0 in stage 11.0 (TID 18) in 719 ms on 10.0.0.133 (executor driver) (10/16) +26/04/01 06:05:26 INFO TaskSetManager: Finished task 9.0 in stage 11.0 (TID 19) in 718 ms on 10.0.0.133 (executor driver) (11/16) +26/04/01 06:05:26 INFO TaskSetManager: Starting task 15.0 in stage 11.0 (TID 25) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9334 bytes) +26/04/01 06:05:26 INFO TaskSetManager: Finished task 10.0 in stage 11.0 (TID 20) in 718 ms on 10.0.0.133 (executor driver) (12/16) +26/04/01 06:05:26 INFO Executor: Running task 15.0 in stage 11.0 (TID 25) +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:27 INFO Executor: Finished task 15.0 in stage 11.0 (TID 25). 6563 bytes result sent to driver +26/04/01 06:05:27 INFO Executor: Finished task 13.0 in stage 11.0 (TID 23). 6606 bytes result sent to driver +26/04/01 06:05:27 INFO Executor: Finished task 14.0 in stage 11.0 (TID 24). 6606 bytes result sent to driver +26/04/01 06:05:27 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 26) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9271 bytes) +26/04/01 06:05:27 INFO Executor: Running task 0.0 in stage 12.0 (TID 26) +26/04/01 06:05:27 INFO TaskSetManager: Starting task 0.0 in stage 13.0 (TID 27) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:27 INFO TaskSetManager: Finished task 15.0 in stage 11.0 (TID 25) in 712 ms on 10.0.0.133 (executor driver) (13/16) +26/04/01 06:05:27 INFO Executor: Running task 0.0 in stage 13.0 (TID 27) +26/04/01 06:05:27 INFO TaskSetManager: Finished task 13.0 in stage 11.0 (TID 23) in 713 ms on 10.0.0.133 (executor driver) (14/16) +26/04/01 06:05:27 INFO TaskSetManager: Starting task 1.0 in stage 13.0 (TID 28) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:05:27 INFO Executor: Running task 1.0 in stage 13.0 (TID 28) +26/04/01 06:05:27 INFO TaskSetManager: Finished task 14.0 in stage 11.0 (TID 24) in 713 ms on 10.0.0.133 (executor driver) (15/16) +26/04/01 06:05:27 INFO Executor: Finished task 12.0 in stage 11.0 (TID 22). 6606 bytes result sent to driver +26/04/01 06:05:27 INFO TaskSetManager: Starting task 2.0 in stage 13.0 (TID 29) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:27 INFO Executor: Running task 2.0 in stage 13.0 (TID 29) +26/04/01 06:05:27 INFO TaskSetManager: Finished task 12.0 in stage 11.0 (TID 22) in 715 ms on 10.0.0.133 (executor driver) (16/16) +26/04/01 06:05:27 INFO TaskSchedulerImpl: Removed TaskSet 11.0, whose tasks have all completed, from pool +26/04/01 06:05:27 INFO DAGScheduler: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.897 s +26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:27 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:05:27 INFO DAGScheduler: running: Set(ResultStage 15, ResultStage 12, ShuffleMapStage 16, ShuffleMapStage 13, ShuffleMapStage 14) +26/04/01 06:05:27 INFO DAGScheduler: waiting: Set() +26/04/01 06:05:27 INFO DAGScheduler: failed: Set() +26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:27 INFO Executor: Finished task 0.0 in stage 12.0 (TID 26). 4405 bytes result sent to driver +26/04/01 06:05:27 INFO TaskSetManager: Starting task 3.0 in stage 13.0 (TID 30) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:27 INFO Executor: Running task 3.0 in stage 13.0 (TID 30) +26/04/01 06:05:27 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 26) in 32 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:05:27 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool +26/04/01 06:05:27 INFO DAGScheduler: ResultStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.924 s +26/04/01 06:05:27 INFO DAGScheduler: Job 11 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:05:27 INFO TaskSchedulerImpl: Killing all running tasks in stage 12: Stage finished +26/04/01 06:05:27 INFO DAGScheduler: Job 11 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 2.925689 s +26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:27 INFO Utils: Coalesced 1 broadcast batches into 1 (1 rows) +26/04/01 06:05:27 INFO MemoryStore: Block broadcast_23 stored as values in memory (estimated size 416.0 B, free 8.6 GiB) +26/04/01 06:05:27 INFO MemoryStore: Block broadcast_23_piece0 stored as bytes in memory (estimated size 367.0 B, free 8.6 GiB) +26/04/01 06:05:27 INFO BlockManagerInfo: Added broadcast_23_piece0 in memory on 10.0.0.133:58259 (size: 367.0 B, free: 8.6 GiB) +26/04/01 06:05:27 INFO SparkContext: Created broadcast 23 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:05:28 INFO Executor: Finished task 1.0 in stage 13.0 (TID 28). 6675 bytes result sent to driver +26/04/01 06:05:28 INFO TaskSetManager: Starting task 4.0 in stage 13.0 (TID 31) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:28 INFO TaskSetManager: Finished task 1.0 in stage 13.0 (TID 28) in 1289 ms on 10.0.0.133 (executor driver) (1/64) +26/04/01 06:05:28 INFO Executor: Running task 4.0 in stage 13.0 (TID 31) +26/04/01 06:05:28 INFO Executor: Finished task 0.0 in stage 13.0 (TID 27). 6675 bytes result sent to driver +26/04/01 06:05:28 INFO TaskSetManager: Starting task 5.0 in stage 13.0 (TID 32) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:05:28 INFO Executor: Finished task 2.0 in stage 13.0 (TID 29). 6675 bytes result sent to driver +26/04/01 06:05:28 INFO Executor: Running task 5.0 in stage 13.0 (TID 32) +26/04/01 06:05:28 INFO TaskSetManager: Starting task 6.0 in stage 13.0 (TID 33) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:28 INFO TaskSetManager: Finished task 0.0 in stage 13.0 (TID 27) in 1291 ms on 10.0.0.133 (executor driver) (2/64) +26/04/01 06:05:28 INFO Executor: Running task 6.0 in stage 13.0 (TID 33) +26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:28 INFO TaskSetManager: Finished task 2.0 in stage 13.0 (TID 29) in 1290 ms on 10.0.0.133 (executor driver) (3/64) +26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:28 INFO Executor: Finished task 3.0 in stage 13.0 (TID 30). 6675 bytes result sent to driver +26/04/01 06:05:28 INFO TaskSetManager: Starting task 7.0 in stage 13.0 (TID 34) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:28 INFO TaskSetManager: Finished task 3.0 in stage 13.0 (TID 30) in 1292 ms on 10.0.0.133 (executor driver) (4/64) +26/04/01 06:05:28 INFO Executor: Running task 7.0 in stage 13.0 (TID 34) +26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:30 INFO Executor: Finished task 5.0 in stage 13.0 (TID 32). 6675 bytes result sent to driver +26/04/01 06:05:30 INFO Executor: Finished task 4.0 in stage 13.0 (TID 31). 6675 bytes result sent to driver +26/04/01 06:05:30 INFO TaskSetManager: Starting task 8.0 in stage 13.0 (TID 35) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:05:30 INFO Executor: Running task 8.0 in stage 13.0 (TID 35) +26/04/01 06:05:30 INFO TaskSetManager: Starting task 9.0 in stage 13.0 (TID 36) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:30 INFO TaskSetManager: Finished task 5.0 in stage 13.0 (TID 32) in 1288 ms on 10.0.0.133 (executor driver) (5/64) +26/04/01 06:05:30 INFO TaskSetManager: Finished task 4.0 in stage 13.0 (TID 31) in 1289 ms on 10.0.0.133 (executor driver) (6/64) +26/04/01 06:05:30 INFO Executor: Running task 9.0 in stage 13.0 (TID 36) +26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:30 INFO Executor: Finished task 6.0 in stage 13.0 (TID 33). 6675 bytes result sent to driver +26/04/01 06:05:30 INFO TaskSetManager: Starting task 10.0 in stage 13.0 (TID 37) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:05:30 INFO TaskSetManager: Finished task 6.0 in stage 13.0 (TID 33) in 1289 ms on 10.0.0.133 (executor driver) (7/64) +26/04/01 06:05:30 INFO Executor: Running task 10.0 in stage 13.0 (TID 37) +26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:30 INFO Executor: Finished task 7.0 in stage 13.0 (TID 34). 6675 bytes result sent to driver +26/04/01 06:05:30 INFO TaskSetManager: Starting task 11.0 in stage 13.0 (TID 38) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:30 INFO TaskSetManager: Finished task 7.0 in stage 13.0 (TID 34) in 1291 ms on 10.0.0.133 (executor driver) (8/64) +26/04/01 06:05:30 INFO Executor: Running task 11.0 in stage 13.0 (TID 38) +26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:31 INFO Executor: Finished task 8.0 in stage 13.0 (TID 35). 6675 bytes result sent to driver +26/04/01 06:05:31 INFO Executor: Finished task 10.0 in stage 13.0 (TID 37). 6675 bytes result sent to driver +26/04/01 06:05:31 INFO TaskSetManager: Starting task 12.0 in stage 13.0 (TID 39) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:31 INFO TaskSetManager: Starting task 13.0 in stage 13.0 (TID 40) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:31 INFO Executor: Running task 12.0 in stage 13.0 (TID 39) +26/04/01 06:05:31 INFO TaskSetManager: Finished task 8.0 in stage 13.0 (TID 35) in 1293 ms on 10.0.0.133 (executor driver) (9/64) +26/04/01 06:05:31 INFO Executor: Running task 13.0 in stage 13.0 (TID 40) +26/04/01 06:05:31 INFO TaskSetManager: Finished task 10.0 in stage 13.0 (TID 37) in 1290 ms on 10.0.0.133 (executor driver) (10/64) +26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:31 INFO Executor: Finished task 9.0 in stage 13.0 (TID 36). 6675 bytes result sent to driver +26/04/01 06:05:31 INFO TaskSetManager: Starting task 14.0 in stage 13.0 (TID 41) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:31 INFO TaskSetManager: Finished task 9.0 in stage 13.0 (TID 36) in 1294 ms on 10.0.0.133 (executor driver) (11/64) +26/04/01 06:05:31 INFO Executor: Running task 14.0 in stage 13.0 (TID 41) +26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:31 INFO Executor: Finished task 11.0 in stage 13.0 (TID 38). 6675 bytes result sent to driver +26/04/01 06:05:31 INFO TaskSetManager: Starting task 15.0 in stage 13.0 (TID 42) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:31 INFO Executor: Running task 15.0 in stage 13.0 (TID 42) +26/04/01 06:05:31 INFO TaskSetManager: Finished task 11.0 in stage 13.0 (TID 38) in 1293 ms on 10.0.0.133 (executor driver) (12/64) +26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:32 INFO Executor: Finished task 14.0 in stage 13.0 (TID 41). 6718 bytes result sent to driver +26/04/01 06:05:32 INFO Executor: Finished task 12.0 in stage 13.0 (TID 39). 6718 bytes result sent to driver +26/04/01 06:05:32 INFO TaskSetManager: Starting task 16.0 in stage 13.0 (TID 43) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:05:32 INFO Executor: Finished task 13.0 in stage 13.0 (TID 40). 6718 bytes result sent to driver +26/04/01 06:05:32 INFO TaskSetManager: Finished task 14.0 in stage 13.0 (TID 41) in 1292 ms on 10.0.0.133 (executor driver) (13/64) +26/04/01 06:05:32 INFO Executor: Running task 16.0 in stage 13.0 (TID 43) +26/04/01 06:05:32 INFO TaskSetManager: Starting task 17.0 in stage 13.0 (TID 44) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:32 INFO Executor: Running task 17.0 in stage 13.0 (TID 44) +26/04/01 06:05:32 INFO TaskSetManager: Starting task 18.0 in stage 13.0 (TID 45) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:05:32 INFO TaskSetManager: Finished task 12.0 in stage 13.0 (TID 39) in 1294 ms on 10.0.0.133 (executor driver) (14/64) +26/04/01 06:05:32 INFO Executor: Running task 18.0 in stage 13.0 (TID 45) +26/04/01 06:05:32 INFO TaskSetManager: Finished task 13.0 in stage 13.0 (TID 40) in 1294 ms on 10.0.0.133 (executor driver) (15/64) +26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:32 INFO Executor: Finished task 15.0 in stage 13.0 (TID 42). 6718 bytes result sent to driver +26/04/01 06:05:32 INFO TaskSetManager: Starting task 19.0 in stage 13.0 (TID 46) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:32 INFO TaskSetManager: Finished task 15.0 in stage 13.0 (TID 42) in 1289 ms on 10.0.0.133 (executor driver) (16/64) +26/04/01 06:05:32 INFO Executor: Running task 19.0 in stage 13.0 (TID 46) +26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:34 INFO Executor: Finished task 18.0 in stage 13.0 (TID 45). 6675 bytes result sent to driver +26/04/01 06:05:34 INFO Executor: Finished task 16.0 in stage 13.0 (TID 43). 6675 bytes result sent to driver +26/04/01 06:05:34 INFO TaskSetManager: Starting task 20.0 in stage 13.0 (TID 47) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:34 INFO Executor: Running task 20.0 in stage 13.0 (TID 47) +26/04/01 06:05:34 INFO TaskSetManager: Starting task 21.0 in stage 13.0 (TID 48) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:05:34 INFO TaskSetManager: Finished task 18.0 in stage 13.0 (TID 45) in 1289 ms on 10.0.0.133 (executor driver) (17/64) +26/04/01 06:05:34 INFO Executor: Running task 21.0 in stage 13.0 (TID 48) +26/04/01 06:05:34 INFO TaskSetManager: Finished task 16.0 in stage 13.0 (TID 43) in 1291 ms on 10.0.0.133 (executor driver) (18/64) +26/04/01 06:05:34 INFO Executor: Finished task 17.0 in stage 13.0 (TID 44). 6675 bytes result sent to driver +26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:34 INFO TaskSetManager: Starting task 22.0 in stage 13.0 (TID 49) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:34 INFO TaskSetManager: Finished task 17.0 in stage 13.0 (TID 44) in 1291 ms on 10.0.0.133 (executor driver) (19/64) +26/04/01 06:05:34 INFO Executor: Running task 22.0 in stage 13.0 (TID 49) +26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:34 INFO Executor: Finished task 19.0 in stage 13.0 (TID 46). 6675 bytes result sent to driver +26/04/01 06:05:34 INFO TaskSetManager: Starting task 23.0 in stage 13.0 (TID 50) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:34 INFO TaskSetManager: Finished task 19.0 in stage 13.0 (TID 46) in 1288 ms on 10.0.0.133 (executor driver) (20/64) +26/04/01 06:05:34 INFO Executor: Running task 23.0 in stage 13.0 (TID 50) +26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:35 INFO Executor: Finished task 21.0 in stage 13.0 (TID 48). 6675 bytes result sent to driver +26/04/01 06:05:35 INFO TaskSetManager: Starting task 24.0 in stage 13.0 (TID 51) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:05:35 INFO TaskSetManager: Finished task 21.0 in stage 13.0 (TID 48) in 1287 ms on 10.0.0.133 (executor driver) (21/64) +26/04/01 06:05:35 INFO Executor: Running task 24.0 in stage 13.0 (TID 51) +26/04/01 06:05:35 INFO Executor: Finished task 20.0 in stage 13.0 (TID 47). 6675 bytes result sent to driver +26/04/01 06:05:35 INFO TaskSetManager: Starting task 25.0 in stage 13.0 (TID 52) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:35 INFO TaskSetManager: Finished task 20.0 in stage 13.0 (TID 47) in 1287 ms on 10.0.0.133 (executor driver) (22/64) +26/04/01 06:05:35 INFO Executor: Running task 25.0 in stage 13.0 (TID 52) +26/04/01 06:05:35 INFO Executor: Finished task 22.0 in stage 13.0 (TID 49). 6675 bytes result sent to driver +26/04/01 06:05:35 INFO TaskSetManager: Starting task 26.0 in stage 13.0 (TID 53) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:35 INFO TaskSetManager: Finished task 22.0 in stage 13.0 (TID 49) in 1286 ms on 10.0.0.133 (executor driver) (23/64) +26/04/01 06:05:35 INFO Executor: Running task 26.0 in stage 13.0 (TID 53) +26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:35 INFO Executor: Finished task 23.0 in stage 13.0 (TID 50). 6675 bytes result sent to driver +26/04/01 06:05:35 INFO TaskSetManager: Starting task 27.0 in stage 13.0 (TID 54) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:35 INFO TaskSetManager: Finished task 23.0 in stage 13.0 (TID 50) in 1289 ms on 10.0.0.133 (executor driver) (24/64) +26/04/01 06:05:35 INFO Executor: Running task 27.0 in stage 13.0 (TID 54) +26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:36 INFO Executor: Finished task 26.0 in stage 13.0 (TID 53). 6675 bytes result sent to driver +26/04/01 06:05:36 INFO TaskSetManager: Starting task 28.0 in stage 13.0 (TID 55) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:36 INFO Executor: Running task 28.0 in stage 13.0 (TID 55) +26/04/01 06:05:36 INFO TaskSetManager: Finished task 26.0 in stage 13.0 (TID 53) in 1287 ms on 10.0.0.133 (executor driver) (25/64) +26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:36 INFO Executor: Finished task 24.0 in stage 13.0 (TID 51). 6675 bytes result sent to driver +26/04/01 06:05:36 INFO TaskSetManager: Starting task 29.0 in stage 13.0 (TID 56) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:36 INFO Executor: Running task 29.0 in stage 13.0 (TID 56) +26/04/01 06:05:36 INFO TaskSetManager: Finished task 24.0 in stage 13.0 (TID 51) in 1290 ms on 10.0.0.133 (executor driver) (26/64) +26/04/01 06:05:36 INFO Executor: Finished task 25.0 in stage 13.0 (TID 52). 6675 bytes result sent to driver +26/04/01 06:05:36 INFO TaskSetManager: Starting task 30.0 in stage 13.0 (TID 57) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:36 INFO Executor: Running task 30.0 in stage 13.0 (TID 57) +26/04/01 06:05:36 INFO TaskSetManager: Finished task 25.0 in stage 13.0 (TID 52) in 1291 ms on 10.0.0.133 (executor driver) (27/64) +26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:36 INFO Executor: Finished task 27.0 in stage 13.0 (TID 54). 6675 bytes result sent to driver +26/04/01 06:05:36 INFO TaskSetManager: Starting task 31.0 in stage 13.0 (TID 58) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:05:36 INFO TaskSetManager: Finished task 27.0 in stage 13.0 (TID 54) in 1293 ms on 10.0.0.133 (executor driver) (28/64) +26/04/01 06:05:36 INFO Executor: Running task 31.0 in stage 13.0 (TID 58) +26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:37 INFO Executor: Finished task 30.0 in stage 13.0 (TID 57). 6675 bytes result sent to driver +26/04/01 06:05:37 INFO Executor: Finished task 28.0 in stage 13.0 (TID 55). 6675 bytes result sent to driver +26/04/01 06:05:37 INFO Executor: Finished task 29.0 in stage 13.0 (TID 56). 6675 bytes result sent to driver +26/04/01 06:05:37 INFO TaskSetManager: Starting task 32.0 in stage 13.0 (TID 59) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:37 INFO Executor: Running task 32.0 in stage 13.0 (TID 59) +26/04/01 06:05:37 INFO TaskSetManager: Starting task 33.0 in stage 13.0 (TID 60) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:37 INFO TaskSetManager: Finished task 30.0 in stage 13.0 (TID 57) in 1289 ms on 10.0.0.133 (executor driver) (29/64) +26/04/01 06:05:37 INFO Executor: Running task 33.0 in stage 13.0 (TID 60) +26/04/01 06:05:37 INFO TaskSetManager: Starting task 34.0 in stage 13.0 (TID 61) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:37 INFO TaskSetManager: Finished task 28.0 in stage 13.0 (TID 55) in 1292 ms on 10.0.0.133 (executor driver) (30/64) +26/04/01 06:05:37 INFO Executor: Running task 34.0 in stage 13.0 (TID 61) +26/04/01 06:05:37 INFO TaskSetManager: Finished task 29.0 in stage 13.0 (TID 56) in 1290 ms on 10.0.0.133 (executor driver) (31/64) +26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:37 INFO Executor: Finished task 31.0 in stage 13.0 (TID 58). 6675 bytes result sent to driver +26/04/01 06:05:37 INFO TaskSetManager: Starting task 35.0 in stage 13.0 (TID 62) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:05:37 INFO Executor: Running task 35.0 in stage 13.0 (TID 62) +26/04/01 06:05:37 INFO TaskSetManager: Finished task 31.0 in stage 13.0 (TID 58) in 1290 ms on 10.0.0.133 (executor driver) (32/64) +26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:38 INFO Executor: Finished task 33.0 in stage 13.0 (TID 60). 6675 bytes result sent to driver +26/04/01 06:05:38 INFO Executor: Finished task 32.0 in stage 13.0 (TID 59). 6675 bytes result sent to driver +26/04/01 06:05:38 INFO TaskSetManager: Starting task 36.0 in stage 13.0 (TID 63) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:38 INFO Executor: Running task 36.0 in stage 13.0 (TID 63) +26/04/01 06:05:38 INFO TaskSetManager: Starting task 37.0 in stage 13.0 (TID 64) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:38 INFO TaskSetManager: Finished task 33.0 in stage 13.0 (TID 60) in 920 ms on 10.0.0.133 (executor driver) (33/64) +26/04/01 06:05:38 INFO TaskSetManager: Finished task 32.0 in stage 13.0 (TID 59) in 920 ms on 10.0.0.133 (executor driver) (34/64) +26/04/01 06:05:38 INFO Executor: Running task 37.0 in stage 13.0 (TID 64) +26/04/01 06:05:38 INFO Executor: Finished task 34.0 in stage 13.0 (TID 61). 6675 bytes result sent to driver +26/04/01 06:05:38 INFO TaskSetManager: Starting task 38.0 in stage 13.0 (TID 65) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:38 INFO TaskSetManager: Finished task 34.0 in stage 13.0 (TID 61) in 919 ms on 10.0.0.133 (executor driver) (35/64) +26/04/01 06:05:38 INFO Executor: Running task 38.0 in stage 13.0 (TID 65) +26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:38 INFO Executor: Finished task 35.0 in stage 13.0 (TID 62). 6675 bytes result sent to driver +26/04/01 06:05:38 INFO TaskSetManager: Starting task 39.0 in stage 13.0 (TID 66) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:38 INFO TaskSetManager: Finished task 35.0 in stage 13.0 (TID 62) in 921 ms on 10.0.0.133 (executor driver) (36/64) +26/04/01 06:05:38 INFO Executor: Running task 39.0 in stage 13.0 (TID 66) +26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:39 INFO Executor: Finished task 37.0 in stage 13.0 (TID 64). 6675 bytes result sent to driver +26/04/01 06:05:39 INFO Executor: Finished task 38.0 in stage 13.0 (TID 65). 6675 bytes result sent to driver +26/04/01 06:05:39 INFO TaskSetManager: Starting task 40.0 in stage 13.0 (TID 67) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:05:39 INFO Executor: Running task 40.0 in stage 13.0 (TID 67) +26/04/01 06:05:39 INFO TaskSetManager: Starting task 41.0 in stage 13.0 (TID 68) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:39 INFO TaskSetManager: Finished task 37.0 in stage 13.0 (TID 64) in 925 ms on 10.0.0.133 (executor driver) (37/64) +26/04/01 06:05:39 INFO Executor: Running task 41.0 in stage 13.0 (TID 68) +26/04/01 06:05:39 INFO TaskSetManager: Finished task 38.0 in stage 13.0 (TID 65) in 924 ms on 10.0.0.133 (executor driver) (38/64) +26/04/01 06:05:39 INFO Executor: Finished task 36.0 in stage 13.0 (TID 63). 6675 bytes result sent to driver +26/04/01 06:05:39 INFO TaskSetManager: Starting task 42.0 in stage 13.0 (TID 69) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:39 INFO TaskSetManager: Finished task 36.0 in stage 13.0 (TID 63) in 926 ms on 10.0.0.133 (executor driver) (39/64) +26/04/01 06:05:39 INFO Executor: Running task 42.0 in stage 13.0 (TID 69) +26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:39 INFO Executor: Finished task 39.0 in stage 13.0 (TID 66). 6675 bytes result sent to driver +26/04/01 06:05:39 INFO TaskSetManager: Starting task 43.0 in stage 13.0 (TID 70) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:05:39 INFO Executor: Running task 43.0 in stage 13.0 (TID 70) +26/04/01 06:05:39 INFO TaskSetManager: Finished task 39.0 in stage 13.0 (TID 66) in 926 ms on 10.0.0.133 (executor driver) (40/64) +26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:40 INFO Executor: Finished task 41.0 in stage 13.0 (TID 68). 6675 bytes result sent to driver +26/04/01 06:05:40 INFO TaskSetManager: Starting task 44.0 in stage 13.0 (TID 71) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:05:40 INFO TaskSetManager: Finished task 41.0 in stage 13.0 (TID 68) in 921 ms on 10.0.0.133 (executor driver) (41/64) +26/04/01 06:05:40 INFO Executor: Running task 44.0 in stage 13.0 (TID 71) +26/04/01 06:05:40 INFO Executor: Finished task 40.0 in stage 13.0 (TID 67). 6675 bytes result sent to driver +26/04/01 06:05:40 INFO TaskSetManager: Starting task 45.0 in stage 13.0 (TID 72) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:40 INFO TaskSetManager: Finished task 40.0 in stage 13.0 (TID 67) in 922 ms on 10.0.0.133 (executor driver) (42/64) +26/04/01 06:05:40 INFO Executor: Running task 45.0 in stage 13.0 (TID 72) +26/04/01 06:05:40 INFO Executor: Finished task 42.0 in stage 13.0 (TID 69). 6675 bytes result sent to driver +26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:40 INFO TaskSetManager: Starting task 46.0 in stage 13.0 (TID 73) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:40 INFO Executor: Running task 46.0 in stage 13.0 (TID 73) +26/04/01 06:05:40 INFO TaskSetManager: Finished task 42.0 in stage 13.0 (TID 69) in 922 ms on 10.0.0.133 (executor driver) (43/64) +26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:40 INFO Executor: Finished task 43.0 in stage 13.0 (TID 70). 6675 bytes result sent to driver +26/04/01 06:05:40 INFO TaskSetManager: Starting task 47.0 in stage 13.0 (TID 74) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:05:40 INFO TaskSetManager: Finished task 43.0 in stage 13.0 (TID 70) in 923 ms on 10.0.0.133 (executor driver) (44/64) +26/04/01 06:05:40 INFO Executor: Running task 47.0 in stage 13.0 (TID 74) +26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:41 INFO Executor: Finished task 46.0 in stage 13.0 (TID 73). 6675 bytes result sent to driver +26/04/01 06:05:41 INFO Executor: Finished task 45.0 in stage 13.0 (TID 72). 6675 bytes result sent to driver +26/04/01 06:05:41 INFO Executor: Finished task 44.0 in stage 13.0 (TID 71). 6675 bytes result sent to driver +26/04/01 06:05:41 INFO TaskSetManager: Starting task 48.0 in stage 13.0 (TID 75) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:41 INFO Executor: Running task 48.0 in stage 13.0 (TID 75) +26/04/01 06:05:41 INFO TaskSetManager: Starting task 49.0 in stage 13.0 (TID 76) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:41 INFO TaskSetManager: Finished task 46.0 in stage 13.0 (TID 73) in 917 ms on 10.0.0.133 (executor driver) (45/64) +26/04/01 06:05:41 INFO Executor: Running task 49.0 in stage 13.0 (TID 76) +26/04/01 06:05:41 INFO TaskSetManager: Finished task 45.0 in stage 13.0 (TID 72) in 918 ms on 10.0.0.133 (executor driver) (46/64) +26/04/01 06:05:41 INFO TaskSetManager: Starting task 50.0 in stage 13.0 (TID 77) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:41 INFO TaskSetManager: Finished task 44.0 in stage 13.0 (TID 71) in 919 ms on 10.0.0.133 (executor driver) (47/64) +26/04/01 06:05:41 INFO Executor: Running task 50.0 in stage 13.0 (TID 77) +26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:41 INFO Executor: Finished task 47.0 in stage 13.0 (TID 74). 6675 bytes result sent to driver +26/04/01 06:05:41 INFO TaskSetManager: Starting task 51.0 in stage 13.0 (TID 78) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:41 INFO Executor: Running task 51.0 in stage 13.0 (TID 78) +26/04/01 06:05:41 INFO TaskSetManager: Finished task 47.0 in stage 13.0 (TID 74) in 920 ms on 10.0.0.133 (executor driver) (48/64) +26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:42 INFO Executor: Finished task 49.0 in stage 13.0 (TID 76). 6675 bytes result sent to driver +26/04/01 06:05:42 INFO Executor: Finished task 50.0 in stage 13.0 (TID 77). 6675 bytes result sent to driver +26/04/01 06:05:42 INFO TaskSetManager: Starting task 52.0 in stage 13.0 (TID 79) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:42 INFO Executor: Finished task 48.0 in stage 13.0 (TID 75). 6675 bytes result sent to driver +26/04/01 06:05:42 INFO Executor: Running task 52.0 in stage 13.0 (TID 79) +26/04/01 06:05:42 INFO TaskSetManager: Starting task 53.0 in stage 13.0 (TID 80) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:42 INFO TaskSetManager: Finished task 49.0 in stage 13.0 (TID 76) in 919 ms on 10.0.0.133 (executor driver) (49/64) +26/04/01 06:05:42 INFO Executor: Running task 53.0 in stage 13.0 (TID 80) +26/04/01 06:05:42 INFO TaskSetManager: Finished task 50.0 in stage 13.0 (TID 77) in 919 ms on 10.0.0.133 (executor driver) (50/64) +26/04/01 06:05:42 INFO TaskSetManager: Starting task 54.0 in stage 13.0 (TID 81) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:05:42 INFO TaskSetManager: Finished task 48.0 in stage 13.0 (TID 75) in 920 ms on 10.0.0.133 (executor driver) (51/64) +26/04/01 06:05:42 INFO Executor: Running task 54.0 in stage 13.0 (TID 81) +26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:42 INFO Executor: Finished task 51.0 in stage 13.0 (TID 78). 6675 bytes result sent to driver +26/04/01 06:05:42 INFO TaskSetManager: Starting task 55.0 in stage 13.0 (TID 82) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:42 INFO TaskSetManager: Finished task 51.0 in stage 13.0 (TID 78) in 917 ms on 10.0.0.133 (executor driver) (52/64) +26/04/01 06:05:42 INFO Executor: Running task 55.0 in stage 13.0 (TID 82) +26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:43 INFO Executor: Finished task 53.0 in stage 13.0 (TID 80). 6675 bytes result sent to driver +26/04/01 06:05:43 INFO TaskSetManager: Starting task 56.0 in stage 13.0 (TID 83) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:05:43 INFO Executor: Finished task 52.0 in stage 13.0 (TID 79). 6675 bytes result sent to driver +26/04/01 06:05:43 INFO TaskSetManager: Finished task 53.0 in stage 13.0 (TID 80) in 918 ms on 10.0.0.133 (executor driver) (53/64) +26/04/01 06:05:43 INFO Executor: Running task 56.0 in stage 13.0 (TID 83) +26/04/01 06:05:43 INFO TaskSetManager: Starting task 57.0 in stage 13.0 (TID 84) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:43 INFO TaskSetManager: Finished task 52.0 in stage 13.0 (TID 79) in 919 ms on 10.0.0.133 (executor driver) (54/64) +26/04/01 06:05:43 INFO Executor: Running task 57.0 in stage 13.0 (TID 84) +26/04/01 06:05:43 INFO Executor: Finished task 54.0 in stage 13.0 (TID 81). 6675 bytes result sent to driver +26/04/01 06:05:43 INFO TaskSetManager: Starting task 58.0 in stage 13.0 (TID 85) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:43 INFO TaskSetManager: Finished task 54.0 in stage 13.0 (TID 81) in 919 ms on 10.0.0.133 (executor driver) (55/64) +26/04/01 06:05:43 INFO Executor: Running task 58.0 in stage 13.0 (TID 85) +26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:43 INFO Executor: Finished task 55.0 in stage 13.0 (TID 82). 6675 bytes result sent to driver +26/04/01 06:05:43 INFO TaskSetManager: Starting task 59.0 in stage 13.0 (TID 86) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:43 INFO TaskSetManager: Finished task 55.0 in stage 13.0 (TID 82) in 918 ms on 10.0.0.133 (executor driver) (56/64) +26/04/01 06:05:43 INFO Executor: Running task 59.0 in stage 13.0 (TID 86) +26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:44 INFO Executor: Finished task 56.0 in stage 13.0 (TID 83). 6675 bytes result sent to driver +26/04/01 06:05:44 INFO Executor: Finished task 58.0 in stage 13.0 (TID 85). 6675 bytes result sent to driver +26/04/01 06:05:44 INFO TaskSetManager: Starting task 60.0 in stage 13.0 (TID 87) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:44 INFO Executor: Running task 60.0 in stage 13.0 (TID 87) +26/04/01 06:05:44 INFO TaskSetManager: Starting task 61.0 in stage 13.0 (TID 88) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:44 INFO TaskSetManager: Finished task 56.0 in stage 13.0 (TID 83) in 920 ms on 10.0.0.133 (executor driver) (57/64) +26/04/01 06:05:44 INFO Executor: Running task 61.0 in stage 13.0 (TID 88) +26/04/01 06:05:44 INFO TaskSetManager: Finished task 58.0 in stage 13.0 (TID 85) in 919 ms on 10.0.0.133 (executor driver) (58/64) +26/04/01 06:05:44 INFO Executor: Finished task 57.0 in stage 13.0 (TID 84). 6675 bytes result sent to driver +26/04/01 06:05:44 INFO TaskSetManager: Starting task 62.0 in stage 13.0 (TID 89) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:05:44 INFO TaskSetManager: Finished task 57.0 in stage 13.0 (TID 84) in 920 ms on 10.0.0.133 (executor driver) (59/64) +26/04/01 06:05:44 INFO Executor: Running task 62.0 in stage 13.0 (TID 89) +26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:44 INFO Executor: Finished task 59.0 in stage 13.0 (TID 86). 6675 bytes result sent to driver +26/04/01 06:05:44 INFO TaskSetManager: Starting task 63.0 in stage 13.0 (TID 90) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:05:44 INFO Executor: Running task 63.0 in stage 13.0 (TID 90) +26/04/01 06:05:44 INFO TaskSetManager: Finished task 59.0 in stage 13.0 (TID 86) in 919 ms on 10.0.0.133 (executor driver) (60/64) +26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:45 INFO Executor: Finished task 60.0 in stage 13.0 (TID 87). 6675 bytes result sent to driver +26/04/01 06:05:45 INFO Executor: Finished task 62.0 in stage 13.0 (TID 89). 6675 bytes result sent to driver +26/04/01 06:05:45 INFO TaskSetManager: Starting task 0.0 in stage 14.0 (TID 91) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:45 INFO Executor: Running task 0.0 in stage 14.0 (TID 91) +26/04/01 06:05:45 INFO TaskSetManager: Starting task 1.0 in stage 14.0 (TID 92) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:05:45 INFO TaskSetManager: Finished task 60.0 in stage 13.0 (TID 87) in 927 ms on 10.0.0.133 (executor driver) (61/64) +26/04/01 06:05:45 INFO Executor: Running task 1.0 in stage 14.0 (TID 92) +26/04/01 06:05:45 INFO TaskSetManager: Finished task 62.0 in stage 13.0 (TID 89) in 926 ms on 10.0.0.133 (executor driver) (62/64) +26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:45 INFO Executor: Finished task 61.0 in stage 13.0 (TID 88). 6675 bytes result sent to driver +26/04/01 06:05:45 INFO TaskSetManager: Starting task 2.0 in stage 14.0 (TID 93) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:05:45 INFO TaskSetManager: Finished task 61.0 in stage 13.0 (TID 88) in 928 ms on 10.0.0.133 (executor driver) (63/64) +26/04/01 06:05:45 INFO Executor: Running task 2.0 in stage 14.0 (TID 93) +26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:45 INFO Executor: Finished task 63.0 in stage 13.0 (TID 90). 6675 bytes result sent to driver +26/04/01 06:05:45 INFO TaskSetManager: Starting task 3.0 in stage 14.0 (TID 94) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:05:45 INFO Executor: Running task 3.0 in stage 14.0 (TID 94) +26/04/01 06:05:45 INFO TaskSetManager: Finished task 63.0 in stage 13.0 (TID 90) in 925 ms on 10.0.0.133 (executor driver) (64/64) +26/04/01 06:05:45 INFO TaskSchedulerImpl: Removed TaskSet 13.0, whose tasks have all completed, from pool +26/04/01 06:05:45 INFO DAGScheduler: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 20.593 s +26/04/01 06:05:45 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:05:45 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 16, ShuffleMapStage 14) +26/04/01 06:05:45 INFO DAGScheduler: waiting: Set() +26/04/01 06:05:45 INFO DAGScheduler: failed: Set() +26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:45 INFO ShufflePartitionsUtil: For shuffle(1, 2), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 06:05:45 INFO DAGScheduler: Registering RDD 44 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 5 +26/04/01 06:05:45 INFO DAGScheduler: Got map stage job 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 6 output partitions +26/04/01 06:05:45 INFO DAGScheduler: Final stage: ShuffleMapStage 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:05:45 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 17, ShuffleMapStage 18) +26/04/01 06:05:45 INFO DAGScheduler: Missing parents: List() +26/04/01 06:05:45 INFO DAGScheduler: Submitting ShuffleMapStage 19 (MapPartitionsRDD[44] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:05:45 INFO MemoryStore: Block broadcast_24 stored as values in memory (estimated size 22.7 KiB, free 8.6 GiB) +26/04/01 06:05:45 INFO MemoryStore: Block broadcast_24_piece0 stored as bytes in memory (estimated size 9.5 KiB, free 8.6 GiB) +26/04/01 06:05:45 INFO BlockManagerInfo: Added broadcast_24_piece0 in memory on 10.0.0.133:58259 (size: 9.5 KiB, free: 8.6 GiB) +26/04/01 06:05:45 INFO SparkContext: Created broadcast 24 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:05:45 INFO DAGScheduler: Submitting 6 missing tasks from ShuffleMapStage 19 (MapPartitionsRDD[44] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5)) +26/04/01 06:05:45 INFO TaskSchedulerImpl: Adding task set 19.0 with 6 tasks resource profile 0 +26/04/01 06:05:49 INFO Executor: Finished task 0.0 in stage 14.0 (TID 91). 6563 bytes result sent to driver +26/04/01 06:05:49 INFO TaskSetManager: Starting task 4.0 in stage 14.0 (TID 95) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:05:49 INFO Executor: Running task 4.0 in stage 14.0 (TID 95) +26/04/01 06:05:49 INFO TaskSetManager: Finished task 0.0 in stage 14.0 (TID 91) in 4156 ms on 10.0.0.133 (executor driver) (1/208) +26/04/01 06:05:49 INFO Executor: Finished task 2.0 in stage 14.0 (TID 93). 6563 bytes result sent to driver +26/04/01 06:05:49 INFO TaskSetManager: Starting task 5.0 in stage 14.0 (TID 96) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:05:49 INFO TaskSetManager: Finished task 2.0 in stage 14.0 (TID 93) in 4154 ms on 10.0.0.133 (executor driver) (2/208) +26/04/01 06:05:49 INFO Executor: Running task 5.0 in stage 14.0 (TID 96) +26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:49 INFO Executor: Finished task 1.0 in stage 14.0 (TID 92). 6563 bytes result sent to driver +26/04/01 06:05:49 INFO TaskSetManager: Starting task 6.0 in stage 14.0 (TID 97) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:49 INFO TaskSetManager: Finished task 1.0 in stage 14.0 (TID 92) in 4160 ms on 10.0.0.133 (executor driver) (3/208) +26/04/01 06:05:49 INFO Executor: Running task 6.0 in stage 14.0 (TID 97) +26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:49 INFO Executor: Finished task 3.0 in stage 14.0 (TID 94). 6563 bytes result sent to driver +26/04/01 06:05:49 INFO TaskSetManager: Starting task 7.0 in stage 14.0 (TID 98) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:05:49 INFO TaskSetManager: Finished task 3.0 in stage 14.0 (TID 94) in 4150 ms on 10.0.0.133 (executor driver) (4/208) +26/04/01 06:05:49 INFO Executor: Running task 7.0 in stage 14.0 (TID 98) +26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:53 INFO Executor: Finished task 4.0 in stage 14.0 (TID 95). 6563 bytes result sent to driver +26/04/01 06:05:53 INFO TaskSetManager: Starting task 8.0 in stage 14.0 (TID 99) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:05:53 INFO TaskSetManager: Finished task 4.0 in stage 14.0 (TID 95) in 4143 ms on 10.0.0.133 (executor driver) (5/208) +26/04/01 06:05:53 INFO Executor: Running task 8.0 in stage 14.0 (TID 99) +26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:53 INFO Executor: Finished task 5.0 in stage 14.0 (TID 96). 6563 bytes result sent to driver +26/04/01 06:05:53 INFO TaskSetManager: Starting task 9.0 in stage 14.0 (TID 100) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:05:53 INFO TaskSetManager: Finished task 5.0 in stage 14.0 (TID 96) in 4144 ms on 10.0.0.133 (executor driver) (6/208) +26/04/01 06:05:53 INFO Executor: Running task 9.0 in stage 14.0 (TID 100) +26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:53 INFO Executor: Finished task 6.0 in stage 14.0 (TID 97). 6563 bytes result sent to driver +26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:53 INFO TaskSetManager: Starting task 10.0 in stage 14.0 (TID 101) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:05:53 INFO TaskSetManager: Finished task 6.0 in stage 14.0 (TID 97) in 4141 ms on 10.0.0.133 (executor driver) (7/208) +26/04/01 06:05:53 INFO Executor: Running task 10.0 in stage 14.0 (TID 101) +26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:53 INFO Executor: Finished task 7.0 in stage 14.0 (TID 98). 6563 bytes result sent to driver +26/04/01 06:05:53 INFO TaskSetManager: Starting task 11.0 in stage 14.0 (TID 102) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:05:53 INFO TaskSetManager: Finished task 7.0 in stage 14.0 (TID 98) in 4124 ms on 10.0.0.133 (executor driver) (8/208) +26/04/01 06:05:53 INFO Executor: Running task 11.0 in stage 14.0 (TID 102) +26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:57 INFO Executor: Finished task 9.0 in stage 14.0 (TID 100). 6563 bytes result sent to driver +26/04/01 06:05:57 INFO TaskSetManager: Starting task 12.0 in stage 14.0 (TID 103) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:05:57 INFO TaskSetManager: Finished task 9.0 in stage 14.0 (TID 100) in 4150 ms on 10.0.0.133 (executor driver) (9/208) +26/04/01 06:05:57 INFO Executor: Running task 12.0 in stage 14.0 (TID 103) +26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:57 INFO Executor: Finished task 8.0 in stage 14.0 (TID 99). 6563 bytes result sent to driver +26/04/01 06:05:57 INFO TaskSetManager: Starting task 13.0 in stage 14.0 (TID 104) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:05:57 INFO TaskSetManager: Finished task 8.0 in stage 14.0 (TID 99) in 4156 ms on 10.0.0.133 (executor driver) (10/208) +26/04/01 06:05:57 INFO Executor: Running task 13.0 in stage 14.0 (TID 104) +26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:57 INFO Executor: Finished task 10.0 in stage 14.0 (TID 101). 6563 bytes result sent to driver +26/04/01 06:05:57 INFO TaskSetManager: Starting task 14.0 in stage 14.0 (TID 105) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:05:57 INFO Executor: Finished task 11.0 in stage 14.0 (TID 102). 6563 bytes result sent to driver +26/04/01 06:05:57 INFO Executor: Running task 14.0 in stage 14.0 (TID 105) +26/04/01 06:05:57 INFO TaskSetManager: Finished task 10.0 in stage 14.0 (TID 101) in 4162 ms on 10.0.0.133 (executor driver) (11/208) +26/04/01 06:05:57 INFO TaskSetManager: Starting task 15.0 in stage 14.0 (TID 106) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:05:57 INFO TaskSetManager: Finished task 11.0 in stage 14.0 (TID 102) in 4154 ms on 10.0.0.133 (executor driver) (12/208) +26/04/01 06:05:57 INFO Executor: Running task 15.0 in stage 14.0 (TID 106) +26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:01 INFO Executor: Finished task 14.0 in stage 14.0 (TID 105). 6563 bytes result sent to driver +26/04/01 06:06:01 INFO TaskSetManager: Starting task 16.0 in stage 14.0 (TID 107) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:01 INFO TaskSetManager: Finished task 14.0 in stage 14.0 (TID 105) in 4123 ms on 10.0.0.133 (executor driver) (13/208) +26/04/01 06:06:01 INFO Executor: Running task 16.0 in stage 14.0 (TID 107) +26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:01 INFO Executor: Finished task 12.0 in stage 14.0 (TID 103). 6563 bytes result sent to driver +26/04/01 06:06:01 INFO TaskSetManager: Starting task 17.0 in stage 14.0 (TID 108) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:01 INFO Executor: Running task 17.0 in stage 14.0 (TID 108) +26/04/01 06:06:01 INFO TaskSetManager: Finished task 12.0 in stage 14.0 (TID 103) in 4143 ms on 10.0.0.133 (executor driver) (14/208) +26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:01 INFO Executor: Finished task 13.0 in stage 14.0 (TID 104). 6563 bytes result sent to driver +26/04/01 06:06:01 INFO TaskSetManager: Starting task 18.0 in stage 14.0 (TID 109) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:06:01 INFO TaskSetManager: Finished task 13.0 in stage 14.0 (TID 104) in 4141 ms on 10.0.0.133 (executor driver) (15/208) +26/04/01 06:06:01 INFO Executor: Running task 18.0 in stage 14.0 (TID 109) +26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:01 INFO Executor: Finished task 15.0 in stage 14.0 (TID 106). 6563 bytes result sent to driver +26/04/01 06:06:01 INFO TaskSetManager: Starting task 19.0 in stage 14.0 (TID 110) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:06:01 INFO Executor: Running task 19.0 in stage 14.0 (TID 110) +26/04/01 06:06:01 INFO TaskSetManager: Finished task 15.0 in stage 14.0 (TID 106) in 4135 ms on 10.0.0.133 (executor driver) (16/208) +26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:06 INFO Executor: Finished task 18.0 in stage 14.0 (TID 109). 6563 bytes result sent to driver +26/04/01 06:06:06 INFO TaskSetManager: Starting task 20.0 in stage 14.0 (TID 111) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:06 INFO TaskSetManager: Finished task 18.0 in stage 14.0 (TID 109) in 4134 ms on 10.0.0.133 (executor driver) (17/208) +26/04/01 06:06:06 INFO Executor: Running task 20.0 in stage 14.0 (TID 111) +26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:06 INFO Executor: Finished task 16.0 in stage 14.0 (TID 107). 6563 bytes result sent to driver +26/04/01 06:06:06 INFO TaskSetManager: Starting task 21.0 in stage 14.0 (TID 112) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:06 INFO Executor: Running task 21.0 in stage 14.0 (TID 112) +26/04/01 06:06:06 INFO TaskSetManager: Finished task 16.0 in stage 14.0 (TID 107) in 4146 ms on 10.0.0.133 (executor driver) (18/208) +26/04/01 06:06:06 INFO Executor: Finished task 17.0 in stage 14.0 (TID 108). 6563 bytes result sent to driver +26/04/01 06:06:06 INFO TaskSetManager: Starting task 22.0 in stage 14.0 (TID 113) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:06 INFO Executor: Running task 22.0 in stage 14.0 (TID 113) +26/04/01 06:06:06 INFO TaskSetManager: Finished task 17.0 in stage 14.0 (TID 108) in 4142 ms on 10.0.0.133 (executor driver) (19/208) +26/04/01 06:06:06 INFO Executor: Finished task 19.0 in stage 14.0 (TID 110). 6563 bytes result sent to driver +26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:06 INFO TaskSetManager: Starting task 23.0 in stage 14.0 (TID 114) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:06 INFO TaskSetManager: Finished task 19.0 in stage 14.0 (TID 110) in 4137 ms on 10.0.0.133 (executor driver) (20/208) +26/04/01 06:06:06 INFO Executor: Running task 23.0 in stage 14.0 (TID 114) +26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:10 INFO Executor: Finished task 20.0 in stage 14.0 (TID 111). 6563 bytes result sent to driver +26/04/01 06:06:10 INFO Executor: Finished task 22.0 in stage 14.0 (TID 113). 6563 bytes result sent to driver +26/04/01 06:06:10 INFO TaskSetManager: Starting task 24.0 in stage 14.0 (TID 115) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:06:10 INFO Executor: Running task 24.0 in stage 14.0 (TID 115) +26/04/01 06:06:10 INFO TaskSetManager: Starting task 25.0 in stage 14.0 (TID 116) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:10 INFO TaskSetManager: Finished task 20.0 in stage 14.0 (TID 111) in 4146 ms on 10.0.0.133 (executor driver) (21/208) +26/04/01 06:06:10 INFO Executor: Running task 25.0 in stage 14.0 (TID 116) +26/04/01 06:06:10 INFO TaskSetManager: Finished task 22.0 in stage 14.0 (TID 113) in 4140 ms on 10.0.0.133 (executor driver) (22/208) +26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:10 INFO Executor: Finished task 23.0 in stage 14.0 (TID 114). 6563 bytes result sent to driver +26/04/01 06:06:10 INFO TaskSetManager: Starting task 26.0 in stage 14.0 (TID 117) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:10 INFO TaskSetManager: Finished task 23.0 in stage 14.0 (TID 114) in 4141 ms on 10.0.0.133 (executor driver) (23/208) +26/04/01 06:06:10 INFO Executor: Running task 26.0 in stage 14.0 (TID 117) +26/04/01 06:06:10 INFO Executor: Finished task 21.0 in stage 14.0 (TID 112). 6563 bytes result sent to driver +26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:10 INFO TaskSetManager: Starting task 27.0 in stage 14.0 (TID 118) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:10 INFO TaskSetManager: Finished task 21.0 in stage 14.0 (TID 112) in 4143 ms on 10.0.0.133 (executor driver) (24/208) +26/04/01 06:06:10 INFO Executor: Running task 27.0 in stage 14.0 (TID 118) +26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:14 INFO Executor: Finished task 27.0 in stage 14.0 (TID 118). 6563 bytes result sent to driver +26/04/01 06:06:14 INFO TaskSetManager: Starting task 28.0 in stage 14.0 (TID 119) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:14 INFO TaskSetManager: Finished task 27.0 in stage 14.0 (TID 118) in 4091 ms on 10.0.0.133 (executor driver) (25/208) +26/04/01 06:06:14 INFO Executor: Running task 28.0 in stage 14.0 (TID 119) +26/04/01 06:06:14 INFO Executor: Finished task 26.0 in stage 14.0 (TID 117). 6563 bytes result sent to driver +26/04/01 06:06:14 INFO TaskSetManager: Starting task 29.0 in stage 14.0 (TID 120) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:14 INFO TaskSetManager: Finished task 26.0 in stage 14.0 (TID 117) in 4093 ms on 10.0.0.133 (executor driver) (26/208) +26/04/01 06:06:14 INFO Executor: Running task 29.0 in stage 14.0 (TID 120) +26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:14 INFO Executor: Finished task 25.0 in stage 14.0 (TID 116). 6563 bytes result sent to driver +26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:14 INFO TaskSetManager: Starting task 30.0 in stage 14.0 (TID 121) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:06:14 INFO Executor: Finished task 24.0 in stage 14.0 (TID 115). 6563 bytes result sent to driver +26/04/01 06:06:14 INFO Executor: Running task 30.0 in stage 14.0 (TID 121) +26/04/01 06:06:14 INFO TaskSetManager: Starting task 31.0 in stage 14.0 (TID 122) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:14 INFO TaskSetManager: Finished task 25.0 in stage 14.0 (TID 116) in 4095 ms on 10.0.0.133 (executor driver) (27/208) +26/04/01 06:06:14 INFO TaskSetManager: Finished task 24.0 in stage 14.0 (TID 115) in 4096 ms on 10.0.0.133 (executor driver) (28/208) +26/04/01 06:06:14 INFO Executor: Running task 31.0 in stage 14.0 (TID 122) +26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:14 INFO BlockManagerInfo: Removed broadcast_17_piece0 on 10.0.0.133:58259 in memory (size: 8.1 KiB, free: 8.6 GiB) +26/04/01 06:06:18 INFO Executor: Finished task 28.0 in stage 14.0 (TID 119). 6606 bytes result sent to driver +26/04/01 06:06:18 INFO TaskSetManager: Starting task 32.0 in stage 14.0 (TID 123) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:18 INFO Executor: Finished task 31.0 in stage 14.0 (TID 122). 6606 bytes result sent to driver +26/04/01 06:06:18 INFO Executor: Running task 32.0 in stage 14.0 (TID 123) +26/04/01 06:06:18 INFO TaskSetManager: Starting task 33.0 in stage 14.0 (TID 124) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:18 INFO Executor: Finished task 30.0 in stage 14.0 (TID 121). 6606 bytes result sent to driver +26/04/01 06:06:18 INFO TaskSetManager: Finished task 28.0 in stage 14.0 (TID 119) in 4088 ms on 10.0.0.133 (executor driver) (29/208) +26/04/01 06:06:18 INFO Executor: Running task 33.0 in stage 14.0 (TID 124) +26/04/01 06:06:18 INFO TaskSetManager: Finished task 31.0 in stage 14.0 (TID 122) in 4086 ms on 10.0.0.133 (executor driver) (30/208) +26/04/01 06:06:18 INFO TaskSetManager: Starting task 34.0 in stage 14.0 (TID 125) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:18 INFO TaskSetManager: Finished task 30.0 in stage 14.0 (TID 121) in 4086 ms on 10.0.0.133 (executor driver) (31/208) +26/04/01 06:06:18 INFO Executor: Running task 34.0 in stage 14.0 (TID 125) +26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:18 INFO Executor: Finished task 29.0 in stage 14.0 (TID 120). 6606 bytes result sent to driver +26/04/01 06:06:18 INFO TaskSetManager: Starting task 35.0 in stage 14.0 (TID 126) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:18 INFO TaskSetManager: Finished task 29.0 in stage 14.0 (TID 120) in 4093 ms on 10.0.0.133 (executor driver) (32/208) +26/04/01 06:06:18 INFO Executor: Running task 35.0 in stage 14.0 (TID 126) +26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:22 INFO Executor: Finished task 33.0 in stage 14.0 (TID 124). 6563 bytes result sent to driver +26/04/01 06:06:22 INFO Executor: Finished task 32.0 in stage 14.0 (TID 123). 6563 bytes result sent to driver +26/04/01 06:06:22 INFO TaskSetManager: Starting task 36.0 in stage 14.0 (TID 127) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:06:22 INFO TaskSetManager: Finished task 33.0 in stage 14.0 (TID 124) in 4094 ms on 10.0.0.133 (executor driver) (33/208) +26/04/01 06:06:22 INFO Executor: Running task 36.0 in stage 14.0 (TID 127) +26/04/01 06:06:22 INFO TaskSetManager: Starting task 37.0 in stage 14.0 (TID 128) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:22 INFO Executor: Running task 37.0 in stage 14.0 (TID 128) +26/04/01 06:06:22 INFO TaskSetManager: Finished task 32.0 in stage 14.0 (TID 123) in 4094 ms on 10.0.0.133 (executor driver) (34/208) +26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:22 INFO Executor: Finished task 34.0 in stage 14.0 (TID 125). 6563 bytes result sent to driver +26/04/01 06:06:22 INFO Executor: Finished task 35.0 in stage 14.0 (TID 126). 6563 bytes result sent to driver +26/04/01 06:06:22 INFO TaskSetManager: Starting task 38.0 in stage 14.0 (TID 129) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:22 INFO Executor: Running task 38.0 in stage 14.0 (TID 129) +26/04/01 06:06:22 INFO TaskSetManager: Finished task 34.0 in stage 14.0 (TID 125) in 4100 ms on 10.0.0.133 (executor driver) (35/208) +26/04/01 06:06:22 INFO TaskSetManager: Starting task 39.0 in stage 14.0 (TID 130) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:22 INFO Executor: Running task 39.0 in stage 14.0 (TID 130) +26/04/01 06:06:22 INFO TaskSetManager: Finished task 35.0 in stage 14.0 (TID 126) in 4094 ms on 10.0.0.133 (executor driver) (36/208) +26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:26 INFO Executor: Finished task 38.0 in stage 14.0 (TID 129). 6563 bytes result sent to driver +26/04/01 06:06:26 INFO Executor: Finished task 36.0 in stage 14.0 (TID 127). 6563 bytes result sent to driver +26/04/01 06:06:26 INFO TaskSetManager: Starting task 40.0 in stage 14.0 (TID 131) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:26 INFO Executor: Running task 40.0 in stage 14.0 (TID 131) +26/04/01 06:06:26 INFO TaskSetManager: Finished task 38.0 in stage 14.0 (TID 129) in 4154 ms on 10.0.0.133 (executor driver) (37/208) +26/04/01 06:06:26 INFO Executor: Finished task 37.0 in stage 14.0 (TID 128). 6563 bytes result sent to driver +26/04/01 06:06:26 INFO TaskSetManager: Starting task 41.0 in stage 14.0 (TID 132) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:26 INFO TaskSetManager: Finished task 36.0 in stage 14.0 (TID 127) in 4161 ms on 10.0.0.133 (executor driver) (38/208) +26/04/01 06:06:26 INFO Executor: Running task 41.0 in stage 14.0 (TID 132) +26/04/01 06:06:26 INFO TaskSetManager: Starting task 42.0 in stage 14.0 (TID 133) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:06:26 INFO TaskSetManager: Finished task 37.0 in stage 14.0 (TID 128) in 4161 ms on 10.0.0.133 (executor driver) (39/208) +26/04/01 06:06:26 INFO Executor: Running task 42.0 in stage 14.0 (TID 133) +26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:26 INFO Executor: Finished task 39.0 in stage 14.0 (TID 130). 6563 bytes result sent to driver +26/04/01 06:06:26 INFO TaskSetManager: Starting task 43.0 in stage 14.0 (TID 134) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:06:26 INFO TaskSetManager: Finished task 39.0 in stage 14.0 (TID 130) in 4165 ms on 10.0.0.133 (executor driver) (40/208) +26/04/01 06:06:26 INFO Executor: Running task 43.0 in stage 14.0 (TID 134) +26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:30 INFO Executor: Finished task 42.0 in stage 14.0 (TID 133). 6563 bytes result sent to driver +26/04/01 06:06:30 INFO TaskSetManager: Starting task 44.0 in stage 14.0 (TID 135) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:30 INFO TaskSetManager: Finished task 42.0 in stage 14.0 (TID 133) in 4088 ms on 10.0.0.133 (executor driver) (41/208) +26/04/01 06:06:30 INFO Executor: Running task 44.0 in stage 14.0 (TID 135) +26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:30 INFO Executor: Finished task 43.0 in stage 14.0 (TID 134). 6563 bytes result sent to driver +26/04/01 06:06:30 INFO TaskSetManager: Starting task 45.0 in stage 14.0 (TID 136) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:30 INFO Executor: Running task 45.0 in stage 14.0 (TID 136) +26/04/01 06:06:30 INFO TaskSetManager: Finished task 43.0 in stage 14.0 (TID 134) in 4083 ms on 10.0.0.133 (executor driver) (42/208) +26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:30 INFO Executor: Finished task 41.0 in stage 14.0 (TID 132). 6563 bytes result sent to driver +26/04/01 06:06:30 INFO TaskSetManager: Starting task 46.0 in stage 14.0 (TID 137) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:30 INFO TaskSetManager: Finished task 41.0 in stage 14.0 (TID 132) in 4137 ms on 10.0.0.133 (executor driver) (43/208) +26/04/01 06:06:30 INFO Executor: Running task 46.0 in stage 14.0 (TID 137) +26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:30 INFO Executor: Finished task 40.0 in stage 14.0 (TID 131). 6563 bytes result sent to driver +26/04/01 06:06:30 INFO TaskSetManager: Starting task 47.0 in stage 14.0 (TID 138) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:30 INFO TaskSetManager: Finished task 40.0 in stage 14.0 (TID 131) in 4153 ms on 10.0.0.133 (executor driver) (44/208) +26/04/01 06:06:30 INFO Executor: Running task 47.0 in stage 14.0 (TID 138) +26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:34 INFO Executor: Finished task 45.0 in stage 14.0 (TID 136). 6563 bytes result sent to driver +26/04/01 06:06:34 INFO TaskSetManager: Starting task 48.0 in stage 14.0 (TID 139) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:06:34 INFO Executor: Running task 48.0 in stage 14.0 (TID 139) +26/04/01 06:06:34 INFO TaskSetManager: Finished task 45.0 in stage 14.0 (TID 136) in 3932 ms on 10.0.0.133 (executor driver) (45/208) +26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:34 INFO Executor: Finished task 46.0 in stage 14.0 (TID 137). 6563 bytes result sent to driver +26/04/01 06:06:34 INFO TaskSetManager: Starting task 49.0 in stage 14.0 (TID 140) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:34 INFO TaskSetManager: Finished task 46.0 in stage 14.0 (TID 137) in 4089 ms on 10.0.0.133 (executor driver) (46/208) +26/04/01 06:06:34 INFO Executor: Running task 49.0 in stage 14.0 (TID 140) +26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:34 INFO Executor: Finished task 47.0 in stage 14.0 (TID 138). 6563 bytes result sent to driver +26/04/01 06:06:34 INFO TaskSetManager: Starting task 50.0 in stage 14.0 (TID 141) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:34 INFO Executor: Running task 50.0 in stage 14.0 (TID 141) +26/04/01 06:06:34 INFO TaskSetManager: Finished task 47.0 in stage 14.0 (TID 138) in 4078 ms on 10.0.0.133 (executor driver) (47/208) +26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:34 INFO Executor: Finished task 44.0 in stage 14.0 (TID 135). 6563 bytes result sent to driver +26/04/01 06:06:34 INFO TaskSetManager: Starting task 51.0 in stage 14.0 (TID 142) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:34 INFO TaskSetManager: Finished task 44.0 in stage 14.0 (TID 135) in 4240 ms on 10.0.0.133 (executor driver) (48/208) +26/04/01 06:06:34 INFO Executor: Running task 51.0 in stage 14.0 (TID 142) +26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:38 INFO Executor: Finished task 48.0 in stage 14.0 (TID 139). 6563 bytes result sent to driver +26/04/01 06:06:38 INFO TaskSetManager: Starting task 52.0 in stage 14.0 (TID 143) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:38 INFO TaskSetManager: Finished task 48.0 in stage 14.0 (TID 139) in 4162 ms on 10.0.0.133 (executor driver) (49/208) +26/04/01 06:06:38 INFO Executor: Running task 52.0 in stage 14.0 (TID 143) +26/04/01 06:06:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:38 INFO Executor: Finished task 49.0 in stage 14.0 (TID 140). 6563 bytes result sent to driver +26/04/01 06:06:38 INFO TaskSetManager: Starting task 53.0 in stage 14.0 (TID 144) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:38 INFO TaskSetManager: Finished task 49.0 in stage 14.0 (TID 140) in 4146 ms on 10.0.0.133 (executor driver) (50/208) +26/04/01 06:06:38 INFO Executor: Running task 53.0 in stage 14.0 (TID 144) +26/04/01 06:06:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:38 INFO Executor: Finished task 50.0 in stage 14.0 (TID 141). 6563 bytes result sent to driver +26/04/01 06:06:38 INFO TaskSetManager: Starting task 54.0 in stage 14.0 (TID 145) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:06:38 INFO TaskSetManager: Finished task 50.0 in stage 14.0 (TID 141) in 4156 ms on 10.0.0.133 (executor driver) (51/208) +26/04/01 06:06:38 INFO Executor: Running task 54.0 in stage 14.0 (TID 145) +26/04/01 06:06:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:39 INFO Executor: Finished task 51.0 in stage 14.0 (TID 142). 6563 bytes result sent to driver +26/04/01 06:06:39 INFO TaskSetManager: Starting task 55.0 in stage 14.0 (TID 146) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:39 INFO Executor: Running task 55.0 in stage 14.0 (TID 146) +26/04/01 06:06:39 INFO TaskSetManager: Finished task 51.0 in stage 14.0 (TID 142) in 4157 ms on 10.0.0.133 (executor driver) (52/208) +26/04/01 06:06:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:42 INFO Executor: Finished task 52.0 in stage 14.0 (TID 143). 6563 bytes result sent to driver +26/04/01 06:06:42 INFO TaskSetManager: Starting task 56.0 in stage 14.0 (TID 147) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:42 INFO Executor: Running task 56.0 in stage 14.0 (TID 147) +26/04/01 06:06:42 INFO TaskSetManager: Finished task 52.0 in stage 14.0 (TID 143) in 4152 ms on 10.0.0.133 (executor driver) (53/208) +26/04/01 06:06:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:43 INFO Executor: Finished task 53.0 in stage 14.0 (TID 144). 6563 bytes result sent to driver +26/04/01 06:06:43 INFO TaskSetManager: Starting task 57.0 in stage 14.0 (TID 148) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:43 INFO TaskSetManager: Finished task 53.0 in stage 14.0 (TID 144) in 4134 ms on 10.0.0.133 (executor driver) (54/208) +26/04/01 06:06:43 INFO Executor: Running task 57.0 in stage 14.0 (TID 148) +26/04/01 06:06:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:43 INFO Executor: Finished task 54.0 in stage 14.0 (TID 145). 6563 bytes result sent to driver +26/04/01 06:06:43 INFO TaskSetManager: Starting task 58.0 in stage 14.0 (TID 149) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:43 INFO TaskSetManager: Finished task 54.0 in stage 14.0 (TID 145) in 4140 ms on 10.0.0.133 (executor driver) (55/208) +26/04/01 06:06:43 INFO Executor: Running task 58.0 in stage 14.0 (TID 149) +26/04/01 06:06:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:43 INFO Executor: Finished task 55.0 in stage 14.0 (TID 146). 6563 bytes result sent to driver +26/04/01 06:06:43 INFO TaskSetManager: Starting task 59.0 in stage 14.0 (TID 150) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:43 INFO TaskSetManager: Finished task 55.0 in stage 14.0 (TID 146) in 4131 ms on 10.0.0.133 (executor driver) (56/208) +26/04/01 06:06:43 INFO Executor: Running task 59.0 in stage 14.0 (TID 150) +26/04/01 06:06:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:47 INFO Executor: Finished task 56.0 in stage 14.0 (TID 147). 6563 bytes result sent to driver +26/04/01 06:06:47 INFO TaskSetManager: Starting task 60.0 in stage 14.0 (TID 151) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:06:47 INFO TaskSetManager: Finished task 56.0 in stage 14.0 (TID 147) in 4143 ms on 10.0.0.133 (executor driver) (57/208) +26/04/01 06:06:47 INFO Executor: Running task 60.0 in stage 14.0 (TID 151) +26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:47 INFO Executor: Finished task 57.0 in stage 14.0 (TID 148). 6563 bytes result sent to driver +26/04/01 06:06:47 INFO TaskSetManager: Starting task 61.0 in stage 14.0 (TID 152) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:47 INFO TaskSetManager: Finished task 57.0 in stage 14.0 (TID 148) in 4150 ms on 10.0.0.133 (executor driver) (58/208) +26/04/01 06:06:47 INFO Executor: Running task 61.0 in stage 14.0 (TID 152) +26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:47 INFO Executor: Finished task 58.0 in stage 14.0 (TID 149). 6563 bytes result sent to driver +26/04/01 06:06:47 INFO TaskSetManager: Starting task 62.0 in stage 14.0 (TID 153) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:47 INFO TaskSetManager: Finished task 58.0 in stage 14.0 (TID 149) in 4140 ms on 10.0.0.133 (executor driver) (59/208) +26/04/01 06:06:47 INFO Executor: Running task 62.0 in stage 14.0 (TID 153) +26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:47 INFO Executor: Finished task 59.0 in stage 14.0 (TID 150). 6563 bytes result sent to driver +26/04/01 06:06:47 INFO TaskSetManager: Starting task 63.0 in stage 14.0 (TID 154) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:47 INFO TaskSetManager: Finished task 59.0 in stage 14.0 (TID 150) in 4141 ms on 10.0.0.133 (executor driver) (60/208) +26/04/01 06:06:47 INFO Executor: Running task 63.0 in stage 14.0 (TID 154) +26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:51 INFO Executor: Finished task 60.0 in stage 14.0 (TID 151). 6563 bytes result sent to driver +26/04/01 06:06:51 INFO TaskSetManager: Starting task 64.0 in stage 14.0 (TID 155) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:51 INFO TaskSetManager: Finished task 60.0 in stage 14.0 (TID 151) in 4102 ms on 10.0.0.133 (executor driver) (61/208) +26/04/01 06:06:51 INFO Executor: Running task 64.0 in stage 14.0 (TID 155) +26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:51 INFO Executor: Finished task 61.0 in stage 14.0 (TID 152). 6563 bytes result sent to driver +26/04/01 06:06:51 INFO Executor: Finished task 62.0 in stage 14.0 (TID 153). 6563 bytes result sent to driver +26/04/01 06:06:51 INFO TaskSetManager: Starting task 65.0 in stage 14.0 (TID 156) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:06:51 INFO Executor: Running task 65.0 in stage 14.0 (TID 156) +26/04/01 06:06:51 INFO TaskSetManager: Starting task 66.0 in stage 14.0 (TID 157) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:06:51 INFO TaskSetManager: Finished task 61.0 in stage 14.0 (TID 152) in 4102 ms on 10.0.0.133 (executor driver) (62/208) +26/04/01 06:06:51 INFO Executor: Running task 66.0 in stage 14.0 (TID 157) +26/04/01 06:06:51 INFO TaskSetManager: Finished task 62.0 in stage 14.0 (TID 153) in 4092 ms on 10.0.0.133 (executor driver) (63/208) +26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:51 INFO Executor: Finished task 63.0 in stage 14.0 (TID 154). 6563 bytes result sent to driver +26/04/01 06:06:51 INFO TaskSetManager: Starting task 67.0 in stage 14.0 (TID 158) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:06:51 INFO TaskSetManager: Finished task 63.0 in stage 14.0 (TID 154) in 4098 ms on 10.0.0.133 (executor driver) (64/208) +26/04/01 06:06:51 INFO Executor: Running task 67.0 in stage 14.0 (TID 158) +26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:55 INFO Executor: Finished task 64.0 in stage 14.0 (TID 155). 6563 bytes result sent to driver +26/04/01 06:06:55 INFO TaskSetManager: Starting task 68.0 in stage 14.0 (TID 159) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:55 INFO Executor: Running task 68.0 in stage 14.0 (TID 159) +26/04/01 06:06:55 INFO TaskSetManager: Finished task 64.0 in stage 14.0 (TID 155) in 4101 ms on 10.0.0.133 (executor driver) (65/208) +26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:55 INFO Executor: Finished task 66.0 in stage 14.0 (TID 157). 6563 bytes result sent to driver +26/04/01 06:06:55 INFO TaskSetManager: Starting task 69.0 in stage 14.0 (TID 160) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:55 INFO Executor: Running task 69.0 in stage 14.0 (TID 160) +26/04/01 06:06:55 INFO TaskSetManager: Finished task 66.0 in stage 14.0 (TID 157) in 4090 ms on 10.0.0.133 (executor driver) (66/208) +26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:55 INFO Executor: Finished task 65.0 in stage 14.0 (TID 156). 6563 bytes result sent to driver +26/04/01 06:06:55 INFO TaskSetManager: Starting task 70.0 in stage 14.0 (TID 161) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:55 INFO TaskSetManager: Finished task 65.0 in stage 14.0 (TID 156) in 4100 ms on 10.0.0.133 (executor driver) (67/208) +26/04/01 06:06:55 INFO Executor: Running task 70.0 in stage 14.0 (TID 161) +26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:55 INFO Executor: Finished task 67.0 in stage 14.0 (TID 158). 6563 bytes result sent to driver +26/04/01 06:06:55 INFO TaskSetManager: Starting task 71.0 in stage 14.0 (TID 162) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:55 INFO Executor: Running task 71.0 in stage 14.0 (TID 162) +26/04/01 06:06:55 INFO TaskSetManager: Finished task 67.0 in stage 14.0 (TID 158) in 4100 ms on 10.0.0.133 (executor driver) (68/208) +26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:59 INFO Executor: Finished task 68.0 in stage 14.0 (TID 159). 6606 bytes result sent to driver +26/04/01 06:06:59 INFO TaskSetManager: Starting task 72.0 in stage 14.0 (TID 163) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:06:59 INFO TaskSetManager: Finished task 68.0 in stage 14.0 (TID 159) in 4097 ms on 10.0.0.133 (executor driver) (69/208) +26/04/01 06:06:59 INFO Executor: Running task 72.0 in stage 14.0 (TID 163) +26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:59 INFO Executor: Finished task 69.0 in stage 14.0 (TID 160). 6606 bytes result sent to driver +26/04/01 06:06:59 INFO TaskSetManager: Starting task 73.0 in stage 14.0 (TID 164) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:06:59 INFO TaskSetManager: Finished task 69.0 in stage 14.0 (TID 160) in 4081 ms on 10.0.0.133 (executor driver) (70/208) +26/04/01 06:06:59 INFO Executor: Running task 73.0 in stage 14.0 (TID 164) +26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:59 INFO Executor: Finished task 70.0 in stage 14.0 (TID 161). 6606 bytes result sent to driver +26/04/01 06:06:59 INFO TaskSetManager: Starting task 74.0 in stage 14.0 (TID 165) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:59 INFO TaskSetManager: Finished task 70.0 in stage 14.0 (TID 161) in 4097 ms on 10.0.0.133 (executor driver) (71/208) +26/04/01 06:06:59 INFO Executor: Running task 74.0 in stage 14.0 (TID 165) +26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:59 INFO Executor: Finished task 71.0 in stage 14.0 (TID 162). 6606 bytes result sent to driver +26/04/01 06:06:59 INFO TaskSetManager: Starting task 75.0 in stage 14.0 (TID 166) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:06:59 INFO TaskSetManager: Finished task 71.0 in stage 14.0 (TID 162) in 4092 ms on 10.0.0.133 (executor driver) (72/208) +26/04/01 06:06:59 INFO Executor: Running task 75.0 in stage 14.0 (TID 166) +26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:03 INFO Executor: Finished task 72.0 in stage 14.0 (TID 163). 6563 bytes result sent to driver +26/04/01 06:07:03 INFO TaskSetManager: Starting task 76.0 in stage 14.0 (TID 167) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:03 INFO TaskSetManager: Finished task 72.0 in stage 14.0 (TID 163) in 4143 ms on 10.0.0.133 (executor driver) (73/208) +26/04/01 06:07:03 INFO Executor: Running task 76.0 in stage 14.0 (TID 167) +26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:03 INFO Executor: Finished task 73.0 in stage 14.0 (TID 164). 6563 bytes result sent to driver +26/04/01 06:07:03 INFO TaskSetManager: Starting task 77.0 in stage 14.0 (TID 168) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:03 INFO TaskSetManager: Finished task 73.0 in stage 14.0 (TID 164) in 4138 ms on 10.0.0.133 (executor driver) (74/208) +26/04/01 06:07:03 INFO Executor: Running task 77.0 in stage 14.0 (TID 168) +26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:03 INFO Executor: Finished task 74.0 in stage 14.0 (TID 165). 6563 bytes result sent to driver +26/04/01 06:07:03 INFO TaskSetManager: Starting task 78.0 in stage 14.0 (TID 169) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:07:03 INFO TaskSetManager: Finished task 74.0 in stage 14.0 (TID 165) in 4125 ms on 10.0.0.133 (executor driver) (75/208) +26/04/01 06:07:03 INFO Executor: Running task 78.0 in stage 14.0 (TID 169) +26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:03 INFO Executor: Finished task 75.0 in stage 14.0 (TID 166). 6563 bytes result sent to driver +26/04/01 06:07:03 INFO TaskSetManager: Starting task 79.0 in stage 14.0 (TID 170) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:03 INFO TaskSetManager: Finished task 75.0 in stage 14.0 (TID 166) in 4127 ms on 10.0.0.133 (executor driver) (76/208) +26/04/01 06:07:03 INFO Executor: Running task 79.0 in stage 14.0 (TID 170) +26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:07 INFO Executor: Finished task 76.0 in stage 14.0 (TID 167). 6563 bytes result sent to driver +26/04/01 06:07:07 INFO TaskSetManager: Starting task 80.0 in stage 14.0 (TID 171) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:07 INFO TaskSetManager: Finished task 76.0 in stage 14.0 (TID 167) in 4140 ms on 10.0.0.133 (executor driver) (77/208) +26/04/01 06:07:07 INFO Executor: Running task 80.0 in stage 14.0 (TID 171) +26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:07 INFO Executor: Finished task 77.0 in stage 14.0 (TID 168). 6563 bytes result sent to driver +26/04/01 06:07:07 INFO TaskSetManager: Starting task 81.0 in stage 14.0 (TID 172) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:07 INFO TaskSetManager: Finished task 77.0 in stage 14.0 (TID 168) in 4130 ms on 10.0.0.133 (executor driver) (78/208) +26/04/01 06:07:07 INFO Executor: Running task 81.0 in stage 14.0 (TID 172) +26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:07 INFO Executor: Finished task 78.0 in stage 14.0 (TID 169). 6563 bytes result sent to driver +26/04/01 06:07:07 INFO TaskSetManager: Starting task 82.0 in stage 14.0 (TID 173) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:07 INFO TaskSetManager: Finished task 78.0 in stage 14.0 (TID 169) in 4127 ms on 10.0.0.133 (executor driver) (79/208) +26/04/01 06:07:07 INFO Executor: Running task 82.0 in stage 14.0 (TID 173) +26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:07 INFO Executor: Finished task 79.0 in stage 14.0 (TID 170). 6563 bytes result sent to driver +26/04/01 06:07:07 INFO TaskSetManager: Starting task 83.0 in stage 14.0 (TID 174) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:07 INFO TaskSetManager: Finished task 79.0 in stage 14.0 (TID 170) in 4126 ms on 10.0.0.133 (executor driver) (80/208) +26/04/01 06:07:07 INFO Executor: Running task 83.0 in stage 14.0 (TID 174) +26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:11 INFO Executor: Finished task 80.0 in stage 14.0 (TID 171). 6563 bytes result sent to driver +26/04/01 06:07:11 INFO TaskSetManager: Starting task 84.0 in stage 14.0 (TID 175) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:07:11 INFO TaskSetManager: Finished task 80.0 in stage 14.0 (TID 171) in 4139 ms on 10.0.0.133 (executor driver) (81/208) +26/04/01 06:07:11 INFO Executor: Running task 84.0 in stage 14.0 (TID 175) +26/04/01 06:07:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:11 INFO Executor: Finished task 81.0 in stage 14.0 (TID 172). 6563 bytes result sent to driver +26/04/01 06:07:11 INFO TaskSetManager: Starting task 85.0 in stage 14.0 (TID 176) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:11 INFO Executor: Running task 85.0 in stage 14.0 (TID 176) +26/04/01 06:07:11 INFO TaskSetManager: Finished task 81.0 in stage 14.0 (TID 172) in 4144 ms on 10.0.0.133 (executor driver) (82/208) +26/04/01 06:07:11 INFO Executor: Finished task 82.0 in stage 14.0 (TID 173). 6563 bytes result sent to driver +26/04/01 06:07:11 INFO TaskSetManager: Starting task 86.0 in stage 14.0 (TID 177) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:11 INFO Executor: Running task 86.0 in stage 14.0 (TID 177) +26/04/01 06:07:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:11 INFO TaskSetManager: Finished task 82.0 in stage 14.0 (TID 173) in 4135 ms on 10.0.0.133 (executor driver) (83/208) +26/04/01 06:07:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:12 INFO Executor: Finished task 83.0 in stage 14.0 (TID 174). 6563 bytes result sent to driver +26/04/01 06:07:12 INFO TaskSetManager: Starting task 87.0 in stage 14.0 (TID 178) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:12 INFO TaskSetManager: Finished task 83.0 in stage 14.0 (TID 174) in 4141 ms on 10.0.0.133 (executor driver) (84/208) +26/04/01 06:07:12 INFO Executor: Running task 87.0 in stage 14.0 (TID 178) +26/04/01 06:07:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:15 INFO Executor: Finished task 84.0 in stage 14.0 (TID 175). 6563 bytes result sent to driver +26/04/01 06:07:15 INFO TaskSetManager: Starting task 88.0 in stage 14.0 (TID 179) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:15 INFO TaskSetManager: Finished task 84.0 in stage 14.0 (TID 175) in 4147 ms on 10.0.0.133 (executor driver) (85/208) +26/04/01 06:07:15 INFO Executor: Running task 88.0 in stage 14.0 (TID 179) +26/04/01 06:07:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:16 INFO Executor: Finished task 86.0 in stage 14.0 (TID 177). 6563 bytes result sent to driver +26/04/01 06:07:16 INFO TaskSetManager: Starting task 89.0 in stage 14.0 (TID 180) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:16 INFO TaskSetManager: Finished task 86.0 in stage 14.0 (TID 177) in 4139 ms on 10.0.0.133 (executor driver) (86/208) +26/04/01 06:07:16 INFO Executor: Running task 89.0 in stage 14.0 (TID 180) +26/04/01 06:07:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:16 INFO Executor: Finished task 85.0 in stage 14.0 (TID 176). 6563 bytes result sent to driver +26/04/01 06:07:16 INFO TaskSetManager: Starting task 90.0 in stage 14.0 (TID 181) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:07:16 INFO Executor: Running task 90.0 in stage 14.0 (TID 181) +26/04/01 06:07:16 INFO TaskSetManager: Finished task 85.0 in stage 14.0 (TID 176) in 4154 ms on 10.0.0.133 (executor driver) (87/208) +26/04/01 06:07:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:16 INFO Executor: Finished task 87.0 in stage 14.0 (TID 178). 6563 bytes result sent to driver +26/04/01 06:07:16 INFO TaskSetManager: Starting task 91.0 in stage 14.0 (TID 182) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:16 INFO Executor: Running task 91.0 in stage 14.0 (TID 182) +26/04/01 06:07:16 INFO TaskSetManager: Finished task 87.0 in stage 14.0 (TID 178) in 4146 ms on 10.0.0.133 (executor driver) (88/208) +26/04/01 06:07:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:20 INFO Executor: Finished task 88.0 in stage 14.0 (TID 179). 6563 bytes result sent to driver +26/04/01 06:07:20 INFO TaskSetManager: Starting task 92.0 in stage 14.0 (TID 183) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:20 INFO TaskSetManager: Finished task 88.0 in stage 14.0 (TID 179) in 4137 ms on 10.0.0.133 (executor driver) (89/208) +26/04/01 06:07:20 INFO Executor: Running task 92.0 in stage 14.0 (TID 183) +26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:20 INFO Executor: Finished task 91.0 in stage 14.0 (TID 182). 6563 bytes result sent to driver +26/04/01 06:07:20 INFO TaskSetManager: Starting task 93.0 in stage 14.0 (TID 184) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:20 INFO Executor: Running task 93.0 in stage 14.0 (TID 184) +26/04/01 06:07:20 INFO TaskSetManager: Finished task 91.0 in stage 14.0 (TID 182) in 3937 ms on 10.0.0.133 (executor driver) (90/208) +26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:20 INFO Executor: Finished task 89.0 in stage 14.0 (TID 180). 6563 bytes result sent to driver +26/04/01 06:07:20 INFO TaskSetManager: Starting task 94.0 in stage 14.0 (TID 185) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:20 INFO TaskSetManager: Finished task 89.0 in stage 14.0 (TID 180) in 4147 ms on 10.0.0.133 (executor driver) (91/208) +26/04/01 06:07:20 INFO Executor: Running task 94.0 in stage 14.0 (TID 185) +26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:20 INFO Executor: Finished task 90.0 in stage 14.0 (TID 181). 6563 bytes result sent to driver +26/04/01 06:07:20 INFO TaskSetManager: Starting task 95.0 in stage 14.0 (TID 186) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:20 INFO Executor: Running task 95.0 in stage 14.0 (TID 186) +26/04/01 06:07:20 INFO TaskSetManager: Finished task 90.0 in stage 14.0 (TID 181) in 4231 ms on 10.0.0.133 (executor driver) (92/208) +26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:24 INFO Executor: Finished task 92.0 in stage 14.0 (TID 183). 6563 bytes result sent to driver +26/04/01 06:07:24 INFO TaskSetManager: Starting task 96.0 in stage 14.0 (TID 187) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:07:24 INFO TaskSetManager: Finished task 92.0 in stage 14.0 (TID 183) in 4083 ms on 10.0.0.133 (executor driver) (93/208) +26/04/01 06:07:24 INFO Executor: Running task 96.0 in stage 14.0 (TID 187) +26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:24 INFO Executor: Finished task 93.0 in stage 14.0 (TID 184). 6563 bytes result sent to driver +26/04/01 06:07:24 INFO TaskSetManager: Starting task 97.0 in stage 14.0 (TID 188) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:07:24 INFO TaskSetManager: Finished task 93.0 in stage 14.0 (TID 184) in 4072 ms on 10.0.0.133 (executor driver) (94/208) +26/04/01 06:07:24 INFO Executor: Running task 97.0 in stage 14.0 (TID 188) +26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:24 INFO Executor: Finished task 94.0 in stage 14.0 (TID 185). 6563 bytes result sent to driver +26/04/01 06:07:24 INFO TaskSetManager: Starting task 98.0 in stage 14.0 (TID 189) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:24 INFO TaskSetManager: Finished task 94.0 in stage 14.0 (TID 185) in 4083 ms on 10.0.0.133 (executor driver) (95/208) +26/04/01 06:07:24 INFO Executor: Running task 98.0 in stage 14.0 (TID 189) +26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:24 INFO Executor: Finished task 95.0 in stage 14.0 (TID 186). 6563 bytes result sent to driver +26/04/01 06:07:24 INFO TaskSetManager: Starting task 99.0 in stage 14.0 (TID 190) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:24 INFO TaskSetManager: Finished task 95.0 in stage 14.0 (TID 186) in 4092 ms on 10.0.0.133 (executor driver) (96/208) +26/04/01 06:07:24 INFO Executor: Running task 99.0 in stage 14.0 (TID 190) +26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:28 INFO Executor: Finished task 96.0 in stage 14.0 (TID 187). 6563 bytes result sent to driver +26/04/01 06:07:28 INFO TaskSetManager: Starting task 100.0 in stage 14.0 (TID 191) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:28 INFO TaskSetManager: Finished task 96.0 in stage 14.0 (TID 187) in 4149 ms on 10.0.0.133 (executor driver) (97/208) +26/04/01 06:07:28 INFO Executor: Running task 100.0 in stage 14.0 (TID 191) +26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:28 INFO Executor: Finished task 97.0 in stage 14.0 (TID 188). 6563 bytes result sent to driver +26/04/01 06:07:28 INFO TaskSetManager: Starting task 101.0 in stage 14.0 (TID 192) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:28 INFO Executor: Running task 101.0 in stage 14.0 (TID 192) +26/04/01 06:07:28 INFO TaskSetManager: Finished task 97.0 in stage 14.0 (TID 188) in 4139 ms on 10.0.0.133 (executor driver) (98/208) +26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:28 INFO Executor: Finished task 98.0 in stage 14.0 (TID 189). 6563 bytes result sent to driver +26/04/01 06:07:28 INFO TaskSetManager: Starting task 102.0 in stage 14.0 (TID 193) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:07:28 INFO Executor: Running task 102.0 in stage 14.0 (TID 193) +26/04/01 06:07:28 INFO TaskSetManager: Finished task 98.0 in stage 14.0 (TID 189) in 4151 ms on 10.0.0.133 (executor driver) (99/208) +26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:28 INFO Executor: Finished task 99.0 in stage 14.0 (TID 190). 6563 bytes result sent to driver +26/04/01 06:07:28 INFO TaskSetManager: Starting task 103.0 in stage 14.0 (TID 194) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:28 INFO TaskSetManager: Finished task 99.0 in stage 14.0 (TID 190) in 4133 ms on 10.0.0.133 (executor driver) (100/208) +26/04/01 06:07:28 INFO Executor: Running task 103.0 in stage 14.0 (TID 194) +26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:32 INFO Executor: Finished task 100.0 in stage 14.0 (TID 191). 6563 bytes result sent to driver +26/04/01 06:07:32 INFO TaskSetManager: Starting task 104.0 in stage 14.0 (TID 195) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:32 INFO TaskSetManager: Finished task 100.0 in stage 14.0 (TID 191) in 4150 ms on 10.0.0.133 (executor driver) (101/208) +26/04/01 06:07:32 INFO Executor: Running task 104.0 in stage 14.0 (TID 195) +26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:32 INFO Executor: Finished task 101.0 in stage 14.0 (TID 192). 6563 bytes result sent to driver +26/04/01 06:07:32 INFO TaskSetManager: Starting task 105.0 in stage 14.0 (TID 196) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:32 INFO Executor: Running task 105.0 in stage 14.0 (TID 196) +26/04/01 06:07:32 INFO TaskSetManager: Finished task 101.0 in stage 14.0 (TID 192) in 4149 ms on 10.0.0.133 (executor driver) (102/208) +26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:32 INFO Executor: Finished task 102.0 in stage 14.0 (TID 193). 6563 bytes result sent to driver +26/04/01 06:07:32 INFO TaskSetManager: Starting task 106.0 in stage 14.0 (TID 197) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:32 INFO TaskSetManager: Finished task 102.0 in stage 14.0 (TID 193) in 4134 ms on 10.0.0.133 (executor driver) (103/208) +26/04/01 06:07:32 INFO Executor: Running task 106.0 in stage 14.0 (TID 197) +26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:32 INFO Executor: Finished task 103.0 in stage 14.0 (TID 194). 6563 bytes result sent to driver +26/04/01 06:07:32 INFO TaskSetManager: Starting task 107.0 in stage 14.0 (TID 198) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:32 INFO TaskSetManager: Finished task 103.0 in stage 14.0 (TID 194) in 4153 ms on 10.0.0.133 (executor driver) (104/208) +26/04/01 06:07:32 INFO Executor: Running task 107.0 in stage 14.0 (TID 198) +26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:36 INFO Executor: Finished task 104.0 in stage 14.0 (TID 195). 6563 bytes result sent to driver +26/04/01 06:07:36 INFO TaskSetManager: Starting task 108.0 in stage 14.0 (TID 199) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:07:36 INFO TaskSetManager: Finished task 104.0 in stage 14.0 (TID 195) in 4140 ms on 10.0.0.133 (executor driver) (105/208) +26/04/01 06:07:36 INFO Executor: Running task 108.0 in stage 14.0 (TID 199) +26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:36 INFO Executor: Finished task 105.0 in stage 14.0 (TID 196). 6563 bytes result sent to driver +26/04/01 06:07:36 INFO TaskSetManager: Starting task 109.0 in stage 14.0 (TID 200) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:07:36 INFO TaskSetManager: Finished task 105.0 in stage 14.0 (TID 196) in 4136 ms on 10.0.0.133 (executor driver) (106/208) +26/04/01 06:07:36 INFO Executor: Running task 109.0 in stage 14.0 (TID 200) +26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:36 INFO Executor: Finished task 106.0 in stage 14.0 (TID 197). 6563 bytes result sent to driver +26/04/01 06:07:36 INFO TaskSetManager: Starting task 110.0 in stage 14.0 (TID 201) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:36 INFO TaskSetManager: Finished task 106.0 in stage 14.0 (TID 197) in 4155 ms on 10.0.0.133 (executor driver) (107/208) +26/04/01 06:07:36 INFO Executor: Running task 110.0 in stage 14.0 (TID 201) +26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:36 INFO Executor: Finished task 107.0 in stage 14.0 (TID 198). 6563 bytes result sent to driver +26/04/01 06:07:36 INFO TaskSetManager: Starting task 111.0 in stage 14.0 (TID 202) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:36 INFO TaskSetManager: Finished task 107.0 in stage 14.0 (TID 198) in 4144 ms on 10.0.0.133 (executor driver) (108/208) +26/04/01 06:07:36 INFO Executor: Running task 111.0 in stage 14.0 (TID 202) +26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:40 INFO Executor: Finished task 108.0 in stage 14.0 (TID 199). 6563 bytes result sent to driver +26/04/01 06:07:40 INFO TaskSetManager: Starting task 112.0 in stage 14.0 (TID 203) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:40 INFO TaskSetManager: Finished task 108.0 in stage 14.0 (TID 199) in 4150 ms on 10.0.0.133 (executor driver) (109/208) +26/04/01 06:07:40 INFO Executor: Running task 112.0 in stage 14.0 (TID 203) +26/04/01 06:07:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:40 INFO Executor: Finished task 109.0 in stage 14.0 (TID 200). 6563 bytes result sent to driver +26/04/01 06:07:40 INFO TaskSetManager: Starting task 113.0 in stage 14.0 (TID 204) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:40 INFO TaskSetManager: Finished task 109.0 in stage 14.0 (TID 200) in 4149 ms on 10.0.0.133 (executor driver) (110/208) +26/04/01 06:07:40 INFO Executor: Running task 113.0 in stage 14.0 (TID 204) +26/04/01 06:07:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:40 INFO Executor: Finished task 110.0 in stage 14.0 (TID 201). 6563 bytes result sent to driver +26/04/01 06:07:40 INFO TaskSetManager: Starting task 114.0 in stage 14.0 (TID 205) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:07:40 INFO Executor: Running task 114.0 in stage 14.0 (TID 205) +26/04/01 06:07:40 INFO TaskSetManager: Finished task 110.0 in stage 14.0 (TID 201) in 4157 ms on 10.0.0.133 (executor driver) (111/208) +26/04/01 06:07:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:41 INFO Executor: Finished task 111.0 in stage 14.0 (TID 202). 6606 bytes result sent to driver +26/04/01 06:07:41 INFO TaskSetManager: Starting task 115.0 in stage 14.0 (TID 206) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:41 INFO TaskSetManager: Finished task 111.0 in stage 14.0 (TID 202) in 4154 ms on 10.0.0.133 (executor driver) (112/208) +26/04/01 06:07:41 INFO Executor: Running task 115.0 in stage 14.0 (TID 206) +26/04/01 06:07:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:44 INFO Executor: Finished task 112.0 in stage 14.0 (TID 203). 6606 bytes result sent to driver +26/04/01 06:07:44 INFO TaskSetManager: Starting task 116.0 in stage 14.0 (TID 207) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:44 INFO TaskSetManager: Finished task 112.0 in stage 14.0 (TID 203) in 4147 ms on 10.0.0.133 (executor driver) (113/208) +26/04/01 06:07:44 INFO Executor: Running task 116.0 in stage 14.0 (TID 207) +26/04/01 06:07:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:44 INFO Executor: Finished task 113.0 in stage 14.0 (TID 204). 6606 bytes result sent to driver +26/04/01 06:07:44 INFO TaskSetManager: Starting task 117.0 in stage 14.0 (TID 208) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:44 INFO TaskSetManager: Finished task 113.0 in stage 14.0 (TID 204) in 4150 ms on 10.0.0.133 (executor driver) (114/208) +26/04/01 06:07:44 INFO Executor: Running task 117.0 in stage 14.0 (TID 208) +26/04/01 06:07:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:45 INFO Executor: Finished task 114.0 in stage 14.0 (TID 205). 6563 bytes result sent to driver +26/04/01 06:07:45 INFO TaskSetManager: Starting task 118.0 in stage 14.0 (TID 209) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:45 INFO TaskSetManager: Finished task 114.0 in stage 14.0 (TID 205) in 4153 ms on 10.0.0.133 (executor driver) (115/208) +26/04/01 06:07:45 INFO Executor: Running task 118.0 in stage 14.0 (TID 209) +26/04/01 06:07:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:45 INFO Executor: Finished task 115.0 in stage 14.0 (TID 206). 6563 bytes result sent to driver +26/04/01 06:07:45 INFO TaskSetManager: Starting task 119.0 in stage 14.0 (TID 210) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:45 INFO TaskSetManager: Finished task 115.0 in stage 14.0 (TID 206) in 4147 ms on 10.0.0.133 (executor driver) (116/208) +26/04/01 06:07:45 INFO Executor: Running task 119.0 in stage 14.0 (TID 210) +26/04/01 06:07:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:49 INFO Executor: Finished task 116.0 in stage 14.0 (TID 207). 6563 bytes result sent to driver +26/04/01 06:07:49 INFO TaskSetManager: Starting task 120.0 in stage 14.0 (TID 211) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:07:49 INFO TaskSetManager: Finished task 116.0 in stage 14.0 (TID 207) in 4158 ms on 10.0.0.133 (executor driver) (117/208) +26/04/01 06:07:49 INFO Executor: Running task 120.0 in stage 14.0 (TID 211) +26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:49 INFO Executor: Finished task 117.0 in stage 14.0 (TID 208). 6563 bytes result sent to driver +26/04/01 06:07:49 INFO TaskSetManager: Starting task 121.0 in stage 14.0 (TID 212) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:49 INFO TaskSetManager: Finished task 117.0 in stage 14.0 (TID 208) in 4152 ms on 10.0.0.133 (executor driver) (118/208) +26/04/01 06:07:49 INFO Executor: Running task 121.0 in stage 14.0 (TID 212) +26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:49 INFO Executor: Finished task 118.0 in stage 14.0 (TID 209). 6563 bytes result sent to driver +26/04/01 06:07:49 INFO TaskSetManager: Starting task 122.0 in stage 14.0 (TID 213) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:49 INFO TaskSetManager: Finished task 118.0 in stage 14.0 (TID 209) in 4153 ms on 10.0.0.133 (executor driver) (119/208) +26/04/01 06:07:49 INFO Executor: Running task 122.0 in stage 14.0 (TID 213) +26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:49 INFO Executor: Finished task 119.0 in stage 14.0 (TID 210). 6563 bytes result sent to driver +26/04/01 06:07:49 INFO TaskSetManager: Starting task 123.0 in stage 14.0 (TID 214) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:49 INFO TaskSetManager: Finished task 119.0 in stage 14.0 (TID 210) in 4157 ms on 10.0.0.133 (executor driver) (120/208) +26/04/01 06:07:49 INFO Executor: Running task 123.0 in stage 14.0 (TID 214) +26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:53 INFO Executor: Finished task 120.0 in stage 14.0 (TID 211). 6563 bytes result sent to driver +26/04/01 06:07:53 INFO Executor: Finished task 121.0 in stage 14.0 (TID 212). 6563 bytes result sent to driver +26/04/01 06:07:53 INFO TaskSetManager: Starting task 124.0 in stage 14.0 (TID 215) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:53 INFO TaskSetManager: Finished task 120.0 in stage 14.0 (TID 211) in 4155 ms on 10.0.0.133 (executor driver) (121/208) +26/04/01 06:07:53 INFO Executor: Running task 124.0 in stage 14.0 (TID 215) +26/04/01 06:07:53 INFO TaskSetManager: Starting task 125.0 in stage 14.0 (TID 216) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:53 INFO TaskSetManager: Finished task 121.0 in stage 14.0 (TID 212) in 4152 ms on 10.0.0.133 (executor driver) (122/208) +26/04/01 06:07:53 INFO Executor: Running task 125.0 in stage 14.0 (TID 216) +26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:53 INFO Executor: Finished task 122.0 in stage 14.0 (TID 213). 6563 bytes result sent to driver +26/04/01 06:07:53 INFO TaskSetManager: Starting task 126.0 in stage 14.0 (TID 217) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:07:53 INFO TaskSetManager: Finished task 122.0 in stage 14.0 (TID 213) in 4151 ms on 10.0.0.133 (executor driver) (123/208) +26/04/01 06:07:53 INFO Executor: Running task 126.0 in stage 14.0 (TID 217) +26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:53 INFO Executor: Finished task 123.0 in stage 14.0 (TID 214). 6563 bytes result sent to driver +26/04/01 06:07:53 INFO TaskSetManager: Starting task 127.0 in stage 14.0 (TID 218) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:07:53 INFO TaskSetManager: Finished task 123.0 in stage 14.0 (TID 214) in 4155 ms on 10.0.0.133 (executor driver) (124/208) +26/04/01 06:07:53 INFO Executor: Running task 127.0 in stage 14.0 (TID 218) +26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:57 INFO Executor: Finished task 124.0 in stage 14.0 (TID 215). 6563 bytes result sent to driver +26/04/01 06:07:57 INFO TaskSetManager: Starting task 128.0 in stage 14.0 (TID 219) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:57 INFO TaskSetManager: Finished task 124.0 in stage 14.0 (TID 215) in 4144 ms on 10.0.0.133 (executor driver) (125/208) +26/04/01 06:07:57 INFO Executor: Running task 128.0 in stage 14.0 (TID 219) +26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:57 INFO Executor: Finished task 125.0 in stage 14.0 (TID 216). 6563 bytes result sent to driver +26/04/01 06:07:57 INFO TaskSetManager: Starting task 129.0 in stage 14.0 (TID 220) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:57 INFO Executor: Running task 129.0 in stage 14.0 (TID 220) +26/04/01 06:07:57 INFO TaskSetManager: Finished task 125.0 in stage 14.0 (TID 216) in 4149 ms on 10.0.0.133 (executor driver) (126/208) +26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:57 INFO Executor: Finished task 126.0 in stage 14.0 (TID 217). 6563 bytes result sent to driver +26/04/01 06:07:57 INFO TaskSetManager: Starting task 130.0 in stage 14.0 (TID 221) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:57 INFO TaskSetManager: Finished task 126.0 in stage 14.0 (TID 217) in 4149 ms on 10.0.0.133 (executor driver) (127/208) +26/04/01 06:07:57 INFO Executor: Running task 130.0 in stage 14.0 (TID 221) +26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:57 INFO Executor: Finished task 127.0 in stage 14.0 (TID 218). 6563 bytes result sent to driver +26/04/01 06:07:57 INFO TaskSetManager: Starting task 131.0 in stage 14.0 (TID 222) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:07:57 INFO TaskSetManager: Finished task 127.0 in stage 14.0 (TID 218) in 4153 ms on 10.0.0.133 (executor driver) (128/208) +26/04/01 06:07:57 INFO Executor: Running task 131.0 in stage 14.0 (TID 222) +26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:01 INFO Executor: Finished task 129.0 in stage 14.0 (TID 220). 6563 bytes result sent to driver +26/04/01 06:08:01 INFO TaskSetManager: Starting task 132.0 in stage 14.0 (TID 223) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:08:01 INFO TaskSetManager: Finished task 129.0 in stage 14.0 (TID 220) in 4129 ms on 10.0.0.133 (executor driver) (129/208) +26/04/01 06:08:01 INFO Executor: Running task 132.0 in stage 14.0 (TID 223) +26/04/01 06:08:01 INFO Executor: Finished task 128.0 in stage 14.0 (TID 219). 6563 bytes result sent to driver +26/04/01 06:08:01 INFO TaskSetManager: Starting task 133.0 in stage 14.0 (TID 224) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:01 INFO TaskSetManager: Finished task 128.0 in stage 14.0 (TID 219) in 4135 ms on 10.0.0.133 (executor driver) (130/208) +26/04/01 06:08:01 INFO Executor: Running task 133.0 in stage 14.0 (TID 224) +26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:01 INFO Executor: Finished task 130.0 in stage 14.0 (TID 221). 6563 bytes result sent to driver +26/04/01 06:08:01 INFO TaskSetManager: Starting task 134.0 in stage 14.0 (TID 225) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:01 INFO TaskSetManager: Finished task 130.0 in stage 14.0 (TID 221) in 4132 ms on 10.0.0.133 (executor driver) (131/208) +26/04/01 06:08:01 INFO Executor: Running task 134.0 in stage 14.0 (TID 225) +26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:01 INFO Executor: Finished task 131.0 in stage 14.0 (TID 222). 6563 bytes result sent to driver +26/04/01 06:08:01 INFO TaskSetManager: Starting task 135.0 in stage 14.0 (TID 226) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:01 INFO TaskSetManager: Finished task 131.0 in stage 14.0 (TID 222) in 4130 ms on 10.0.0.133 (executor driver) (132/208) +26/04/01 06:08:01 INFO Executor: Running task 135.0 in stage 14.0 (TID 226) +26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:05 INFO Executor: Finished task 133.0 in stage 14.0 (TID 224). 6563 bytes result sent to driver +26/04/01 06:08:05 INFO TaskSetManager: Starting task 136.0 in stage 14.0 (TID 227) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:05 INFO TaskSetManager: Finished task 133.0 in stage 14.0 (TID 224) in 4082 ms on 10.0.0.133 (executor driver) (133/208) +26/04/01 06:08:05 INFO Executor: Running task 136.0 in stage 14.0 (TID 227) +26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:05 INFO Executor: Finished task 132.0 in stage 14.0 (TID 223). 6563 bytes result sent to driver +26/04/01 06:08:05 INFO TaskSetManager: Starting task 137.0 in stage 14.0 (TID 228) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:05 INFO TaskSetManager: Finished task 132.0 in stage 14.0 (TID 223) in 4086 ms on 10.0.0.133 (executor driver) (134/208) +26/04/01 06:08:05 INFO Executor: Running task 137.0 in stage 14.0 (TID 228) +26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:05 INFO Executor: Finished task 134.0 in stage 14.0 (TID 225). 6563 bytes result sent to driver +26/04/01 06:08:05 INFO TaskSetManager: Starting task 138.0 in stage 14.0 (TID 229) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:08:05 INFO TaskSetManager: Finished task 134.0 in stage 14.0 (TID 225) in 4086 ms on 10.0.0.133 (executor driver) (135/208) +26/04/01 06:08:05 INFO Executor: Running task 138.0 in stage 14.0 (TID 229) +26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:05 INFO Executor: Finished task 135.0 in stage 14.0 (TID 226). 6563 bytes result sent to driver +26/04/01 06:08:05 INFO TaskSetManager: Starting task 139.0 in stage 14.0 (TID 230) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:05 INFO TaskSetManager: Finished task 135.0 in stage 14.0 (TID 226) in 4086 ms on 10.0.0.133 (executor driver) (136/208) +26/04/01 06:08:05 INFO Executor: Running task 139.0 in stage 14.0 (TID 230) +26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:09 INFO Executor: Finished task 136.0 in stage 14.0 (TID 227). 6563 bytes result sent to driver +26/04/01 06:08:09 INFO TaskSetManager: Starting task 140.0 in stage 14.0 (TID 231) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:09 INFO TaskSetManager: Finished task 136.0 in stage 14.0 (TID 227) in 4086 ms on 10.0.0.133 (executor driver) (137/208) +26/04/01 06:08:09 INFO Executor: Running task 140.0 in stage 14.0 (TID 231) +26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:09 INFO Executor: Finished task 137.0 in stage 14.0 (TID 228). 6563 bytes result sent to driver +26/04/01 06:08:09 INFO TaskSetManager: Starting task 141.0 in stage 14.0 (TID 232) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:09 INFO TaskSetManager: Finished task 137.0 in stage 14.0 (TID 228) in 4188 ms on 10.0.0.133 (executor driver) (138/208) +26/04/01 06:08:09 INFO Executor: Running task 141.0 in stage 14.0 (TID 232) +26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:09 INFO Executor: Finished task 138.0 in stage 14.0 (TID 229). 6563 bytes result sent to driver +26/04/01 06:08:09 INFO TaskSetManager: Starting task 142.0 in stage 14.0 (TID 233) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:09 INFO TaskSetManager: Finished task 138.0 in stage 14.0 (TID 229) in 4084 ms on 10.0.0.133 (executor driver) (139/208) +26/04/01 06:08:09 INFO Executor: Running task 142.0 in stage 14.0 (TID 233) +26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:09 INFO Executor: Finished task 139.0 in stage 14.0 (TID 230). 6563 bytes result sent to driver +26/04/01 06:08:09 INFO TaskSetManager: Starting task 143.0 in stage 14.0 (TID 234) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:09 INFO Executor: Running task 143.0 in stage 14.0 (TID 234) +26/04/01 06:08:09 INFO TaskSetManager: Finished task 139.0 in stage 14.0 (TID 230) in 4080 ms on 10.0.0.133 (executor driver) (140/208) +26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:13 INFO Executor: Finished task 140.0 in stage 14.0 (TID 231). 6563 bytes result sent to driver +26/04/01 06:08:13 INFO TaskSetManager: Starting task 144.0 in stage 14.0 (TID 235) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:08:13 INFO TaskSetManager: Finished task 140.0 in stage 14.0 (TID 231) in 4126 ms on 10.0.0.133 (executor driver) (141/208) +26/04/01 06:08:13 INFO Executor: Running task 144.0 in stage 14.0 (TID 235) +26/04/01 06:08:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:13 INFO Executor: Finished task 141.0 in stage 14.0 (TID 232). 6563 bytes result sent to driver +26/04/01 06:08:13 INFO TaskSetManager: Starting task 145.0 in stage 14.0 (TID 236) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:08:13 INFO TaskSetManager: Finished task 141.0 in stage 14.0 (TID 232) in 4100 ms on 10.0.0.133 (executor driver) (142/208) +26/04/01 06:08:13 INFO Executor: Running task 145.0 in stage 14.0 (TID 236) +26/04/01 06:08:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:13 INFO Executor: Finished task 142.0 in stage 14.0 (TID 233). 6563 bytes result sent to driver +26/04/01 06:08:13 INFO TaskSetManager: Starting task 146.0 in stage 14.0 (TID 237) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:13 INFO Executor: Running task 146.0 in stage 14.0 (TID 237) +26/04/01 06:08:13 INFO TaskSetManager: Finished task 142.0 in stage 14.0 (TID 233) in 4087 ms on 10.0.0.133 (executor driver) (143/208) +26/04/01 06:08:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:14 INFO Executor: Finished task 143.0 in stage 14.0 (TID 234). 6563 bytes result sent to driver +26/04/01 06:08:14 INFO TaskSetManager: Starting task 147.0 in stage 14.0 (TID 238) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:14 INFO TaskSetManager: Finished task 143.0 in stage 14.0 (TID 234) in 4095 ms on 10.0.0.133 (executor driver) (144/208) +26/04/01 06:08:14 INFO Executor: Running task 147.0 in stage 14.0 (TID 238) +26/04/01 06:08:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:17 INFO Executor: Finished task 144.0 in stage 14.0 (TID 235). 6563 bytes result sent to driver +26/04/01 06:08:17 INFO TaskSetManager: Starting task 148.0 in stage 14.0 (TID 239) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:17 INFO TaskSetManager: Finished task 144.0 in stage 14.0 (TID 235) in 4140 ms on 10.0.0.133 (executor driver) (145/208) +26/04/01 06:08:17 INFO Executor: Running task 148.0 in stage 14.0 (TID 239) +26/04/01 06:08:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:18 INFO Executor: Finished task 145.0 in stage 14.0 (TID 236). 6563 bytes result sent to driver +26/04/01 06:08:18 INFO TaskSetManager: Starting task 149.0 in stage 14.0 (TID 240) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:18 INFO TaskSetManager: Finished task 145.0 in stage 14.0 (TID 236) in 4132 ms on 10.0.0.133 (executor driver) (146/208) +26/04/01 06:08:18 INFO Executor: Running task 149.0 in stage 14.0 (TID 240) +26/04/01 06:08:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:18 INFO Executor: Finished task 146.0 in stage 14.0 (TID 237). 6563 bytes result sent to driver +26/04/01 06:08:18 INFO TaskSetManager: Starting task 150.0 in stage 14.0 (TID 241) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:08:18 INFO Executor: Running task 150.0 in stage 14.0 (TID 241) +26/04/01 06:08:18 INFO TaskSetManager: Finished task 146.0 in stage 14.0 (TID 237) in 4125 ms on 10.0.0.133 (executor driver) (147/208) +26/04/01 06:08:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:18 INFO Executor: Finished task 147.0 in stage 14.0 (TID 238). 6563 bytes result sent to driver +26/04/01 06:08:18 INFO TaskSetManager: Starting task 151.0 in stage 14.0 (TID 242) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:18 INFO TaskSetManager: Finished task 147.0 in stage 14.0 (TID 238) in 4125 ms on 10.0.0.133 (executor driver) (148/208) +26/04/01 06:08:18 INFO Executor: Running task 151.0 in stage 14.0 (TID 242) +26/04/01 06:08:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:22 INFO Executor: Finished task 148.0 in stage 14.0 (TID 239). 6563 bytes result sent to driver +26/04/01 06:08:22 INFO TaskSetManager: Starting task 152.0 in stage 14.0 (TID 243) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:22 INFO TaskSetManager: Finished task 148.0 in stage 14.0 (TID 239) in 4157 ms on 10.0.0.133 (executor driver) (149/208) +26/04/01 06:08:22 INFO Executor: Running task 152.0 in stage 14.0 (TID 243) +26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:22 INFO Executor: Finished task 149.0 in stage 14.0 (TID 240). 6563 bytes result sent to driver +26/04/01 06:08:22 INFO TaskSetManager: Starting task 153.0 in stage 14.0 (TID 244) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:22 INFO Executor: Running task 153.0 in stage 14.0 (TID 244) +26/04/01 06:08:22 INFO TaskSetManager: Finished task 149.0 in stage 14.0 (TID 240) in 4140 ms on 10.0.0.133 (executor driver) (150/208) +26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:22 INFO Executor: Finished task 150.0 in stage 14.0 (TID 241). 6563 bytes result sent to driver +26/04/01 06:08:22 INFO TaskSetManager: Starting task 154.0 in stage 14.0 (TID 245) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:22 INFO TaskSetManager: Finished task 150.0 in stage 14.0 (TID 241) in 4160 ms on 10.0.0.133 (executor driver) (151/208) +26/04/01 06:08:22 INFO Executor: Running task 154.0 in stage 14.0 (TID 245) +26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:22 INFO Executor: Finished task 151.0 in stage 14.0 (TID 242). 6563 bytes result sent to driver +26/04/01 06:08:22 INFO TaskSetManager: Starting task 155.0 in stage 14.0 (TID 246) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:22 INFO Executor: Running task 155.0 in stage 14.0 (TID 246) +26/04/01 06:08:22 INFO TaskSetManager: Finished task 151.0 in stage 14.0 (TID 242) in 4147 ms on 10.0.0.133 (executor driver) (152/208) +26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:26 INFO Executor: Finished task 152.0 in stage 14.0 (TID 243). 6606 bytes result sent to driver +26/04/01 06:08:26 INFO TaskSetManager: Starting task 156.0 in stage 14.0 (TID 247) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:08:26 INFO TaskSetManager: Finished task 152.0 in stage 14.0 (TID 243) in 4136 ms on 10.0.0.133 (executor driver) (153/208) +26/04/01 06:08:26 INFO Executor: Running task 156.0 in stage 14.0 (TID 247) +26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:26 INFO Executor: Finished task 153.0 in stage 14.0 (TID 244). 6606 bytes result sent to driver +26/04/01 06:08:26 INFO TaskSetManager: Starting task 157.0 in stage 14.0 (TID 248) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:26 INFO TaskSetManager: Finished task 153.0 in stage 14.0 (TID 244) in 4143 ms on 10.0.0.133 (executor driver) (154/208) +26/04/01 06:08:26 INFO Executor: Running task 157.0 in stage 14.0 (TID 248) +26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:26 INFO Executor: Finished task 154.0 in stage 14.0 (TID 245). 6606 bytes result sent to driver +26/04/01 06:08:26 INFO TaskSetManager: Starting task 158.0 in stage 14.0 (TID 249) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:26 INFO Executor: Running task 158.0 in stage 14.0 (TID 249) +26/04/01 06:08:26 INFO TaskSetManager: Finished task 154.0 in stage 14.0 (TID 245) in 4143 ms on 10.0.0.133 (executor driver) (155/208) +26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:26 INFO Executor: Finished task 155.0 in stage 14.0 (TID 246). 6606 bytes result sent to driver +26/04/01 06:08:26 INFO TaskSetManager: Starting task 159.0 in stage 14.0 (TID 250) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:26 INFO TaskSetManager: Finished task 155.0 in stage 14.0 (TID 246) in 4140 ms on 10.0.0.133 (executor driver) (156/208) +26/04/01 06:08:26 INFO Executor: Running task 159.0 in stage 14.0 (TID 250) +26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:30 INFO Executor: Finished task 156.0 in stage 14.0 (TID 247). 6563 bytes result sent to driver +26/04/01 06:08:30 INFO TaskSetManager: Starting task 160.0 in stage 14.0 (TID 251) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:30 INFO TaskSetManager: Finished task 156.0 in stage 14.0 (TID 247) in 4109 ms on 10.0.0.133 (executor driver) (157/208) +26/04/01 06:08:30 INFO Executor: Running task 160.0 in stage 14.0 (TID 251) +26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:30 INFO Executor: Finished task 157.0 in stage 14.0 (TID 248). 6563 bytes result sent to driver +26/04/01 06:08:30 INFO TaskSetManager: Starting task 161.0 in stage 14.0 (TID 252) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:30 INFO TaskSetManager: Finished task 157.0 in stage 14.0 (TID 248) in 4116 ms on 10.0.0.133 (executor driver) (158/208) +26/04/01 06:08:30 INFO Executor: Running task 161.0 in stage 14.0 (TID 252) +26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:30 INFO Executor: Finished task 158.0 in stage 14.0 (TID 249). 6563 bytes result sent to driver +26/04/01 06:08:30 INFO TaskSetManager: Starting task 162.0 in stage 14.0 (TID 253) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:08:30 INFO TaskSetManager: Finished task 158.0 in stage 14.0 (TID 249) in 4106 ms on 10.0.0.133 (executor driver) (159/208) +26/04/01 06:08:30 INFO Executor: Running task 162.0 in stage 14.0 (TID 253) +26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:30 INFO Executor: Finished task 159.0 in stage 14.0 (TID 250). 6563 bytes result sent to driver +26/04/01 06:08:30 INFO TaskSetManager: Starting task 163.0 in stage 14.0 (TID 254) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:08:30 INFO TaskSetManager: Finished task 159.0 in stage 14.0 (TID 250) in 4118 ms on 10.0.0.133 (executor driver) (160/208) +26/04/01 06:08:30 INFO Executor: Running task 163.0 in stage 14.0 (TID 254) +26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:34 INFO Executor: Finished task 160.0 in stage 14.0 (TID 251). 6563 bytes result sent to driver +26/04/01 06:08:34 INFO TaskSetManager: Starting task 164.0 in stage 14.0 (TID 255) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:34 INFO TaskSetManager: Finished task 160.0 in stage 14.0 (TID 251) in 4102 ms on 10.0.0.133 (executor driver) (161/208) +26/04/01 06:08:34 INFO Executor: Running task 164.0 in stage 14.0 (TID 255) +26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:34 INFO Executor: Finished task 161.0 in stage 14.0 (TID 252). 6563 bytes result sent to driver +26/04/01 06:08:34 INFO TaskSetManager: Starting task 165.0 in stage 14.0 (TID 256) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:34 INFO TaskSetManager: Finished task 161.0 in stage 14.0 (TID 252) in 4091 ms on 10.0.0.133 (executor driver) (162/208) +26/04/01 06:08:34 INFO Executor: Running task 165.0 in stage 14.0 (TID 256) +26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:34 INFO Executor: Finished task 162.0 in stage 14.0 (TID 253). 6563 bytes result sent to driver +26/04/01 06:08:34 INFO TaskSetManager: Starting task 166.0 in stage 14.0 (TID 257) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:34 INFO TaskSetManager: Finished task 162.0 in stage 14.0 (TID 253) in 4097 ms on 10.0.0.133 (executor driver) (163/208) +26/04/01 06:08:34 INFO Executor: Running task 166.0 in stage 14.0 (TID 257) +26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:34 INFO Executor: Finished task 163.0 in stage 14.0 (TID 254). 6563 bytes result sent to driver +26/04/01 06:08:34 INFO TaskSetManager: Starting task 167.0 in stage 14.0 (TID 258) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:34 INFO TaskSetManager: Finished task 163.0 in stage 14.0 (TID 254) in 4103 ms on 10.0.0.133 (executor driver) (164/208) +26/04/01 06:08:34 INFO Executor: Running task 167.0 in stage 14.0 (TID 258) +26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:38 INFO Executor: Finished task 164.0 in stage 14.0 (TID 255). 6563 bytes result sent to driver +26/04/01 06:08:38 INFO TaskSetManager: Starting task 168.0 in stage 14.0 (TID 259) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:08:38 INFO TaskSetManager: Finished task 164.0 in stage 14.0 (TID 255) in 4099 ms on 10.0.0.133 (executor driver) (165/208) +26/04/01 06:08:38 INFO Executor: Running task 168.0 in stage 14.0 (TID 259) +26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:38 INFO Executor: Finished task 165.0 in stage 14.0 (TID 256). 6563 bytes result sent to driver +26/04/01 06:08:38 INFO TaskSetManager: Starting task 169.0 in stage 14.0 (TID 260) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:38 INFO TaskSetManager: Finished task 165.0 in stage 14.0 (TID 256) in 4101 ms on 10.0.0.133 (executor driver) (166/208) +26/04/01 06:08:38 INFO Executor: Running task 169.0 in stage 14.0 (TID 260) +26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:38 INFO Executor: Finished task 166.0 in stage 14.0 (TID 257). 6563 bytes result sent to driver +26/04/01 06:08:38 INFO TaskSetManager: Starting task 170.0 in stage 14.0 (TID 261) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:38 INFO TaskSetManager: Finished task 166.0 in stage 14.0 (TID 257) in 4112 ms on 10.0.0.133 (executor driver) (167/208) +26/04/01 06:08:38 INFO Executor: Running task 170.0 in stage 14.0 (TID 261) +26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:38 INFO Executor: Finished task 167.0 in stage 14.0 (TID 258). 6563 bytes result sent to driver +26/04/01 06:08:38 INFO TaskSetManager: Starting task 171.0 in stage 14.0 (TID 262) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:38 INFO TaskSetManager: Finished task 167.0 in stage 14.0 (TID 258) in 4103 ms on 10.0.0.133 (executor driver) (168/208) +26/04/01 06:08:38 INFO Executor: Running task 171.0 in stage 14.0 (TID 262) +26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:42 INFO Executor: Finished task 168.0 in stage 14.0 (TID 259). 6563 bytes result sent to driver +26/04/01 06:08:42 INFO TaskSetManager: Starting task 172.0 in stage 14.0 (TID 263) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:42 INFO TaskSetManager: Finished task 168.0 in stage 14.0 (TID 259) in 4116 ms on 10.0.0.133 (executor driver) (169/208) +26/04/01 06:08:42 INFO Executor: Running task 172.0 in stage 14.0 (TID 263) +26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:42 INFO Executor: Finished task 169.0 in stage 14.0 (TID 260). 6563 bytes result sent to driver +26/04/01 06:08:42 INFO TaskSetManager: Starting task 173.0 in stage 14.0 (TID 264) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:42 INFO TaskSetManager: Finished task 169.0 in stage 14.0 (TID 260) in 4101 ms on 10.0.0.133 (executor driver) (170/208) +26/04/01 06:08:42 INFO Executor: Running task 173.0 in stage 14.0 (TID 264) +26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:42 INFO Executor: Finished task 170.0 in stage 14.0 (TID 261). 6563 bytes result sent to driver +26/04/01 06:08:42 INFO TaskSetManager: Starting task 174.0 in stage 14.0 (TID 265) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:08:42 INFO Executor: Running task 174.0 in stage 14.0 (TID 265) +26/04/01 06:08:42 INFO TaskSetManager: Finished task 170.0 in stage 14.0 (TID 261) in 4110 ms on 10.0.0.133 (executor driver) (171/208) +26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:42 INFO Executor: Finished task 171.0 in stage 14.0 (TID 262). 6563 bytes result sent to driver +26/04/01 06:08:42 INFO TaskSetManager: Starting task 175.0 in stage 14.0 (TID 266) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:08:42 INFO TaskSetManager: Finished task 171.0 in stage 14.0 (TID 262) in 4104 ms on 10.0.0.133 (executor driver) (172/208) +26/04/01 06:08:42 INFO Executor: Running task 175.0 in stage 14.0 (TID 266) +26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:46 INFO Executor: Finished task 172.0 in stage 14.0 (TID 263). 6563 bytes result sent to driver +26/04/01 06:08:46 INFO TaskSetManager: Starting task 176.0 in stage 14.0 (TID 267) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:46 INFO Executor: Running task 176.0 in stage 14.0 (TID 267) +26/04/01 06:08:46 INFO TaskSetManager: Finished task 172.0 in stage 14.0 (TID 263) in 4102 ms on 10.0.0.133 (executor driver) (173/208) +26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:46 INFO Executor: Finished task 173.0 in stage 14.0 (TID 264). 6563 bytes result sent to driver +26/04/01 06:08:46 INFO TaskSetManager: Starting task 177.0 in stage 14.0 (TID 268) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:46 INFO Executor: Running task 177.0 in stage 14.0 (TID 268) +26/04/01 06:08:46 INFO TaskSetManager: Finished task 173.0 in stage 14.0 (TID 264) in 4091 ms on 10.0.0.133 (executor driver) (174/208) +26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:46 INFO Executor: Finished task 174.0 in stage 14.0 (TID 265). 6563 bytes result sent to driver +26/04/01 06:08:46 INFO TaskSetManager: Starting task 178.0 in stage 14.0 (TID 269) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:46 INFO Executor: Running task 178.0 in stage 14.0 (TID 269) +26/04/01 06:08:46 INFO TaskSetManager: Finished task 174.0 in stage 14.0 (TID 265) in 4146 ms on 10.0.0.133 (executor driver) (175/208) +26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:46 INFO Executor: Finished task 175.0 in stage 14.0 (TID 266). 6563 bytes result sent to driver +26/04/01 06:08:46 INFO TaskSetManager: Starting task 179.0 in stage 14.0 (TID 270) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:46 INFO Executor: Running task 179.0 in stage 14.0 (TID 270) +26/04/01 06:08:46 INFO TaskSetManager: Finished task 175.0 in stage 14.0 (TID 266) in 4143 ms on 10.0.0.133 (executor driver) (176/208) +26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:50 INFO Executor: Finished task 176.0 in stage 14.0 (TID 267). 6563 bytes result sent to driver +26/04/01 06:08:50 INFO TaskSetManager: Starting task 180.0 in stage 14.0 (TID 271) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:08:50 INFO Executor: Running task 180.0 in stage 14.0 (TID 271) +26/04/01 06:08:50 INFO TaskSetManager: Finished task 176.0 in stage 14.0 (TID 267) in 4158 ms on 10.0.0.133 (executor driver) (177/208) +26/04/01 06:08:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:50 INFO Executor: Finished task 177.0 in stage 14.0 (TID 268). 6563 bytes result sent to driver +26/04/01 06:08:50 INFO TaskSetManager: Starting task 181.0 in stage 14.0 (TID 272) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:50 INFO TaskSetManager: Finished task 177.0 in stage 14.0 (TID 268) in 4148 ms on 10.0.0.133 (executor driver) (178/208) +26/04/01 06:08:50 INFO Executor: Running task 181.0 in stage 14.0 (TID 272) +26/04/01 06:08:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:51 INFO Executor: Finished task 178.0 in stage 14.0 (TID 269). 6563 bytes result sent to driver +26/04/01 06:08:51 INFO TaskSetManager: Starting task 182.0 in stage 14.0 (TID 273) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:51 INFO TaskSetManager: Finished task 178.0 in stage 14.0 (TID 269) in 4151 ms on 10.0.0.133 (executor driver) (179/208) +26/04/01 06:08:51 INFO Executor: Running task 182.0 in stage 14.0 (TID 273) +26/04/01 06:08:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:51 INFO Executor: Finished task 179.0 in stage 14.0 (TID 270). 6563 bytes result sent to driver +26/04/01 06:08:51 INFO TaskSetManager: Starting task 183.0 in stage 14.0 (TID 274) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:51 INFO TaskSetManager: Finished task 179.0 in stage 14.0 (TID 270) in 4147 ms on 10.0.0.133 (executor driver) (180/208) +26/04/01 06:08:51 INFO Executor: Running task 183.0 in stage 14.0 (TID 274) +26/04/01 06:08:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:55 INFO Executor: Finished task 180.0 in stage 14.0 (TID 271). 6563 bytes result sent to driver +26/04/01 06:08:55 INFO TaskSetManager: Starting task 184.0 in stage 14.0 (TID 275) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:55 INFO TaskSetManager: Finished task 180.0 in stage 14.0 (TID 271) in 4147 ms on 10.0.0.133 (executor driver) (181/208) +26/04/01 06:08:55 INFO Executor: Running task 184.0 in stage 14.0 (TID 275) +26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:55 INFO Executor: Finished task 181.0 in stage 14.0 (TID 272). 6563 bytes result sent to driver +26/04/01 06:08:55 INFO TaskSetManager: Starting task 185.0 in stage 14.0 (TID 276) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:55 INFO TaskSetManager: Finished task 181.0 in stage 14.0 (TID 272) in 4138 ms on 10.0.0.133 (executor driver) (182/208) +26/04/01 06:08:55 INFO Executor: Running task 185.0 in stage 14.0 (TID 276) +26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:55 INFO Executor: Finished task 182.0 in stage 14.0 (TID 273). 6563 bytes result sent to driver +26/04/01 06:08:55 INFO TaskSetManager: Starting task 186.0 in stage 14.0 (TID 277) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:08:55 INFO Executor: Running task 186.0 in stage 14.0 (TID 277) +26/04/01 06:08:55 INFO TaskSetManager: Finished task 182.0 in stage 14.0 (TID 273) in 4137 ms on 10.0.0.133 (executor driver) (183/208) +26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:55 INFO Executor: Finished task 183.0 in stage 14.0 (TID 274). 6563 bytes result sent to driver +26/04/01 06:08:55 INFO TaskSetManager: Starting task 187.0 in stage 14.0 (TID 278) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:08:55 INFO TaskSetManager: Finished task 183.0 in stage 14.0 (TID 274) in 4143 ms on 10.0.0.133 (executor driver) (184/208) +26/04/01 06:08:55 INFO Executor: Running task 187.0 in stage 14.0 (TID 278) +26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:59 INFO Executor: Finished task 184.0 in stage 14.0 (TID 275). 6563 bytes result sent to driver +26/04/01 06:08:59 INFO TaskSetManager: Starting task 188.0 in stage 14.0 (TID 279) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:59 INFO TaskSetManager: Finished task 184.0 in stage 14.0 (TID 275) in 4155 ms on 10.0.0.133 (executor driver) (185/208) +26/04/01 06:08:59 INFO Executor: Running task 188.0 in stage 14.0 (TID 279) +26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:59 INFO Executor: Finished task 185.0 in stage 14.0 (TID 276). 6563 bytes result sent to driver +26/04/01 06:08:59 INFO TaskSetManager: Starting task 189.0 in stage 14.0 (TID 280) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:59 INFO TaskSetManager: Finished task 185.0 in stage 14.0 (TID 276) in 4142 ms on 10.0.0.133 (executor driver) (186/208) +26/04/01 06:08:59 INFO Executor: Running task 189.0 in stage 14.0 (TID 280) +26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:59 INFO Executor: Finished task 186.0 in stage 14.0 (TID 277). 6563 bytes result sent to driver +26/04/01 06:08:59 INFO TaskSetManager: Starting task 190.0 in stage 14.0 (TID 281) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:59 INFO TaskSetManager: Finished task 186.0 in stage 14.0 (TID 277) in 4117 ms on 10.0.0.133 (executor driver) (187/208) +26/04/01 06:08:59 INFO Executor: Running task 190.0 in stage 14.0 (TID 281) +26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:59 INFO Executor: Finished task 187.0 in stage 14.0 (TID 278). 6563 bytes result sent to driver +26/04/01 06:08:59 INFO TaskSetManager: Starting task 191.0 in stage 14.0 (TID 282) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:08:59 INFO TaskSetManager: Finished task 187.0 in stage 14.0 (TID 278) in 4117 ms on 10.0.0.133 (executor driver) (188/208) +26/04/01 06:08:59 INFO Executor: Running task 191.0 in stage 14.0 (TID 282) +26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:03 INFO Executor: Finished task 188.0 in stage 14.0 (TID 279). 6563 bytes result sent to driver +26/04/01 06:09:03 INFO TaskSetManager: Starting task 192.0 in stage 14.0 (TID 283) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:09:03 INFO TaskSetManager: Finished task 188.0 in stage 14.0 (TID 279) in 4108 ms on 10.0.0.133 (executor driver) (189/208) +26/04/01 06:09:03 INFO Executor: Running task 192.0 in stage 14.0 (TID 283) +26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:03 INFO Executor: Finished task 189.0 in stage 14.0 (TID 280). 6563 bytes result sent to driver +26/04/01 06:09:03 INFO TaskSetManager: Starting task 193.0 in stage 14.0 (TID 284) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:09:03 INFO TaskSetManager: Finished task 189.0 in stage 14.0 (TID 280) in 4112 ms on 10.0.0.133 (executor driver) (190/208) +26/04/01 06:09:03 INFO Executor: Running task 193.0 in stage 14.0 (TID 284) +26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:03 INFO Executor: Finished task 190.0 in stage 14.0 (TID 281). 6563 bytes result sent to driver +26/04/01 06:09:03 INFO TaskSetManager: Starting task 194.0 in stage 14.0 (TID 285) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:09:03 INFO TaskSetManager: Finished task 190.0 in stage 14.0 (TID 281) in 4105 ms on 10.0.0.133 (executor driver) (191/208) +26/04/01 06:09:03 INFO Executor: Running task 194.0 in stage 14.0 (TID 285) +26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:03 INFO Executor: Finished task 191.0 in stage 14.0 (TID 282). 6563 bytes result sent to driver +26/04/01 06:09:03 INFO TaskSetManager: Starting task 195.0 in stage 14.0 (TID 286) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:09:03 INFO Executor: Running task 195.0 in stage 14.0 (TID 286) +26/04/01 06:09:03 INFO TaskSetManager: Finished task 191.0 in stage 14.0 (TID 282) in 4105 ms on 10.0.0.133 (executor driver) (192/208) +26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:05 INFO Executor: Finished task 192.0 in stage 14.0 (TID 283). 6563 bytes result sent to driver +26/04/01 06:09:05 INFO TaskSetManager: Starting task 196.0 in stage 14.0 (TID 287) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:09:05 INFO TaskSetManager: Finished task 192.0 in stage 14.0 (TID 283) in 2584 ms on 10.0.0.133 (executor driver) (193/208) +26/04/01 06:09:05 INFO Executor: Running task 196.0 in stage 14.0 (TID 287) +26/04/01 06:09:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:05 INFO Executor: Finished task 193.0 in stage 14.0 (TID 284). 6563 bytes result sent to driver +26/04/01 06:09:05 INFO TaskSetManager: Starting task 197.0 in stage 14.0 (TID 288) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:09:05 INFO TaskSetManager: Finished task 193.0 in stage 14.0 (TID 284) in 2580 ms on 10.0.0.133 (executor driver) (194/208) +26/04/01 06:09:05 INFO Executor: Running task 197.0 in stage 14.0 (TID 288) +26/04/01 06:09:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:05 INFO Executor: Finished task 194.0 in stage 14.0 (TID 285). 6563 bytes result sent to driver +26/04/01 06:09:05 INFO TaskSetManager: Starting task 198.0 in stage 14.0 (TID 289) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:09:05 INFO TaskSetManager: Finished task 194.0 in stage 14.0 (TID 285) in 2585 ms on 10.0.0.133 (executor driver) (195/208) +26/04/01 06:09:05 INFO Executor: Running task 198.0 in stage 14.0 (TID 289) +26/04/01 06:09:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:06 INFO Executor: Finished task 195.0 in stage 14.0 (TID 286). 6563 bytes result sent to driver +26/04/01 06:09:06 INFO TaskSetManager: Starting task 199.0 in stage 14.0 (TID 290) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:09:06 INFO TaskSetManager: Finished task 195.0 in stage 14.0 (TID 286) in 2585 ms on 10.0.0.133 (executor driver) (196/208) +26/04/01 06:09:06 INFO Executor: Running task 199.0 in stage 14.0 (TID 290) +26/04/01 06:09:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:08 INFO Executor: Finished task 197.0 in stage 14.0 (TID 288). 6606 bytes result sent to driver +26/04/01 06:09:08 INFO TaskSetManager: Starting task 200.0 in stage 14.0 (TID 291) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9346 bytes) +26/04/01 06:09:08 INFO TaskSetManager: Finished task 197.0 in stage 14.0 (TID 288) in 2569 ms on 10.0.0.133 (executor driver) (197/208) +26/04/01 06:09:08 INFO Executor: Running task 200.0 in stage 14.0 (TID 291) +26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:08 INFO Executor: Finished task 196.0 in stage 14.0 (TID 287). 6606 bytes result sent to driver +26/04/01 06:09:08 INFO TaskSetManager: Starting task 201.0 in stage 14.0 (TID 292) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:09:08 INFO TaskSetManager: Finished task 196.0 in stage 14.0 (TID 287) in 2577 ms on 10.0.0.133 (executor driver) (198/208) +26/04/01 06:09:08 INFO Executor: Running task 201.0 in stage 14.0 (TID 292) +26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:08 INFO Executor: Finished task 198.0 in stage 14.0 (TID 289). 6606 bytes result sent to driver +26/04/01 06:09:08 INFO TaskSetManager: Starting task 202.0 in stage 14.0 (TID 293) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:09:08 INFO TaskSetManager: Finished task 198.0 in stage 14.0 (TID 289) in 2576 ms on 10.0.0.133 (executor driver) (199/208) +26/04/01 06:09:08 INFO Executor: Running task 202.0 in stage 14.0 (TID 293) +26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:08 INFO Executor: Finished task 199.0 in stage 14.0 (TID 290). 6606 bytes result sent to driver +26/04/01 06:09:08 INFO TaskSetManager: Starting task 203.0 in stage 14.0 (TID 294) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:09:08 INFO TaskSetManager: Finished task 199.0 in stage 14.0 (TID 290) in 2578 ms on 10.0.0.133 (executor driver) (200/208) +26/04/01 06:09:08 INFO Executor: Running task 203.0 in stage 14.0 (TID 294) +26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:11 INFO Executor: Finished task 200.0 in stage 14.0 (TID 291). 6563 bytes result sent to driver +26/04/01 06:09:11 INFO Executor: Finished task 201.0 in stage 14.0 (TID 292). 6563 bytes result sent to driver +26/04/01 06:09:11 INFO TaskSetManager: Starting task 204.0 in stage 14.0 (TID 295) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:09:11 INFO TaskSetManager: Starting task 205.0 in stage 14.0 (TID 296) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:09:11 INFO Executor: Running task 204.0 in stage 14.0 (TID 295) +26/04/01 06:09:11 INFO TaskSetManager: Finished task 200.0 in stage 14.0 (TID 291) in 2582 ms on 10.0.0.133 (executor driver) (201/208) +26/04/01 06:09:11 INFO Executor: Running task 205.0 in stage 14.0 (TID 296) +26/04/01 06:09:11 INFO TaskSetManager: Finished task 201.0 in stage 14.0 (TID 292) in 2576 ms on 10.0.0.133 (executor driver) (202/208) +26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:11 INFO Executor: Finished task 202.0 in stage 14.0 (TID 293). 6563 bytes result sent to driver +26/04/01 06:09:11 INFO TaskSetManager: Starting task 206.0 in stage 14.0 (TID 297) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:09:11 INFO TaskSetManager: Finished task 202.0 in stage 14.0 (TID 293) in 2561 ms on 10.0.0.133 (executor driver) (203/208) +26/04/01 06:09:11 INFO Executor: Running task 206.0 in stage 14.0 (TID 297) +26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:11 INFO Executor: Finished task 203.0 in stage 14.0 (TID 294). 6563 bytes result sent to driver +26/04/01 06:09:11 INFO TaskSetManager: Starting task 207.0 in stage 14.0 (TID 298) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:09:11 INFO TaskSetManager: Finished task 203.0 in stage 14.0 (TID 294) in 2563 ms on 10.0.0.133 (executor driver) (204/208) +26/04/01 06:09:11 INFO Executor: Running task 207.0 in stage 14.0 (TID 298) +26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO Executor: Finished task 205.0 in stage 14.0 (TID 296). 6563 bytes result sent to driver +26/04/01 06:09:13 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 299) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:09:13 INFO TaskSetManager: Finished task 205.0 in stage 14.0 (TID 296) in 2554 ms on 10.0.0.133 (executor driver) (205/208) +26/04/01 06:09:13 INFO Executor: Running task 0.0 in stage 15.0 (TID 299) +26/04/01 06:09:13 INFO Executor: Finished task 204.0 in stage 14.0 (TID 295). 6563 bytes result sent to driver +26/04/01 06:09:13 INFO TaskSetManager: Starting task 0.0 in stage 16.0 (TID 300) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9722 bytes) +26/04/01 06:09:13 INFO Executor: Running task 0.0 in stage 16.0 (TID 300) +26/04/01 06:09:13 INFO TaskSetManager: Finished task 204.0 in stage 14.0 (TID 295) in 2555 ms on 10.0.0.133 (executor driver) (206/208) +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO Executor: Finished task 0.0 in stage 15.0 (TID 299). 11159 bytes result sent to driver +26/04/01 06:09:13 INFO TaskSetManager: Starting task 1.0 in stage 16.0 (TID 301) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9724 bytes) +26/04/01 06:09:13 INFO Executor: Running task 1.0 in stage 16.0 (TID 301) +26/04/01 06:09:13 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 299) in 14 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:09:13 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool +26/04/01 06:09:13 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 228.857 s +26/04/01 06:09:13 INFO DAGScheduler: Job 14 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:09:13 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished +26/04/01 06:09:13 INFO DAGScheduler: Job 14 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 228.856920 s +26/04/01 06:09:13 INFO Utils: Coalesced 1 broadcast batches into 1 (5 rows) +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO MemoryStore: Block broadcast_25 stored as values in memory (estimated size 608.0 B, free 8.4 GiB) +26/04/01 06:09:13 INFO MemoryStore: Block broadcast_25_piece0 stored as bytes in memory (estimated size 563.0 B, free 8.4 GiB) +26/04/01 06:09:13 INFO BlockManagerInfo: Added broadcast_25_piece0 in memory on 10.0.0.133:58259 (size: 563.0 B, free: 8.6 GiB) +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO SparkContext: Created broadcast 25 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:09:13 INFO Executor: Finished task 206.0 in stage 14.0 (TID 297). 6563 bytes result sent to driver +26/04/01 06:09:13 INFO TaskSetManager: Starting task 2.0 in stage 16.0 (TID 302) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9724 bytes) +26/04/01 06:09:13 INFO TaskSetManager: Finished task 206.0 in stage 14.0 (TID 297) in 2559 ms on 10.0.0.133 (executor driver) (207/208) +26/04/01 06:09:13 INFO Executor: Running task 2.0 in stage 16.0 (TID 302) +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO Executor: Finished task 207.0 in stage 14.0 (TID 298). 6563 bytes result sent to driver +26/04/01 06:09:13 INFO TaskSetManager: Starting task 3.0 in stage 16.0 (TID 303) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9721 bytes) +26/04/01 06:09:13 INFO TaskSetManager: Finished task 207.0 in stage 14.0 (TID 298) in 2552 ms on 10.0.0.133 (executor driver) (208/208) +26/04/01 06:09:13 INFO TaskSchedulerImpl: Removed TaskSet 14.0, whose tasks have all completed, from pool +26/04/01 06:09:13 INFO Executor: Running task 3.0 in stage 16.0 (TID 303) +26/04/01 06:09:13 INFO DAGScheduler: ShuffleMapStage 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 229.022 s +26/04/01 06:09:13 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:09:13 INFO DAGScheduler: running: Set(ShuffleMapStage 19, ShuffleMapStage 16) +26/04/01 06:09:13 INFO DAGScheduler: waiting: Set() +26/04/01 06:09:13 INFO DAGScheduler: failed: Set() +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO Executor: Finished task 0.0 in stage 16.0 (TID 300). 6563 bytes result sent to driver +26/04/01 06:09:13 INFO TaskSetManager: Starting task 0.0 in stage 19.0 (TID 304) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:13 INFO Executor: Running task 0.0 in stage 19.0 (TID 304) +26/04/01 06:09:13 INFO TaskSetManager: Finished task 0.0 in stage 16.0 (TID 300) in 211 ms on 10.0.0.133 (executor driver) (1/4) +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO Executor: Finished task 1.0 in stage 16.0 (TID 301). 6563 bytes result sent to driver +26/04/01 06:09:13 INFO TaskSetManager: Starting task 1.0 in stage 19.0 (TID 305) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:13 INFO TaskSetManager: Finished task 1.0 in stage 16.0 (TID 301) in 213 ms on 10.0.0.133 (executor driver) (2/4) +26/04/01 06:09:13 INFO Executor: Running task 1.0 in stage 19.0 (TID 305) +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO Executor: Finished task 2.0 in stage 16.0 (TID 302). 6563 bytes result sent to driver +26/04/01 06:09:13 INFO TaskSetManager: Starting task 2.0 in stage 19.0 (TID 306) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:13 INFO TaskSetManager: Finished task 2.0 in stage 16.0 (TID 302) in 212 ms on 10.0.0.133 (executor driver) (3/4) +26/04/01 06:09:13 INFO Executor: Running task 2.0 in stage 19.0 (TID 306) +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO Executor: Finished task 3.0 in stage 16.0 (TID 303). 6563 bytes result sent to driver +26/04/01 06:09:13 INFO TaskSetManager: Starting task 3.0 in stage 19.0 (TID 307) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:13 INFO Executor: Running task 3.0 in stage 19.0 (TID 307) +26/04/01 06:09:13 INFO TaskSetManager: Finished task 3.0 in stage 16.0 (TID 303) in 205 ms on 10.0.0.133 (executor driver) (4/4) +26/04/01 06:09:13 INFO TaskSchedulerImpl: Removed TaskSet 16.0, whose tasks have all completed, from pool +26/04/01 06:09:13 INFO DAGScheduler: ShuffleMapStage 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 229.211 s +26/04/01 06:09:13 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:09:13 INFO DAGScheduler: running: Set(ShuffleMapStage 19) +26/04/01 06:09:13 INFO DAGScheduler: waiting: Set() +26/04/01 06:09:13 INFO DAGScheduler: failed: Set() +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:14 INFO BlockManagerInfo: Removed broadcast_20_piece0 on 10.0.0.133:58259 in memory (size: 8.0 KiB, free: 8.6 GiB) +26/04/01 06:09:14 INFO BlockManagerInfo: Removed broadcast_21_piece0 on 10.0.0.133:58259 in memory (size: 30.7 KiB, free: 8.6 GiB) +26/04/01 06:09:14 INFO BlockManagerInfo: Removed broadcast_22_piece0 on 10.0.0.133:58259 in memory (size: 7.8 KiB, free: 8.6 GiB) +26/04/01 06:09:25 INFO Executor: Finished task 0.0 in stage 19.0 (TID 304). 9594 bytes result sent to driver +26/04/01 06:09:25 INFO TaskSetManager: Starting task 4.0 in stage 19.0 (TID 308) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:25 INFO Executor: Running task 4.0 in stage 19.0 (TID 308) +26/04/01 06:09:25 INFO TaskSetManager: Finished task 0.0 in stage 19.0 (TID 304) in 11253 ms on 10.0.0.133 (executor driver) (1/6) +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:25 INFO Executor: Finished task 1.0 in stage 19.0 (TID 305). 9594 bytes result sent to driver +26/04/01 06:09:25 INFO TaskSetManager: Starting task 5.0 in stage 19.0 (TID 309) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:25 INFO TaskSetManager: Finished task 1.0 in stage 19.0 (TID 305) in 11584 ms on 10.0.0.133 (executor driver) (2/6) +26/04/01 06:09:25 INFO Executor: Running task 5.0 in stage 19.0 (TID 309) +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 16 (11.1 MiB) non-empty blocks including 16 (11.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 64 (23.7 MiB) non-empty blocks including 64 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 16 (11.1 MiB) non-empty blocks including 16 (11.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 64 (23.7 MiB) non-empty blocks including 64 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:25 INFO Executor: Finished task 3.0 in stage 19.0 (TID 307). 9594 bytes result sent to driver +26/04/01 06:09:25 INFO TaskSetManager: Finished task 3.0 in stage 19.0 (TID 307) in 11954 ms on 10.0.0.133 (executor driver) (3/6) +26/04/01 06:09:25 INFO Executor: Finished task 2.0 in stage 19.0 (TID 306). 9594 bytes result sent to driver +26/04/01 06:09:25 INFO TaskSetManager: Finished task 2.0 in stage 19.0 (TID 306) in 12054 ms on 10.0.0.133 (executor driver) (4/6) +26/04/01 06:09:30 INFO Executor: Finished task 5.0 in stage 19.0 (TID 309). 9594 bytes result sent to driver +26/04/01 06:09:30 INFO TaskSetManager: Finished task 5.0 in stage 19.0 (TID 309) in 5491 ms on 10.0.0.133 (executor driver) (5/6) +26/04/01 06:09:35 INFO Executor: Finished task 4.0 in stage 19.0 (TID 308). 9594 bytes result sent to driver +26/04/01 06:09:35 INFO TaskSetManager: Finished task 4.0 in stage 19.0 (TID 308) in 10141 ms on 10.0.0.133 (executor driver) (6/6) +26/04/01 06:09:35 INFO TaskSchedulerImpl: Removed TaskSet 19.0, whose tasks have all completed, from pool +26/04/01 06:09:35 INFO DAGScheduler: ShuffleMapStage 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 229.847 s +26/04/01 06:09:35 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:09:35 INFO DAGScheduler: running: Set() +26/04/01 06:09:35 INFO DAGScheduler: waiting: Set() +26/04/01 06:09:35 INFO DAGScheduler: failed: Set() +26/04/01 06:09:35 INFO ShufflePartitionsUtil: For shuffle(5, 3), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 06:09:35 INFO DAGScheduler: Registering RDD 48 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 6 +26/04/01 06:09:35 INFO DAGScheduler: Got map stage job 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions +26/04/01 06:09:35 INFO DAGScheduler: Final stage: ShuffleMapStage 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:09:35 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 22, ShuffleMapStage 23) +26/04/01 06:09:35 INFO DAGScheduler: Missing parents: List() +26/04/01 06:09:35 INFO DAGScheduler: Submitting ShuffleMapStage 24 (MapPartitionsRDD[48] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:09:35 INFO MemoryStore: Block broadcast_26 stored as values in memory (estimated size 29.4 KiB, free 8.6 GiB) +26/04/01 06:09:35 INFO MemoryStore: Block broadcast_26_piece0 stored as bytes in memory (estimated size 11.7 KiB, free 8.6 GiB) +26/04/01 06:09:35 INFO BlockManagerInfo: Added broadcast_26_piece0 in memory on 10.0.0.133:58259 (size: 11.7 KiB, free: 8.6 GiB) +26/04/01 06:09:35 INFO SparkContext: Created broadcast 26 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:09:35 INFO DAGScheduler: Submitting 200 missing tasks from ShuffleMapStage 24 (MapPartitionsRDD[48] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 06:09:35 INFO TaskSchedulerImpl: Adding task set 24.0 with 200 tasks resource profile 0 +26/04/01 06:09:35 INFO TaskSetManager: Starting task 0.0 in stage 24.0 (TID 310) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:35 INFO TaskSetManager: Starting task 1.0 in stage 24.0 (TID 311) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:35 INFO TaskSetManager: Starting task 2.0 in stage 24.0 (TID 312) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:35 INFO TaskSetManager: Starting task 3.0 in stage 24.0 (TID 313) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:35 INFO Executor: Running task 0.0 in stage 24.0 (TID 310) +26/04/01 06:09:35 INFO Executor: Running task 2.0 in stage 24.0 (TID 312) +26/04/01 06:09:35 INFO Executor: Running task 1.0 in stage 24.0 (TID 311) +26/04/01 06:09:35 INFO Executor: Running task 3.0 in stage 24.0 (TID 313) +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:38 INFO Executor: Finished task 1.0 in stage 24.0 (TID 311). 9201 bytes result sent to driver +26/04/01 06:09:38 INFO Executor: Finished task 0.0 in stage 24.0 (TID 310). 9201 bytes result sent to driver +26/04/01 06:09:38 INFO TaskSetManager: Starting task 4.0 in stage 24.0 (TID 314) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:38 INFO Executor: Running task 4.0 in stage 24.0 (TID 314) +26/04/01 06:09:38 INFO TaskSetManager: Starting task 5.0 in stage 24.0 (TID 315) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:38 INFO TaskSetManager: Finished task 1.0 in stage 24.0 (TID 311) in 3572 ms on 10.0.0.133 (executor driver) (1/200) +26/04/01 06:09:38 INFO Executor: Running task 5.0 in stage 24.0 (TID 315) +26/04/01 06:09:38 INFO TaskSetManager: Finished task 0.0 in stage 24.0 (TID 310) in 3572 ms on 10.0.0.133 (executor driver) (2/200) +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO Executor: Finished task 2.0 in stage 24.0 (TID 312). 9201 bytes result sent to driver +26/04/01 06:09:38 INFO TaskSetManager: Starting task 6.0 in stage 24.0 (TID 316) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:38 INFO TaskSetManager: Finished task 2.0 in stage 24.0 (TID 312) in 3584 ms on 10.0.0.133 (executor driver) (3/200) +26/04/01 06:09:38 INFO Executor: Running task 6.0 in stage 24.0 (TID 316) +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO Executor: Finished task 3.0 in stage 24.0 (TID 313). 9201 bytes result sent to driver +26/04/01 06:09:38 INFO TaskSetManager: Starting task 7.0 in stage 24.0 (TID 317) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:38 INFO Executor: Running task 7.0 in stage 24.0 (TID 317) +26/04/01 06:09:38 INFO TaskSetManager: Finished task 3.0 in stage 24.0 (TID 313) in 3589 ms on 10.0.0.133 (executor driver) (4/200) +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:42 INFO Executor: Finished task 6.0 in stage 24.0 (TID 316). 8470 bytes result sent to driver +26/04/01 06:09:42 INFO TaskSetManager: Starting task 8.0 in stage 24.0 (TID 318) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:42 INFO Executor: Running task 8.0 in stage 24.0 (TID 318) +26/04/01 06:09:42 INFO TaskSetManager: Finished task 6.0 in stage 24.0 (TID 316) in 3535 ms on 10.0.0.133 (executor driver) (5/200) +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:42 INFO Executor: Finished task 4.0 in stage 24.0 (TID 314). 8470 bytes result sent to driver +26/04/01 06:09:42 INFO TaskSetManager: Starting task 9.0 in stage 24.0 (TID 319) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:42 INFO TaskSetManager: Finished task 4.0 in stage 24.0 (TID 314) in 3564 ms on 10.0.0.133 (executor driver) (6/200) +26/04/01 06:09:42 INFO Executor: Running task 9.0 in stage 24.0 (TID 319) +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:42 INFO Executor: Finished task 7.0 in stage 24.0 (TID 317). 8470 bytes result sent to driver +26/04/01 06:09:42 INFO TaskSetManager: Starting task 10.0 in stage 24.0 (TID 320) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:42 INFO Executor: Running task 10.0 in stage 24.0 (TID 320) +26/04/01 06:09:42 INFO TaskSetManager: Finished task 7.0 in stage 24.0 (TID 317) in 3562 ms on 10.0.0.133 (executor driver) (7/200) +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:42 INFO Executor: Finished task 5.0 in stage 24.0 (TID 315). 8470 bytes result sent to driver +26/04/01 06:09:42 INFO TaskSetManager: Starting task 11.0 in stage 24.0 (TID 321) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:42 INFO Executor: Running task 11.0 in stage 24.0 (TID 321) +26/04/01 06:09:42 INFO TaskSetManager: Finished task 5.0 in stage 24.0 (TID 315) in 3591 ms on 10.0.0.133 (executor driver) (8/200) +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:45 INFO Executor: Finished task 8.0 in stage 24.0 (TID 318). 8470 bytes result sent to driver +26/04/01 06:09:45 INFO TaskSetManager: Starting task 12.0 in stage 24.0 (TID 322) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:45 INFO Executor: Running task 12.0 in stage 24.0 (TID 322) +26/04/01 06:09:45 INFO TaskSetManager: Finished task 8.0 in stage 24.0 (TID 318) in 3495 ms on 10.0.0.133 (executor driver) (9/200) +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO Executor: Finished task 9.0 in stage 24.0 (TID 319). 8470 bytes result sent to driver +26/04/01 06:09:45 INFO TaskSetManager: Starting task 13.0 in stage 24.0 (TID 323) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:45 INFO Executor: Running task 13.0 in stage 24.0 (TID 323) +26/04/01 06:09:45 INFO TaskSetManager: Finished task 9.0 in stage 24.0 (TID 319) in 3489 ms on 10.0.0.133 (executor driver) (10/200) +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:45 INFO Executor: Finished task 10.0 in stage 24.0 (TID 320). 8470 bytes result sent to driver +26/04/01 06:09:45 INFO TaskSetManager: Starting task 14.0 in stage 24.0 (TID 324) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:45 INFO TaskSetManager: Finished task 10.0 in stage 24.0 (TID 320) in 3488 ms on 10.0.0.133 (executor driver) (11/200) +26/04/01 06:09:45 INFO Executor: Running task 14.0 in stage 24.0 (TID 324) +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1027.0 KiB) non-empty blocks including 6 (1027.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1027.0 KiB) non-empty blocks including 6 (1027.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:45 INFO Executor: Finished task 11.0 in stage 24.0 (TID 321). 8470 bytes result sent to driver +26/04/01 06:09:45 INFO TaskSetManager: Starting task 15.0 in stage 24.0 (TID 325) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:45 INFO Executor: Running task 15.0 in stage 24.0 (TID 325) +26/04/01 06:09:45 INFO TaskSetManager: Finished task 11.0 in stage 24.0 (TID 321) in 3497 ms on 10.0.0.133 (executor driver) (12/200) +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:49 INFO Executor: Finished task 12.0 in stage 24.0 (TID 322). 9201 bytes result sent to driver +26/04/01 06:09:49 INFO TaskSetManager: Starting task 16.0 in stage 24.0 (TID 326) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:49 INFO TaskSetManager: Finished task 12.0 in stage 24.0 (TID 322) in 3497 ms on 10.0.0.133 (executor driver) (13/200) +26/04/01 06:09:49 INFO Executor: Running task 16.0 in stage 24.0 (TID 326) +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO Executor: Finished task 13.0 in stage 24.0 (TID 323). 9201 bytes result sent to driver +26/04/01 06:09:49 INFO TaskSetManager: Starting task 17.0 in stage 24.0 (TID 327) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:49 INFO TaskSetManager: Finished task 13.0 in stage 24.0 (TID 323) in 3497 ms on 10.0.0.133 (executor driver) (14/200) +26/04/01 06:09:49 INFO Executor: Running task 17.0 in stage 24.0 (TID 327) +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:49 INFO Executor: Finished task 14.0 in stage 24.0 (TID 324). 9201 bytes result sent to driver +26/04/01 06:09:49 INFO TaskSetManager: Starting task 18.0 in stage 24.0 (TID 328) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:49 INFO TaskSetManager: Finished task 14.0 in stage 24.0 (TID 324) in 3530 ms on 10.0.0.133 (executor driver) (15/200) +26/04/01 06:09:49 INFO Executor: Running task 18.0 in stage 24.0 (TID 328) +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO Executor: Finished task 15.0 in stage 24.0 (TID 325). 9201 bytes result sent to driver +26/04/01 06:09:49 INFO TaskSetManager: Starting task 19.0 in stage 24.0 (TID 329) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:49 INFO Executor: Running task 19.0 in stage 24.0 (TID 329) +26/04/01 06:09:49 INFO TaskSetManager: Finished task 15.0 in stage 24.0 (TID 325) in 3519 ms on 10.0.0.133 (executor driver) (16/200) +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:52 INFO Executor: Finished task 16.0 in stage 24.0 (TID 326). 8470 bytes result sent to driver +26/04/01 06:09:52 INFO TaskSetManager: Starting task 20.0 in stage 24.0 (TID 330) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:52 INFO Executor: Running task 20.0 in stage 24.0 (TID 330) +26/04/01 06:09:52 INFO TaskSetManager: Finished task 16.0 in stage 24.0 (TID 326) in 3468 ms on 10.0.0.133 (executor driver) (17/200) +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:52 INFO Executor: Finished task 17.0 in stage 24.0 (TID 327). 8470 bytes result sent to driver +26/04/01 06:09:52 INFO TaskSetManager: Starting task 21.0 in stage 24.0 (TID 331) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:52 INFO Executor: Running task 21.0 in stage 24.0 (TID 331) +26/04/01 06:09:52 INFO TaskSetManager: Finished task 17.0 in stage 24.0 (TID 327) in 3477 ms on 10.0.0.133 (executor driver) (18/200) +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:52 INFO Executor: Finished task 19.0 in stage 24.0 (TID 329). 8470 bytes result sent to driver +26/04/01 06:09:52 INFO TaskSetManager: Starting task 22.0 in stage 24.0 (TID 332) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:52 INFO TaskSetManager: Finished task 19.0 in stage 24.0 (TID 329) in 3475 ms on 10.0.0.133 (executor driver) (19/200) +26/04/01 06:09:52 INFO Executor: Running task 22.0 in stage 24.0 (TID 332) +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO Executor: Finished task 18.0 in stage 24.0 (TID 328). 8470 bytes result sent to driver +26/04/01 06:09:52 INFO TaskSetManager: Starting task 23.0 in stage 24.0 (TID 333) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:52 INFO Executor: Running task 23.0 in stage 24.0 (TID 333) +26/04/01 06:09:52 INFO TaskSetManager: Finished task 18.0 in stage 24.0 (TID 328) in 3486 ms on 10.0.0.133 (executor driver) (20/200) +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:56 INFO Executor: Finished task 20.0 in stage 24.0 (TID 330). 9201 bytes result sent to driver +26/04/01 06:09:56 INFO TaskSetManager: Starting task 24.0 in stage 24.0 (TID 334) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:56 INFO TaskSetManager: Finished task 20.0 in stage 24.0 (TID 330) in 3473 ms on 10.0.0.133 (executor driver) (21/200) +26/04/01 06:09:56 INFO Executor: Running task 24.0 in stage 24.0 (TID 334) +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO Executor: Finished task 21.0 in stage 24.0 (TID 331). 9201 bytes result sent to driver +26/04/01 06:09:56 INFO TaskSetManager: Starting task 25.0 in stage 24.0 (TID 335) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:56 INFO Executor: Running task 25.0 in stage 24.0 (TID 335) +26/04/01 06:09:56 INFO TaskSetManager: Finished task 21.0 in stage 24.0 (TID 331) in 3460 ms on 10.0.0.133 (executor driver) (22/200) +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:56 INFO Executor: Finished task 22.0 in stage 24.0 (TID 332). 9201 bytes result sent to driver +26/04/01 06:09:56 INFO TaskSetManager: Starting task 26.0 in stage 24.0 (TID 336) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:56 INFO Executor: Running task 26.0 in stage 24.0 (TID 336) +26/04/01 06:09:56 INFO TaskSetManager: Finished task 22.0 in stage 24.0 (TID 332) in 3467 ms on 10.0.0.133 (executor driver) (23/200) +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO Executor: Finished task 23.0 in stage 24.0 (TID 333). 9201 bytes result sent to driver +26/04/01 06:09:56 INFO TaskSetManager: Starting task 27.0 in stage 24.0 (TID 337) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:56 INFO TaskSetManager: Finished task 23.0 in stage 24.0 (TID 333) in 3469 ms on 10.0.0.133 (executor driver) (24/200) +26/04/01 06:09:56 INFO Executor: Running task 27.0 in stage 24.0 (TID 337) +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:59 INFO Executor: Finished task 25.0 in stage 24.0 (TID 335). 8470 bytes result sent to driver +26/04/01 06:09:59 INFO TaskSetManager: Starting task 28.0 in stage 24.0 (TID 338) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:59 INFO Executor: Running task 28.0 in stage 24.0 (TID 338) +26/04/01 06:09:59 INFO TaskSetManager: Finished task 25.0 in stage 24.0 (TID 335) in 3485 ms on 10.0.0.133 (executor driver) (25/200) +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:59 INFO Executor: Finished task 24.0 in stage 24.0 (TID 334). 8470 bytes result sent to driver +26/04/01 06:09:59 INFO TaskSetManager: Starting task 29.0 in stage 24.0 (TID 339) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:59 INFO TaskSetManager: Finished task 24.0 in stage 24.0 (TID 334) in 3504 ms on 10.0.0.133 (executor driver) (26/200) +26/04/01 06:09:59 INFO Executor: Running task 29.0 in stage 24.0 (TID 339) +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:59 INFO Executor: Finished task 26.0 in stage 24.0 (TID 336). 8470 bytes result sent to driver +26/04/01 06:09:59 INFO TaskSetManager: Starting task 30.0 in stage 24.0 (TID 340) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:59 INFO TaskSetManager: Finished task 26.0 in stage 24.0 (TID 336) in 3487 ms on 10.0.0.133 (executor driver) (27/200) +26/04/01 06:09:59 INFO Executor: Running task 30.0 in stage 24.0 (TID 340) +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO Executor: Finished task 27.0 in stage 24.0 (TID 337). 8470 bytes result sent to driver +26/04/01 06:09:59 INFO TaskSetManager: Starting task 31.0 in stage 24.0 (TID 341) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9428 bytes) +26/04/01 06:09:59 INFO Executor: Running task 31.0 in stage 24.0 (TID 341) +26/04/01 06:09:59 INFO TaskSetManager: Finished task 27.0 in stage 24.0 (TID 337) in 3487 ms on 10.0.0.133 (executor driver) (28/200) +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:03 INFO Executor: Finished task 28.0 in stage 24.0 (TID 338). 8470 bytes result sent to driver +26/04/01 06:10:03 INFO TaskSetManager: Starting task 32.0 in stage 24.0 (TID 342) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:03 INFO Executor: Running task 32.0 in stage 24.0 (TID 342) +26/04/01 06:10:03 INFO TaskSetManager: Finished task 28.0 in stage 24.0 (TID 338) in 3464 ms on 10.0.0.133 (executor driver) (29/200) +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:03 INFO Executor: Finished task 29.0 in stage 24.0 (TID 339). 8470 bytes result sent to driver +26/04/01 06:10:03 INFO TaskSetManager: Starting task 33.0 in stage 24.0 (TID 343) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:03 INFO TaskSetManager: Finished task 29.0 in stage 24.0 (TID 339) in 3468 ms on 10.0.0.133 (executor driver) (30/200) +26/04/01 06:10:03 INFO Executor: Running task 33.0 in stage 24.0 (TID 343) +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:03 INFO Executor: Finished task 31.0 in stage 24.0 (TID 341). 8470 bytes result sent to driver +26/04/01 06:10:03 INFO TaskSetManager: Starting task 34.0 in stage 24.0 (TID 344) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:03 INFO TaskSetManager: Finished task 31.0 in stage 24.0 (TID 341) in 3457 ms on 10.0.0.133 (executor driver) (31/200) +26/04/01 06:10:03 INFO Executor: Running task 34.0 in stage 24.0 (TID 344) +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO Executor: Finished task 30.0 in stage 24.0 (TID 340). 8470 bytes result sent to driver +26/04/01 06:10:03 INFO TaskSetManager: Starting task 35.0 in stage 24.0 (TID 345) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:03 INFO Executor: Running task 35.0 in stage 24.0 (TID 345) +26/04/01 06:10:03 INFO TaskSetManager: Finished task 30.0 in stage 24.0 (TID 340) in 3472 ms on 10.0.0.133 (executor driver) (32/200) +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:06 INFO Executor: Finished task 32.0 in stage 24.0 (TID 342). 9201 bytes result sent to driver +26/04/01 06:10:06 INFO TaskSetManager: Starting task 36.0 in stage 24.0 (TID 346) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:06 INFO Executor: Running task 36.0 in stage 24.0 (TID 346) +26/04/01 06:10:06 INFO TaskSetManager: Finished task 32.0 in stage 24.0 (TID 342) in 3428 ms on 10.0.0.133 (executor driver) (33/200) +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:06 INFO Executor: Finished task 33.0 in stage 24.0 (TID 343). 9201 bytes result sent to driver +26/04/01 06:10:06 INFO TaskSetManager: Starting task 37.0 in stage 24.0 (TID 347) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:06 INFO TaskSetManager: Finished task 33.0 in stage 24.0 (TID 343) in 3523 ms on 10.0.0.133 (executor driver) (34/200) +26/04/01 06:10:06 INFO Executor: Running task 37.0 in stage 24.0 (TID 347) +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO Executor: Finished task 35.0 in stage 24.0 (TID 345). 9201 bytes result sent to driver +26/04/01 06:10:06 INFO TaskSetManager: Starting task 38.0 in stage 24.0 (TID 348) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:06 INFO Executor: Running task 38.0 in stage 24.0 (TID 348) +26/04/01 06:10:06 INFO TaskSetManager: Finished task 35.0 in stage 24.0 (TID 345) in 3471 ms on 10.0.0.133 (executor driver) (35/200) +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO Executor: Finished task 34.0 in stage 24.0 (TID 344). 9201 bytes result sent to driver +26/04/01 06:10:06 INFO TaskSetManager: Starting task 39.0 in stage 24.0 (TID 349) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:06 INFO Executor: Running task 39.0 in stage 24.0 (TID 349) +26/04/01 06:10:06 INFO TaskSetManager: Finished task 34.0 in stage 24.0 (TID 344) in 3482 ms on 10.0.0.133 (executor driver) (36/200) +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:10 INFO Executor: Finished task 36.0 in stage 24.0 (TID 346). 8470 bytes result sent to driver +26/04/01 06:10:10 INFO TaskSetManager: Starting task 40.0 in stage 24.0 (TID 350) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:10 INFO Executor: Running task 40.0 in stage 24.0 (TID 350) +26/04/01 06:10:10 INFO TaskSetManager: Finished task 36.0 in stage 24.0 (TID 346) in 3370 ms on 10.0.0.133 (executor driver) (37/200) +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:10 INFO Executor: Finished task 38.0 in stage 24.0 (TID 348). 8470 bytes result sent to driver +26/04/01 06:10:10 INFO TaskSetManager: Starting task 41.0 in stage 24.0 (TID 351) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:10 INFO Executor: Finished task 39.0 in stage 24.0 (TID 349). 8470 bytes result sent to driver +26/04/01 06:10:10 INFO TaskSetManager: Finished task 38.0 in stage 24.0 (TID 348) in 3498 ms on 10.0.0.133 (executor driver) (38/200) +26/04/01 06:10:10 INFO Executor: Running task 41.0 in stage 24.0 (TID 351) +26/04/01 06:10:10 INFO TaskSetManager: Starting task 42.0 in stage 24.0 (TID 352) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:10 INFO Executor: Running task 42.0 in stage 24.0 (TID 352) +26/04/01 06:10:10 INFO TaskSetManager: Finished task 39.0 in stage 24.0 (TID 349) in 3497 ms on 10.0.0.133 (executor driver) (39/200) +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO Executor: Finished task 37.0 in stage 24.0 (TID 347). 8470 bytes result sent to driver +26/04/01 06:10:10 INFO TaskSetManager: Starting task 43.0 in stage 24.0 (TID 353) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:10 INFO TaskSetManager: Finished task 37.0 in stage 24.0 (TID 347) in 3504 ms on 10.0.0.133 (executor driver) (40/200) +26/04/01 06:10:10 INFO Executor: Running task 43.0 in stage 24.0 (TID 353) +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:13 INFO Executor: Finished task 40.0 in stage 24.0 (TID 350). 8470 bytes result sent to driver +26/04/01 06:10:13 INFO TaskSetManager: Starting task 44.0 in stage 24.0 (TID 354) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:13 INFO Executor: Running task 44.0 in stage 24.0 (TID 354) +26/04/01 06:10:13 INFO TaskSetManager: Finished task 40.0 in stage 24.0 (TID 350) in 3359 ms on 10.0.0.133 (executor driver) (41/200) +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:13 INFO Executor: Finished task 43.0 in stage 24.0 (TID 353). 8470 bytes result sent to driver +26/04/01 06:10:13 INFO TaskSetManager: Starting task 45.0 in stage 24.0 (TID 355) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:13 INFO TaskSetManager: Finished task 43.0 in stage 24.0 (TID 353) in 3491 ms on 10.0.0.133 (executor driver) (42/200) +26/04/01 06:10:13 INFO Executor: Running task 45.0 in stage 24.0 (TID 355) +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO Executor: Finished task 42.0 in stage 24.0 (TID 352). 8470 bytes result sent to driver +26/04/01 06:10:13 INFO TaskSetManager: Starting task 46.0 in stage 24.0 (TID 356) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:13 INFO Executor: Running task 46.0 in stage 24.0 (TID 356) +26/04/01 06:10:13 INFO TaskSetManager: Finished task 42.0 in stage 24.0 (TID 352) in 3505 ms on 10.0.0.133 (executor driver) (43/200) +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO Executor: Finished task 41.0 in stage 24.0 (TID 351). 8470 bytes result sent to driver +26/04/01 06:10:13 INFO TaskSetManager: Starting task 47.0 in stage 24.0 (TID 357) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:13 INFO TaskSetManager: Finished task 41.0 in stage 24.0 (TID 351) in 3506 ms on 10.0.0.133 (executor driver) (44/200) +26/04/01 06:10:13 INFO Executor: Running task 47.0 in stage 24.0 (TID 357) +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:16 INFO Executor: Finished task 44.0 in stage 24.0 (TID 354). 9201 bytes result sent to driver +26/04/01 06:10:16 INFO TaskSetManager: Starting task 48.0 in stage 24.0 (TID 358) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:16 INFO TaskSetManager: Finished task 44.0 in stage 24.0 (TID 354) in 3362 ms on 10.0.0.133 (executor driver) (45/200) +26/04/01 06:10:16 INFO Executor: Running task 48.0 in stage 24.0 (TID 358) +26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:17 INFO Executor: Finished task 45.0 in stage 24.0 (TID 355). 9201 bytes result sent to driver +26/04/01 06:10:17 INFO TaskSetManager: Starting task 49.0 in stage 24.0 (TID 359) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:17 INFO TaskSetManager: Finished task 45.0 in stage 24.0 (TID 355) in 3408 ms on 10.0.0.133 (executor driver) (46/200) +26/04/01 06:10:17 INFO Executor: Running task 49.0 in stage 24.0 (TID 359) +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:17 INFO Executor: Finished task 46.0 in stage 24.0 (TID 356). 9201 bytes result sent to driver +26/04/01 06:10:17 INFO TaskSetManager: Starting task 50.0 in stage 24.0 (TID 360) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:17 INFO TaskSetManager: Finished task 46.0 in stage 24.0 (TID 356) in 3479 ms on 10.0.0.133 (executor driver) (47/200) +26/04/01 06:10:17 INFO Executor: Running task 50.0 in stage 24.0 (TID 360) +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:17 INFO Executor: Finished task 47.0 in stage 24.0 (TID 357). 9201 bytes result sent to driver +26/04/01 06:10:17 INFO TaskSetManager: Starting task 51.0 in stage 24.0 (TID 361) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:17 INFO Executor: Running task 51.0 in stage 24.0 (TID 361) +26/04/01 06:10:17 INFO TaskSetManager: Finished task 47.0 in stage 24.0 (TID 357) in 3481 ms on 10.0.0.133 (executor driver) (48/200) +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:20 INFO Executor: Finished task 48.0 in stage 24.0 (TID 358). 8470 bytes result sent to driver +26/04/01 06:10:20 INFO TaskSetManager: Starting task 52.0 in stage 24.0 (TID 362) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:20 INFO Executor: Running task 52.0 in stage 24.0 (TID 362) +26/04/01 06:10:20 INFO TaskSetManager: Finished task 48.0 in stage 24.0 (TID 358) in 3396 ms on 10.0.0.133 (executor driver) (49/200) +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:20 INFO Executor: Finished task 49.0 in stage 24.0 (TID 359). 8470 bytes result sent to driver +26/04/01 06:10:20 INFO TaskSetManager: Starting task 53.0 in stage 24.0 (TID 363) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:20 INFO Executor: Running task 53.0 in stage 24.0 (TID 363) +26/04/01 06:10:20 INFO TaskSetManager: Finished task 49.0 in stage 24.0 (TID 359) in 3414 ms on 10.0.0.133 (executor driver) (50/200) +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:20 INFO Executor: Finished task 50.0 in stage 24.0 (TID 360). 8470 bytes result sent to driver +26/04/01 06:10:20 INFO TaskSetManager: Starting task 54.0 in stage 24.0 (TID 364) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:20 INFO Executor: Running task 54.0 in stage 24.0 (TID 364) +26/04/01 06:10:20 INFO TaskSetManager: Finished task 50.0 in stage 24.0 (TID 360) in 3469 ms on 10.0.0.133 (executor driver) (51/200) +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO Executor: Finished task 51.0 in stage 24.0 (TID 361). 8470 bytes result sent to driver +26/04/01 06:10:20 INFO TaskSetManager: Starting task 55.0 in stage 24.0 (TID 365) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:20 INFO Executor: Running task 55.0 in stage 24.0 (TID 365) +26/04/01 06:10:20 INFO TaskSetManager: Finished task 51.0 in stage 24.0 (TID 361) in 3476 ms on 10.0.0.133 (executor driver) (52/200) +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:23 INFO Executor: Finished task 52.0 in stage 24.0 (TID 362). 8470 bytes result sent to driver +26/04/01 06:10:23 INFO TaskSetManager: Starting task 56.0 in stage 24.0 (TID 366) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:23 INFO TaskSetManager: Finished task 52.0 in stage 24.0 (TID 362) in 3346 ms on 10.0.0.133 (executor driver) (53/200) +26/04/01 06:10:23 INFO Executor: Running task 56.0 in stage 24.0 (TID 366) +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:23 INFO Executor: Finished task 53.0 in stage 24.0 (TID 363). 8470 bytes result sent to driver +26/04/01 06:10:23 INFO TaskSetManager: Starting task 57.0 in stage 24.0 (TID 367) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:23 INFO TaskSetManager: Finished task 53.0 in stage 24.0 (TID 363) in 3364 ms on 10.0.0.133 (executor driver) (54/200) +26/04/01 06:10:23 INFO Executor: Running task 57.0 in stage 24.0 (TID 367) +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:24 INFO Executor: Finished task 54.0 in stage 24.0 (TID 364). 8470 bytes result sent to driver +26/04/01 06:10:24 INFO TaskSetManager: Starting task 58.0 in stage 24.0 (TID 368) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:24 INFO TaskSetManager: Finished task 54.0 in stage 24.0 (TID 364) in 3434 ms on 10.0.0.133 (executor driver) (55/200) +26/04/01 06:10:24 INFO Executor: Running task 58.0 in stage 24.0 (TID 368) +26/04/01 06:10:24 INFO Executor: Finished task 55.0 in stage 24.0 (TID 365). 8470 bytes result sent to driver +26/04/01 06:10:24 INFO TaskSetManager: Starting task 59.0 in stage 24.0 (TID 369) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:24 INFO TaskSetManager: Finished task 55.0 in stage 24.0 (TID 365) in 3424 ms on 10.0.0.133 (executor driver) (56/200) +26/04/01 06:10:24 INFO Executor: Running task 59.0 in stage 24.0 (TID 369) +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:26 INFO Executor: Finished task 56.0 in stage 24.0 (TID 366). 9201 bytes result sent to driver +26/04/01 06:10:26 INFO TaskSetManager: Starting task 60.0 in stage 24.0 (TID 370) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:26 INFO TaskSetManager: Finished task 56.0 in stage 24.0 (TID 366) in 3358 ms on 10.0.0.133 (executor driver) (57/200) +26/04/01 06:10:26 INFO Executor: Running task 60.0 in stage 24.0 (TID 370) +26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:27 INFO Executor: Finished task 57.0 in stage 24.0 (TID 367). 9201 bytes result sent to driver +26/04/01 06:10:27 INFO TaskSetManager: Starting task 61.0 in stage 24.0 (TID 371) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:27 INFO Executor: Running task 61.0 in stage 24.0 (TID 371) +26/04/01 06:10:27 INFO TaskSetManager: Finished task 57.0 in stage 24.0 (TID 367) in 3366 ms on 10.0.0.133 (executor driver) (58/200) +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:27 INFO Executor: Finished task 58.0 in stage 24.0 (TID 368). 9201 bytes result sent to driver +26/04/01 06:10:27 INFO TaskSetManager: Starting task 62.0 in stage 24.0 (TID 372) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:27 INFO TaskSetManager: Finished task 58.0 in stage 24.0 (TID 368) in 3433 ms on 10.0.0.133 (executor driver) (59/200) +26/04/01 06:10:27 INFO Executor: Running task 62.0 in stage 24.0 (TID 372) +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:27 INFO Executor: Finished task 59.0 in stage 24.0 (TID 369). 9201 bytes result sent to driver +26/04/01 06:10:27 INFO TaskSetManager: Starting task 63.0 in stage 24.0 (TID 373) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:27 INFO TaskSetManager: Finished task 59.0 in stage 24.0 (TID 369) in 3435 ms on 10.0.0.133 (executor driver) (60/200) +26/04/01 06:10:27 INFO Executor: Running task 63.0 in stage 24.0 (TID 373) +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 6 (999.4 KiB) non-empty blocks including 6 (999.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 6 (999.4 KiB) non-empty blocks including 6 (999.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:30 INFO Executor: Finished task 60.0 in stage 24.0 (TID 370). 8470 bytes result sent to driver +26/04/01 06:10:30 INFO TaskSetManager: Starting task 64.0 in stage 24.0 (TID 374) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:30 INFO Executor: Running task 64.0 in stage 24.0 (TID 374) +26/04/01 06:10:30 INFO TaskSetManager: Finished task 60.0 in stage 24.0 (TID 370) in 3374 ms on 10.0.0.133 (executor driver) (61/200) +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.3 MiB) non-empty blocks including 208 (45.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.3 MiB) non-empty blocks including 208 (45.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:30 INFO Executor: Finished task 61.0 in stage 24.0 (TID 371). 8470 bytes result sent to driver +26/04/01 06:10:30 INFO TaskSetManager: Starting task 65.0 in stage 24.0 (TID 375) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:30 INFO Executor: Running task 65.0 in stage 24.0 (TID 375) +26/04/01 06:10:30 INFO TaskSetManager: Finished task 61.0 in stage 24.0 (TID 371) in 3385 ms on 10.0.0.133 (executor driver) (62/200) +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:31 INFO Executor: Finished task 63.0 in stage 24.0 (TID 373). 8470 bytes result sent to driver +26/04/01 06:10:31 INFO TaskSetManager: Starting task 66.0 in stage 24.0 (TID 376) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:31 INFO TaskSetManager: Finished task 63.0 in stage 24.0 (TID 373) in 3430 ms on 10.0.0.133 (executor driver) (63/200) +26/04/01 06:10:31 INFO Executor: Running task 66.0 in stage 24.0 (TID 376) +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:31 INFO Executor: Finished task 62.0 in stage 24.0 (TID 372). 8470 bytes result sent to driver +26/04/01 06:10:31 INFO TaskSetManager: Starting task 67.0 in stage 24.0 (TID 377) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:31 INFO TaskSetManager: Finished task 62.0 in stage 24.0 (TID 372) in 3443 ms on 10.0.0.133 (executor driver) (64/200) +26/04/01 06:10:31 INFO Executor: Running task 67.0 in stage 24.0 (TID 377) +26/04/01 06:10:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:33 INFO Executor: Finished task 64.0 in stage 24.0 (TID 374). 8470 bytes result sent to driver +26/04/01 06:10:33 INFO TaskSetManager: Starting task 68.0 in stage 24.0 (TID 378) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:33 INFO TaskSetManager: Finished task 64.0 in stage 24.0 (TID 374) in 3352 ms on 10.0.0.133 (executor driver) (65/200) +26/04/01 06:10:33 INFO Executor: Running task 68.0 in stage 24.0 (TID 378) +26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:34 INFO Executor: Finished task 65.0 in stage 24.0 (TID 375). 8470 bytes result sent to driver +26/04/01 06:10:34 INFO TaskSetManager: Starting task 69.0 in stage 24.0 (TID 379) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:34 INFO TaskSetManager: Finished task 65.0 in stage 24.0 (TID 375) in 3340 ms on 10.0.0.133 (executor driver) (66/200) +26/04/01 06:10:34 INFO Executor: Running task 69.0 in stage 24.0 (TID 379) +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:34 INFO Executor: Finished task 67.0 in stage 24.0 (TID 377). 8470 bytes result sent to driver +26/04/01 06:10:34 INFO TaskSetManager: Starting task 70.0 in stage 24.0 (TID 380) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:34 INFO TaskSetManager: Finished task 67.0 in stage 24.0 (TID 377) in 3430 ms on 10.0.0.133 (executor driver) (67/200) +26/04/01 06:10:34 INFO Executor: Running task 70.0 in stage 24.0 (TID 380) +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:34 INFO Executor: Finished task 66.0 in stage 24.0 (TID 376). 8470 bytes result sent to driver +26/04/01 06:10:34 INFO TaskSetManager: Starting task 71.0 in stage 24.0 (TID 381) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:34 INFO Executor: Running task 71.0 in stage 24.0 (TID 381) +26/04/01 06:10:34 INFO TaskSetManager: Finished task 66.0 in stage 24.0 (TID 376) in 3450 ms on 10.0.0.133 (executor driver) (68/200) +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:36 INFO Executor: Finished task 68.0 in stage 24.0 (TID 378). 9201 bytes result sent to driver +26/04/01 06:10:36 INFO TaskSetManager: Starting task 72.0 in stage 24.0 (TID 382) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:36 INFO Executor: Running task 72.0 in stage 24.0 (TID 382) +26/04/01 06:10:36 INFO TaskSetManager: Finished task 68.0 in stage 24.0 (TID 378) in 3350 ms on 10.0.0.133 (executor driver) (69/200) +26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:37 INFO Executor: Finished task 69.0 in stage 24.0 (TID 379). 9201 bytes result sent to driver +26/04/01 06:10:37 INFO TaskSetManager: Starting task 73.0 in stage 24.0 (TID 383) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:37 INFO TaskSetManager: Finished task 69.0 in stage 24.0 (TID 379) in 3337 ms on 10.0.0.133 (executor driver) (70/200) +26/04/01 06:10:37 INFO Executor: Running task 73.0 in stage 24.0 (TID 383) +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:37 INFO Executor: Finished task 71.0 in stage 24.0 (TID 381). 9201 bytes result sent to driver +26/04/01 06:10:37 INFO TaskSetManager: Starting task 74.0 in stage 24.0 (TID 384) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:37 INFO TaskSetManager: Finished task 71.0 in stage 24.0 (TID 381) in 3421 ms on 10.0.0.133 (executor driver) (71/200) +26/04/01 06:10:37 INFO Executor: Running task 74.0 in stage 24.0 (TID 384) +26/04/01 06:10:37 INFO Executor: Finished task 70.0 in stage 24.0 (TID 380). 9201 bytes result sent to driver +26/04/01 06:10:37 INFO TaskSetManager: Starting task 75.0 in stage 24.0 (TID 385) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:37 INFO Executor: Running task 75.0 in stage 24.0 (TID 385) +26/04/01 06:10:37 INFO TaskSetManager: Finished task 70.0 in stage 24.0 (TID 380) in 3430 ms on 10.0.0.133 (executor driver) (72/200) +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:40 INFO Executor: Finished task 72.0 in stage 24.0 (TID 382). 8470 bytes result sent to driver +26/04/01 06:10:40 INFO TaskSetManager: Starting task 76.0 in stage 24.0 (TID 386) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:40 INFO Executor: Running task 76.0 in stage 24.0 (TID 386) +26/04/01 06:10:40 INFO TaskSetManager: Finished task 72.0 in stage 24.0 (TID 382) in 3355 ms on 10.0.0.133 (executor driver) (73/200) +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:40 INFO Executor: Finished task 73.0 in stage 24.0 (TID 383). 8470 bytes result sent to driver +26/04/01 06:10:40 INFO TaskSetManager: Starting task 77.0 in stage 24.0 (TID 387) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:40 INFO Executor: Running task 77.0 in stage 24.0 (TID 387) +26/04/01 06:10:40 INFO TaskSetManager: Finished task 73.0 in stage 24.0 (TID 383) in 3350 ms on 10.0.0.133 (executor driver) (74/200) +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:41 INFO Executor: Finished task 74.0 in stage 24.0 (TID 384). 8470 bytes result sent to driver +26/04/01 06:10:41 INFO TaskSetManager: Starting task 78.0 in stage 24.0 (TID 388) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:41 INFO Executor: Running task 78.0 in stage 24.0 (TID 388) +26/04/01 06:10:41 INFO TaskSetManager: Finished task 74.0 in stage 24.0 (TID 384) in 3429 ms on 10.0.0.133 (executor driver) (75/200) +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:41 INFO Executor: Finished task 75.0 in stage 24.0 (TID 385). 8470 bytes result sent to driver +26/04/01 06:10:41 INFO TaskSetManager: Starting task 79.0 in stage 24.0 (TID 389) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:41 INFO Executor: Running task 79.0 in stage 24.0 (TID 389) +26/04/01 06:10:41 INFO TaskSetManager: Finished task 75.0 in stage 24.0 (TID 385) in 3446 ms on 10.0.0.133 (executor driver) (76/200) +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:43 INFO Executor: Finished task 76.0 in stage 24.0 (TID 386). 8470 bytes result sent to driver +26/04/01 06:10:43 INFO TaskSetManager: Starting task 80.0 in stage 24.0 (TID 390) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:43 INFO Executor: Running task 80.0 in stage 24.0 (TID 390) +26/04/01 06:10:43 INFO TaskSetManager: Finished task 76.0 in stage 24.0 (TID 386) in 3356 ms on 10.0.0.133 (executor driver) (77/200) +26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:44 INFO Executor: Finished task 77.0 in stage 24.0 (TID 387). 8470 bytes result sent to driver +26/04/01 06:10:44 INFO TaskSetManager: Starting task 81.0 in stage 24.0 (TID 391) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:44 INFO Executor: Running task 81.0 in stage 24.0 (TID 391) +26/04/01 06:10:44 INFO TaskSetManager: Finished task 77.0 in stage 24.0 (TID 387) in 3345 ms on 10.0.0.133 (executor driver) (78/200) +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:44 INFO Executor: Finished task 78.0 in stage 24.0 (TID 388). 8470 bytes result sent to driver +26/04/01 06:10:44 INFO TaskSetManager: Starting task 82.0 in stage 24.0 (TID 392) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:44 INFO Executor: Running task 82.0 in stage 24.0 (TID 392) +26/04/01 06:10:44 INFO TaskSetManager: Finished task 78.0 in stage 24.0 (TID 388) in 3430 ms on 10.0.0.133 (executor driver) (79/200) +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:44 INFO Executor: Finished task 79.0 in stage 24.0 (TID 389). 8470 bytes result sent to driver +26/04/01 06:10:44 INFO TaskSetManager: Starting task 83.0 in stage 24.0 (TID 393) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:44 INFO Executor: Running task 83.0 in stage 24.0 (TID 393) +26/04/01 06:10:44 INFO TaskSetManager: Finished task 79.0 in stage 24.0 (TID 389) in 3414 ms on 10.0.0.133 (executor driver) (80/200) +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:46 INFO Executor: Finished task 80.0 in stage 24.0 (TID 390). 9201 bytes result sent to driver +26/04/01 06:10:46 INFO TaskSetManager: Starting task 84.0 in stage 24.0 (TID 394) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:46 INFO Executor: Running task 84.0 in stage 24.0 (TID 394) +26/04/01 06:10:46 INFO TaskSetManager: Finished task 80.0 in stage 24.0 (TID 390) in 3354 ms on 10.0.0.133 (executor driver) (81/200) +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:47 INFO Executor: Finished task 81.0 in stage 24.0 (TID 391). 9201 bytes result sent to driver +26/04/01 06:10:47 INFO TaskSetManager: Starting task 85.0 in stage 24.0 (TID 395) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:47 INFO Executor: Running task 85.0 in stage 24.0 (TID 395) +26/04/01 06:10:47 INFO TaskSetManager: Finished task 81.0 in stage 24.0 (TID 391) in 3360 ms on 10.0.0.133 (executor driver) (82/200) +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:48 INFO Executor: Finished task 83.0 in stage 24.0 (TID 393). 9201 bytes result sent to driver +26/04/01 06:10:48 INFO TaskSetManager: Starting task 86.0 in stage 24.0 (TID 396) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:48 INFO Executor: Running task 86.0 in stage 24.0 (TID 396) +26/04/01 06:10:48 INFO TaskSetManager: Finished task 83.0 in stage 24.0 (TID 393) in 3423 ms on 10.0.0.133 (executor driver) (83/200) +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:48 INFO Executor: Finished task 82.0 in stage 24.0 (TID 392). 9201 bytes result sent to driver +26/04/01 06:10:48 INFO TaskSetManager: Starting task 87.0 in stage 24.0 (TID 397) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:48 INFO TaskSetManager: Finished task 82.0 in stage 24.0 (TID 392) in 3428 ms on 10.0.0.133 (executor driver) (84/200) +26/04/01 06:10:48 INFO Executor: Running task 87.0 in stage 24.0 (TID 397) +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:50 INFO Executor: Finished task 84.0 in stage 24.0 (TID 394). 8470 bytes result sent to driver +26/04/01 06:10:50 INFO TaskSetManager: Starting task 88.0 in stage 24.0 (TID 398) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:50 INFO Executor: Running task 88.0 in stage 24.0 (TID 398) +26/04/01 06:10:50 INFO TaskSetManager: Finished task 84.0 in stage 24.0 (TID 394) in 3363 ms on 10.0.0.133 (executor driver) (85/200) +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:50 INFO Executor: Finished task 85.0 in stage 24.0 (TID 395). 8470 bytes result sent to driver +26/04/01 06:10:50 INFO TaskSetManager: Starting task 89.0 in stage 24.0 (TID 399) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:50 INFO Executor: Running task 89.0 in stage 24.0 (TID 399) +26/04/01 06:10:50 INFO TaskSetManager: Finished task 85.0 in stage 24.0 (TID 395) in 3370 ms on 10.0.0.133 (executor driver) (86/200) +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:51 INFO Executor: Finished task 87.0 in stage 24.0 (TID 397). 8470 bytes result sent to driver +26/04/01 06:10:51 INFO TaskSetManager: Starting task 90.0 in stage 24.0 (TID 400) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:51 INFO Executor: Running task 90.0 in stage 24.0 (TID 400) +26/04/01 06:10:51 INFO TaskSetManager: Finished task 87.0 in stage 24.0 (TID 397) in 3433 ms on 10.0.0.133 (executor driver) (87/200) +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:51 INFO Executor: Finished task 86.0 in stage 24.0 (TID 396). 8470 bytes result sent to driver +26/04/01 06:10:51 INFO TaskSetManager: Starting task 91.0 in stage 24.0 (TID 401) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:51 INFO Executor: Running task 91.0 in stage 24.0 (TID 401) +26/04/01 06:10:51 INFO TaskSetManager: Finished task 86.0 in stage 24.0 (TID 396) in 3441 ms on 10.0.0.133 (executor driver) (88/200) +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:53 INFO Executor: Finished task 88.0 in stage 24.0 (TID 398). 8470 bytes result sent to driver +26/04/01 06:10:53 INFO TaskSetManager: Starting task 92.0 in stage 24.0 (TID 402) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:53 INFO TaskSetManager: Finished task 88.0 in stage 24.0 (TID 398) in 3349 ms on 10.0.0.133 (executor driver) (89/200) +26/04/01 06:10:53 INFO Executor: Running task 92.0 in stage 24.0 (TID 402) +26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:54 INFO Executor: Finished task 89.0 in stage 24.0 (TID 399). 8470 bytes result sent to driver +26/04/01 06:10:54 INFO TaskSetManager: Starting task 93.0 in stage 24.0 (TID 403) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:54 INFO Executor: Running task 93.0 in stage 24.0 (TID 403) +26/04/01 06:10:54 INFO TaskSetManager: Finished task 89.0 in stage 24.0 (TID 399) in 3380 ms on 10.0.0.133 (executor driver) (90/200) +26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:55 INFO Executor: Finished task 91.0 in stage 24.0 (TID 401). 8470 bytes result sent to driver +26/04/01 06:10:55 INFO TaskSetManager: Starting task 94.0 in stage 24.0 (TID 404) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:55 INFO Executor: Running task 94.0 in stage 24.0 (TID 404) +26/04/01 06:10:55 INFO TaskSetManager: Finished task 91.0 in stage 24.0 (TID 401) in 3439 ms on 10.0.0.133 (executor driver) (91/200) +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:55 INFO Executor: Finished task 90.0 in stage 24.0 (TID 400). 8470 bytes result sent to driver +26/04/01 06:10:55 INFO TaskSetManager: Starting task 95.0 in stage 24.0 (TID 405) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:55 INFO TaskSetManager: Finished task 90.0 in stage 24.0 (TID 400) in 3445 ms on 10.0.0.133 (executor driver) (92/200) +26/04/01 06:10:55 INFO Executor: Running task 95.0 in stage 24.0 (TID 405) +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:57 INFO Executor: Finished task 92.0 in stage 24.0 (TID 402). 9201 bytes result sent to driver +26/04/01 06:10:57 INFO TaskSetManager: Starting task 96.0 in stage 24.0 (TID 406) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:57 INFO Executor: Running task 96.0 in stage 24.0 (TID 406) +26/04/01 06:10:57 INFO TaskSetManager: Finished task 92.0 in stage 24.0 (TID 402) in 3346 ms on 10.0.0.133 (executor driver) (93/200) +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:57 INFO Executor: Finished task 93.0 in stage 24.0 (TID 403). 9201 bytes result sent to driver +26/04/01 06:10:57 INFO TaskSetManager: Starting task 97.0 in stage 24.0 (TID 407) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:57 INFO Executor: Running task 97.0 in stage 24.0 (TID 407) +26/04/01 06:10:57 INFO TaskSetManager: Finished task 93.0 in stage 24.0 (TID 403) in 3342 ms on 10.0.0.133 (executor driver) (94/200) +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:58 INFO Executor: Finished task 95.0 in stage 24.0 (TID 405). 9201 bytes result sent to driver +26/04/01 06:10:58 INFO TaskSetManager: Starting task 98.0 in stage 24.0 (TID 408) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:58 INFO TaskSetManager: Finished task 95.0 in stage 24.0 (TID 405) in 3427 ms on 10.0.0.133 (executor driver) (95/200) +26/04/01 06:10:58 INFO Executor: Running task 98.0 in stage 24.0 (TID 408) +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:58 INFO Executor: Finished task 94.0 in stage 24.0 (TID 404). 9201 bytes result sent to driver +26/04/01 06:10:58 INFO TaskSetManager: Starting task 99.0 in stage 24.0 (TID 409) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9428 bytes) +26/04/01 06:10:58 INFO TaskSetManager: Finished task 94.0 in stage 24.0 (TID 404) in 3431 ms on 10.0.0.133 (executor driver) (96/200) +26/04/01 06:10:58 INFO Executor: Running task 99.0 in stage 24.0 (TID 409) +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:10:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:10:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:00 INFO Executor: Finished task 96.0 in stage 24.0 (TID 406). 8470 bytes result sent to driver +26/04/01 06:11:00 INFO TaskSetManager: Starting task 100.0 in stage 24.0 (TID 410) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:00 INFO Executor: Running task 100.0 in stage 24.0 (TID 410) +26/04/01 06:11:00 INFO TaskSetManager: Finished task 96.0 in stage 24.0 (TID 406) in 3368 ms on 10.0.0.133 (executor driver) (97/200) +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:00 INFO Executor: Finished task 97.0 in stage 24.0 (TID 407). 8470 bytes result sent to driver +26/04/01 06:11:00 INFO TaskSetManager: Starting task 101.0 in stage 24.0 (TID 411) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:00 INFO Executor: Running task 101.0 in stage 24.0 (TID 411) +26/04/01 06:11:00 INFO TaskSetManager: Finished task 97.0 in stage 24.0 (TID 407) in 3360 ms on 10.0.0.133 (executor driver) (98/200) +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:01 INFO Executor: Finished task 98.0 in stage 24.0 (TID 408). 8470 bytes result sent to driver +26/04/01 06:11:01 INFO TaskSetManager: Starting task 102.0 in stage 24.0 (TID 412) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:01 INFO Executor: Running task 102.0 in stage 24.0 (TID 412) +26/04/01 06:11:01 INFO TaskSetManager: Finished task 98.0 in stage 24.0 (TID 408) in 3429 ms on 10.0.0.133 (executor driver) (99/200) +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:01 INFO Executor: Finished task 99.0 in stage 24.0 (TID 409). 8470 bytes result sent to driver +26/04/01 06:11:01 INFO TaskSetManager: Starting task 103.0 in stage 24.0 (TID 413) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:01 INFO TaskSetManager: Finished task 99.0 in stage 24.0 (TID 409) in 3430 ms on 10.0.0.133 (executor driver) (100/200) +26/04/01 06:11:01 INFO Executor: Running task 103.0 in stage 24.0 (TID 413) +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:03 INFO Executor: Finished task 100.0 in stage 24.0 (TID 410). 8470 bytes result sent to driver +26/04/01 06:11:03 INFO TaskSetManager: Starting task 104.0 in stage 24.0 (TID 414) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:03 INFO Executor: Running task 104.0 in stage 24.0 (TID 414) +26/04/01 06:11:03 INFO TaskSetManager: Finished task 100.0 in stage 24.0 (TID 410) in 3361 ms on 10.0.0.133 (executor driver) (101/200) +26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:04 INFO Executor: Finished task 101.0 in stage 24.0 (TID 411). 8470 bytes result sent to driver +26/04/01 06:11:04 INFO TaskSetManager: Starting task 105.0 in stage 24.0 (TID 415) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:04 INFO Executor: Running task 105.0 in stage 24.0 (TID 415) +26/04/01 06:11:04 INFO TaskSetManager: Finished task 101.0 in stage 24.0 (TID 411) in 3347 ms on 10.0.0.133 (executor driver) (102/200) +26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:05 INFO Executor: Finished task 102.0 in stage 24.0 (TID 412). 8470 bytes result sent to driver +26/04/01 06:11:05 INFO TaskSetManager: Starting task 106.0 in stage 24.0 (TID 416) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:05 INFO Executor: Running task 106.0 in stage 24.0 (TID 416) +26/04/01 06:11:05 INFO TaskSetManager: Finished task 102.0 in stage 24.0 (TID 412) in 3430 ms on 10.0.0.133 (executor driver) (103/200) +26/04/01 06:11:05 INFO Executor: Finished task 103.0 in stage 24.0 (TID 413). 8470 bytes result sent to driver +26/04/01 06:11:05 INFO TaskSetManager: Starting task 107.0 in stage 24.0 (TID 417) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:05 INFO TaskSetManager: Finished task 103.0 in stage 24.0 (TID 413) in 3427 ms on 10.0.0.133 (executor driver) (104/200) +26/04/01 06:11:05 INFO Executor: Running task 107.0 in stage 24.0 (TID 417) +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:07 INFO Executor: Finished task 104.0 in stage 24.0 (TID 414). 9201 bytes result sent to driver +26/04/01 06:11:07 INFO TaskSetManager: Starting task 108.0 in stage 24.0 (TID 418) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:07 INFO Executor: Running task 108.0 in stage 24.0 (TID 418) +26/04/01 06:11:07 INFO TaskSetManager: Finished task 104.0 in stage 24.0 (TID 414) in 3342 ms on 10.0.0.133 (executor driver) (105/200) +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:07 INFO Executor: Finished task 105.0 in stage 24.0 (TID 415). 9201 bytes result sent to driver +26/04/01 06:11:07 INFO TaskSetManager: Starting task 109.0 in stage 24.0 (TID 419) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:07 INFO Executor: Running task 109.0 in stage 24.0 (TID 419) +26/04/01 06:11:07 INFO TaskSetManager: Finished task 105.0 in stage 24.0 (TID 415) in 3347 ms on 10.0.0.133 (executor driver) (106/200) +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:08 INFO Executor: Finished task 106.0 in stage 24.0 (TID 416). 9201 bytes result sent to driver +26/04/01 06:11:08 INFO TaskSetManager: Starting task 110.0 in stage 24.0 (TID 420) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:08 INFO Executor: Finished task 107.0 in stage 24.0 (TID 417). 9201 bytes result sent to driver +26/04/01 06:11:08 INFO Executor: Running task 110.0 in stage 24.0 (TID 420) +26/04/01 06:11:08 INFO TaskSetManager: Starting task 111.0 in stage 24.0 (TID 421) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:08 INFO Executor: Running task 111.0 in stage 24.0 (TID 421) +26/04/01 06:11:08 INFO TaskSetManager: Finished task 106.0 in stage 24.0 (TID 416) in 3429 ms on 10.0.0.133 (executor driver) (107/200) +26/04/01 06:11:08 INFO TaskSetManager: Finished task 107.0 in stage 24.0 (TID 417) in 3429 ms on 10.0.0.133 (executor driver) (108/200) +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:10 INFO Executor: Finished task 108.0 in stage 24.0 (TID 418). 8470 bytes result sent to driver +26/04/01 06:11:10 INFO TaskSetManager: Starting task 112.0 in stage 24.0 (TID 422) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:10 INFO Executor: Running task 112.0 in stage 24.0 (TID 422) +26/04/01 06:11:10 INFO TaskSetManager: Finished task 108.0 in stage 24.0 (TID 418) in 3353 ms on 10.0.0.133 (executor driver) (109/200) +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:10 INFO Executor: Finished task 109.0 in stage 24.0 (TID 419). 8470 bytes result sent to driver +26/04/01 06:11:10 INFO TaskSetManager: Starting task 113.0 in stage 24.0 (TID 423) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:10 INFO Executor: Running task 113.0 in stage 24.0 (TID 423) +26/04/01 06:11:10 INFO TaskSetManager: Finished task 109.0 in stage 24.0 (TID 419) in 3355 ms on 10.0.0.133 (executor driver) (110/200) +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:12 INFO Executor: Finished task 110.0 in stage 24.0 (TID 420). 8470 bytes result sent to driver +26/04/01 06:11:12 INFO TaskSetManager: Starting task 114.0 in stage 24.0 (TID 424) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:12 INFO Executor: Running task 114.0 in stage 24.0 (TID 424) +26/04/01 06:11:12 INFO TaskSetManager: Finished task 110.0 in stage 24.0 (TID 420) in 3435 ms on 10.0.0.133 (executor driver) (111/200) +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:12 INFO Executor: Finished task 111.0 in stage 24.0 (TID 421). 8470 bytes result sent to driver +26/04/01 06:11:12 INFO TaskSetManager: Starting task 115.0 in stage 24.0 (TID 425) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:12 INFO Executor: Running task 115.0 in stage 24.0 (TID 425) +26/04/01 06:11:12 INFO TaskSetManager: Finished task 111.0 in stage 24.0 (TID 421) in 3443 ms on 10.0.0.133 (executor driver) (112/200) +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:13 INFO Executor: Finished task 112.0 in stage 24.0 (TID 422). 8470 bytes result sent to driver +26/04/01 06:11:13 INFO TaskSetManager: Starting task 116.0 in stage 24.0 (TID 426) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:13 INFO Executor: Running task 116.0 in stage 24.0 (TID 426) +26/04/01 06:11:13 INFO TaskSetManager: Finished task 112.0 in stage 24.0 (TID 422) in 3357 ms on 10.0.0.133 (executor driver) (113/200) +26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:14 INFO Executor: Finished task 113.0 in stage 24.0 (TID 423). 8470 bytes result sent to driver +26/04/01 06:11:14 INFO TaskSetManager: Starting task 117.0 in stage 24.0 (TID 427) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:14 INFO Executor: Running task 117.0 in stage 24.0 (TID 427) +26/04/01 06:11:14 INFO TaskSetManager: Finished task 113.0 in stage 24.0 (TID 423) in 3354 ms on 10.0.0.133 (executor driver) (114/200) +26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:15 INFO Executor: Finished task 114.0 in stage 24.0 (TID 424). 8470 bytes result sent to driver +26/04/01 06:11:15 INFO TaskSetManager: Starting task 118.0 in stage 24.0 (TID 428) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:15 INFO Executor: Running task 118.0 in stage 24.0 (TID 428) +26/04/01 06:11:15 INFO TaskSetManager: Finished task 114.0 in stage 24.0 (TID 424) in 3424 ms on 10.0.0.133 (executor driver) (115/200) +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 6 (1017.0 KiB) non-empty blocks including 6 (1017.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:15 INFO Executor: Finished task 115.0 in stage 24.0 (TID 425). 8470 bytes result sent to driver +26/04/01 06:11:15 INFO TaskSetManager: Starting task 119.0 in stage 24.0 (TID 429) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:15 INFO Executor: Running task 119.0 in stage 24.0 (TID 429) +26/04/01 06:11:15 INFO TaskSetManager: Finished task 115.0 in stage 24.0 (TID 425) in 3420 ms on 10.0.0.133 (executor driver) (116/200) +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 6 (1017.0 KiB) non-empty blocks including 6 (1017.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:17 INFO Executor: Finished task 116.0 in stage 24.0 (TID 426). 9201 bytes result sent to driver +26/04/01 06:11:17 INFO TaskSetManager: Starting task 120.0 in stage 24.0 (TID 430) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:17 INFO Executor: Running task 120.0 in stage 24.0 (TID 430) +26/04/01 06:11:17 INFO TaskSetManager: Finished task 116.0 in stage 24.0 (TID 426) in 3352 ms on 10.0.0.133 (executor driver) (117/200) +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:17 INFO Executor: Finished task 117.0 in stage 24.0 (TID 427). 9201 bytes result sent to driver +26/04/01 06:11:17 INFO TaskSetManager: Starting task 121.0 in stage 24.0 (TID 431) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:17 INFO Executor: Running task 121.0 in stage 24.0 (TID 431) +26/04/01 06:11:17 INFO TaskSetManager: Finished task 117.0 in stage 24.0 (TID 427) in 3373 ms on 10.0.0.133 (executor driver) (118/200) +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:19 INFO Executor: Finished task 118.0 in stage 24.0 (TID 428). 9201 bytes result sent to driver +26/04/01 06:11:19 INFO TaskSetManager: Starting task 122.0 in stage 24.0 (TID 432) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:19 INFO Executor: Running task 122.0 in stage 24.0 (TID 432) +26/04/01 06:11:19 INFO Executor: Finished task 119.0 in stage 24.0 (TID 429). 9201 bytes result sent to driver +26/04/01 06:11:19 INFO TaskSetManager: Starting task 123.0 in stage 24.0 (TID 433) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:19 INFO Executor: Running task 123.0 in stage 24.0 (TID 433) +26/04/01 06:11:19 INFO TaskSetManager: Finished task 118.0 in stage 24.0 (TID 428) in 3427 ms on 10.0.0.133 (executor driver) (119/200) +26/04/01 06:11:19 INFO TaskSetManager: Finished task 119.0 in stage 24.0 (TID 429) in 3422 ms on 10.0.0.133 (executor driver) (120/200) +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:20 INFO Executor: Finished task 120.0 in stage 24.0 (TID 430). 8470 bytes result sent to driver +26/04/01 06:11:20 INFO TaskSetManager: Starting task 124.0 in stage 24.0 (TID 434) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:20 INFO Executor: Running task 124.0 in stage 24.0 (TID 434) +26/04/01 06:11:20 INFO TaskSetManager: Finished task 120.0 in stage 24.0 (TID 430) in 3359 ms on 10.0.0.133 (executor driver) (121/200) +26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:21 INFO Executor: Finished task 121.0 in stage 24.0 (TID 431). 8470 bytes result sent to driver +26/04/01 06:11:21 INFO TaskSetManager: Starting task 125.0 in stage 24.0 (TID 435) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:21 INFO TaskSetManager: Finished task 121.0 in stage 24.0 (TID 431) in 3351 ms on 10.0.0.133 (executor driver) (122/200) +26/04/01 06:11:21 INFO Executor: Running task 125.0 in stage 24.0 (TID 435) +26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:22 INFO Executor: Finished task 122.0 in stage 24.0 (TID 432). 8470 bytes result sent to driver +26/04/01 06:11:22 INFO TaskSetManager: Starting task 126.0 in stage 24.0 (TID 436) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:22 INFO Executor: Running task 126.0 in stage 24.0 (TID 436) +26/04/01 06:11:22 INFO TaskSetManager: Finished task 122.0 in stage 24.0 (TID 432) in 3424 ms on 10.0.0.133 (executor driver) (123/200) +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:22 INFO Executor: Finished task 123.0 in stage 24.0 (TID 433). 8470 bytes result sent to driver +26/04/01 06:11:22 INFO TaskSetManager: Starting task 127.0 in stage 24.0 (TID 437) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:22 INFO Executor: Running task 127.0 in stage 24.0 (TID 437) +26/04/01 06:11:22 INFO TaskSetManager: Finished task 123.0 in stage 24.0 (TID 433) in 3426 ms on 10.0.0.133 (executor driver) (124/200) +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:23 INFO Executor: Finished task 124.0 in stage 24.0 (TID 434). 8470 bytes result sent to driver +26/04/01 06:11:23 INFO TaskSetManager: Starting task 128.0 in stage 24.0 (TID 438) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:23 INFO Executor: Running task 128.0 in stage 24.0 (TID 438) +26/04/01 06:11:23 INFO TaskSetManager: Finished task 124.0 in stage 24.0 (TID 434) in 3341 ms on 10.0.0.133 (executor driver) (125/200) +26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:24 INFO Executor: Finished task 125.0 in stage 24.0 (TID 435). 8470 bytes result sent to driver +26/04/01 06:11:24 INFO TaskSetManager: Starting task 129.0 in stage 24.0 (TID 439) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:24 INFO Executor: Running task 129.0 in stage 24.0 (TID 439) +26/04/01 06:11:24 INFO TaskSetManager: Finished task 125.0 in stage 24.0 (TID 435) in 3350 ms on 10.0.0.133 (executor driver) (126/200) +26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:25 INFO Executor: Finished task 126.0 in stage 24.0 (TID 436). 9201 bytes result sent to driver +26/04/01 06:11:25 INFO TaskSetManager: Starting task 130.0 in stage 24.0 (TID 440) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:25 INFO Executor: Running task 130.0 in stage 24.0 (TID 440) +26/04/01 06:11:25 INFO TaskSetManager: Finished task 126.0 in stage 24.0 (TID 436) in 3437 ms on 10.0.0.133 (executor driver) (127/200) +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:25 INFO Executor: Finished task 127.0 in stage 24.0 (TID 437). 9201 bytes result sent to driver +26/04/01 06:11:25 INFO TaskSetManager: Starting task 131.0 in stage 24.0 (TID 441) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:25 INFO Executor: Running task 131.0 in stage 24.0 (TID 441) +26/04/01 06:11:25 INFO TaskSetManager: Finished task 127.0 in stage 24.0 (TID 437) in 3435 ms on 10.0.0.133 (executor driver) (128/200) +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:27 INFO Executor: Finished task 128.0 in stage 24.0 (TID 438). 9201 bytes result sent to driver +26/04/01 06:11:27 INFO TaskSetManager: Starting task 132.0 in stage 24.0 (TID 442) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:27 INFO Executor: Running task 132.0 in stage 24.0 (TID 442) +26/04/01 06:11:27 INFO TaskSetManager: Finished task 128.0 in stage 24.0 (TID 438) in 3379 ms on 10.0.0.133 (executor driver) (129/200) +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:27 INFO Executor: Finished task 129.0 in stage 24.0 (TID 439). 9201 bytes result sent to driver +26/04/01 06:11:27 INFO TaskSetManager: Starting task 133.0 in stage 24.0 (TID 443) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:27 INFO Executor: Running task 133.0 in stage 24.0 (TID 443) +26/04/01 06:11:27 INFO TaskSetManager: Finished task 129.0 in stage 24.0 (TID 439) in 3364 ms on 10.0.0.133 (executor driver) (130/200) +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:29 INFO Executor: Finished task 131.0 in stage 24.0 (TID 441). 8470 bytes result sent to driver +26/04/01 06:11:29 INFO TaskSetManager: Starting task 134.0 in stage 24.0 (TID 444) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:29 INFO Executor: Running task 134.0 in stage 24.0 (TID 444) +26/04/01 06:11:29 INFO TaskSetManager: Finished task 131.0 in stage 24.0 (TID 441) in 3426 ms on 10.0.0.133 (executor driver) (131/200) +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:29 INFO Executor: Finished task 130.0 in stage 24.0 (TID 440). 8470 bytes result sent to driver +26/04/01 06:11:29 INFO TaskSetManager: Starting task 135.0 in stage 24.0 (TID 445) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:29 INFO Executor: Running task 135.0 in stage 24.0 (TID 445) +26/04/01 06:11:29 INFO TaskSetManager: Finished task 130.0 in stage 24.0 (TID 440) in 3441 ms on 10.0.0.133 (executor driver) (132/200) +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:30 INFO Executor: Finished task 132.0 in stage 24.0 (TID 442). 8470 bytes result sent to driver +26/04/01 06:11:30 INFO TaskSetManager: Starting task 136.0 in stage 24.0 (TID 446) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:30 INFO Executor: Running task 136.0 in stage 24.0 (TID 446) +26/04/01 06:11:30 INFO TaskSetManager: Finished task 132.0 in stage 24.0 (TID 442) in 3370 ms on 10.0.0.133 (executor driver) (133/200) +26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:31 INFO Executor: Finished task 133.0 in stage 24.0 (TID 443). 8470 bytes result sent to driver +26/04/01 06:11:31 INFO TaskSetManager: Starting task 137.0 in stage 24.0 (TID 447) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:31 INFO Executor: Running task 137.0 in stage 24.0 (TID 447) +26/04/01 06:11:31 INFO TaskSetManager: Finished task 133.0 in stage 24.0 (TID 443) in 3355 ms on 10.0.0.133 (executor driver) (134/200) +26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:32 INFO Executor: Finished task 135.0 in stage 24.0 (TID 445). 8470 bytes result sent to driver +26/04/01 06:11:32 INFO TaskSetManager: Starting task 138.0 in stage 24.0 (TID 448) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:32 INFO TaskSetManager: Finished task 135.0 in stage 24.0 (TID 445) in 3386 ms on 10.0.0.133 (executor driver) (135/200) +26/04/01 06:11:32 INFO Executor: Running task 138.0 in stage 24.0 (TID 448) +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:32 INFO Executor: Finished task 134.0 in stage 24.0 (TID 444). 8470 bytes result sent to driver +26/04/01 06:11:32 INFO TaskSetManager: Starting task 139.0 in stage 24.0 (TID 449) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:32 INFO TaskSetManager: Finished task 134.0 in stage 24.0 (TID 444) in 3407 ms on 10.0.0.133 (executor driver) (136/200) +26/04/01 06:11:32 INFO Executor: Running task 139.0 in stage 24.0 (TID 449) +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:34 INFO Executor: Finished task 136.0 in stage 24.0 (TID 446). 8470 bytes result sent to driver +26/04/01 06:11:34 INFO TaskSetManager: Starting task 140.0 in stage 24.0 (TID 450) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:34 INFO Executor: Running task 140.0 in stage 24.0 (TID 450) +26/04/01 06:11:34 INFO TaskSetManager: Finished task 136.0 in stage 24.0 (TID 446) in 3367 ms on 10.0.0.133 (executor driver) (137/200) +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:34 INFO Executor: Finished task 137.0 in stage 24.0 (TID 447). 8470 bytes result sent to driver +26/04/01 06:11:34 INFO TaskSetManager: Starting task 141.0 in stage 24.0 (TID 451) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:34 INFO Executor: Running task 141.0 in stage 24.0 (TID 451) +26/04/01 06:11:34 INFO TaskSetManager: Finished task 137.0 in stage 24.0 (TID 447) in 3365 ms on 10.0.0.133 (executor driver) (138/200) +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:36 INFO Executor: Finished task 138.0 in stage 24.0 (TID 448). 9201 bytes result sent to driver +26/04/01 06:11:36 INFO TaskSetManager: Starting task 142.0 in stage 24.0 (TID 452) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:36 INFO Executor: Running task 142.0 in stage 24.0 (TID 452) +26/04/01 06:11:36 INFO TaskSetManager: Finished task 138.0 in stage 24.0 (TID 448) in 3415 ms on 10.0.0.133 (executor driver) (139/200) +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:36 INFO Executor: Finished task 139.0 in stage 24.0 (TID 449). 9201 bytes result sent to driver +26/04/01 06:11:36 INFO TaskSetManager: Starting task 143.0 in stage 24.0 (TID 453) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:36 INFO Executor: Running task 143.0 in stage 24.0 (TID 453) +26/04/01 06:11:36 INFO TaskSetManager: Finished task 139.0 in stage 24.0 (TID 449) in 3410 ms on 10.0.0.133 (executor driver) (140/200) +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:37 INFO Executor: Finished task 140.0 in stage 24.0 (TID 450). 9201 bytes result sent to driver +26/04/01 06:11:37 INFO TaskSetManager: Starting task 144.0 in stage 24.0 (TID 454) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:37 INFO Executor: Running task 144.0 in stage 24.0 (TID 454) +26/04/01 06:11:37 INFO TaskSetManager: Finished task 140.0 in stage 24.0 (TID 450) in 3385 ms on 10.0.0.133 (executor driver) (141/200) +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:37 INFO Executor: Finished task 141.0 in stage 24.0 (TID 451). 9201 bytes result sent to driver +26/04/01 06:11:37 INFO TaskSetManager: Starting task 145.0 in stage 24.0 (TID 455) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:37 INFO Executor: Running task 145.0 in stage 24.0 (TID 455) +26/04/01 06:11:37 INFO TaskSetManager: Finished task 141.0 in stage 24.0 (TID 451) in 3403 ms on 10.0.0.133 (executor driver) (142/200) +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:39 INFO Executor: Finished task 143.0 in stage 24.0 (TID 453). 8470 bytes result sent to driver +26/04/01 06:11:39 INFO TaskSetManager: Starting task 146.0 in stage 24.0 (TID 456) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:39 INFO TaskSetManager: Finished task 143.0 in stage 24.0 (TID 453) in 3436 ms on 10.0.0.133 (executor driver) (143/200) +26/04/01 06:11:39 INFO Executor: Running task 146.0 in stage 24.0 (TID 456) +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:39 INFO Executor: Finished task 142.0 in stage 24.0 (TID 452). 8470 bytes result sent to driver +26/04/01 06:11:39 INFO TaskSetManager: Starting task 147.0 in stage 24.0 (TID 457) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:39 INFO Executor: Running task 147.0 in stage 24.0 (TID 457) +26/04/01 06:11:39 INFO TaskSetManager: Finished task 142.0 in stage 24.0 (TID 452) in 3441 ms on 10.0.0.133 (executor driver) (144/200) +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:40 INFO Executor: Finished task 144.0 in stage 24.0 (TID 454). 8470 bytes result sent to driver +26/04/01 06:11:40 INFO TaskSetManager: Starting task 148.0 in stage 24.0 (TID 458) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:40 INFO Executor: Running task 148.0 in stage 24.0 (TID 458) +26/04/01 06:11:40 INFO TaskSetManager: Finished task 144.0 in stage 24.0 (TID 454) in 3388 ms on 10.0.0.133 (executor driver) (145/200) +26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:41 INFO Executor: Finished task 145.0 in stage 24.0 (TID 455). 8470 bytes result sent to driver +26/04/01 06:11:41 INFO TaskSetManager: Starting task 149.0 in stage 24.0 (TID 459) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:41 INFO Executor: Running task 149.0 in stage 24.0 (TID 459) +26/04/01 06:11:41 INFO TaskSetManager: Finished task 145.0 in stage 24.0 (TID 455) in 3366 ms on 10.0.0.133 (executor driver) (146/200) +26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:43 INFO Executor: Finished task 147.0 in stage 24.0 (TID 457). 8470 bytes result sent to driver +26/04/01 06:11:43 INFO TaskSetManager: Starting task 150.0 in stage 24.0 (TID 460) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:43 INFO Executor: Running task 150.0 in stage 24.0 (TID 460) +26/04/01 06:11:43 INFO TaskSetManager: Finished task 147.0 in stage 24.0 (TID 457) in 3409 ms on 10.0.0.133 (executor driver) (147/200) +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:43 INFO Executor: Finished task 146.0 in stage 24.0 (TID 456). 8470 bytes result sent to driver +26/04/01 06:11:43 INFO TaskSetManager: Starting task 151.0 in stage 24.0 (TID 461) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:43 INFO TaskSetManager: Finished task 146.0 in stage 24.0 (TID 456) in 3421 ms on 10.0.0.133 (executor driver) (148/200) +26/04/01 06:11:43 INFO Executor: Running task 151.0 in stage 24.0 (TID 461) +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:44 INFO Executor: Finished task 148.0 in stage 24.0 (TID 458). 8470 bytes result sent to driver +26/04/01 06:11:44 INFO TaskSetManager: Starting task 152.0 in stage 24.0 (TID 462) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:44 INFO Executor: Running task 152.0 in stage 24.0 (TID 462) +26/04/01 06:11:44 INFO TaskSetManager: Finished task 148.0 in stage 24.0 (TID 458) in 3359 ms on 10.0.0.133 (executor driver) (149/200) +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:44 INFO Executor: Finished task 149.0 in stage 24.0 (TID 459). 8470 bytes result sent to driver +26/04/01 06:11:44 INFO TaskSetManager: Starting task 153.0 in stage 24.0 (TID 463) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:44 INFO Executor: Running task 153.0 in stage 24.0 (TID 463) +26/04/01 06:11:44 INFO TaskSetManager: Finished task 149.0 in stage 24.0 (TID 459) in 3354 ms on 10.0.0.133 (executor driver) (150/200) +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:46 INFO Executor: Finished task 151.0 in stage 24.0 (TID 461). 9201 bytes result sent to driver +26/04/01 06:11:46 INFO TaskSetManager: Starting task 154.0 in stage 24.0 (TID 464) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:46 INFO Executor: Running task 154.0 in stage 24.0 (TID 464) +26/04/01 06:11:46 INFO TaskSetManager: Finished task 151.0 in stage 24.0 (TID 461) in 3402 ms on 10.0.0.133 (executor driver) (151/200) +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:46 INFO Executor: Finished task 150.0 in stage 24.0 (TID 460). 9201 bytes result sent to driver +26/04/01 06:11:46 INFO TaskSetManager: Starting task 155.0 in stage 24.0 (TID 465) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:46 INFO Executor: Running task 155.0 in stage 24.0 (TID 465) +26/04/01 06:11:46 INFO TaskSetManager: Finished task 150.0 in stage 24.0 (TID 460) in 3422 ms on 10.0.0.133 (executor driver) (152/200) +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:47 INFO Executor: Finished task 152.0 in stage 24.0 (TID 462). 9201 bytes result sent to driver +26/04/01 06:11:47 INFO TaskSetManager: Starting task 156.0 in stage 24.0 (TID 466) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:47 INFO Executor: Running task 156.0 in stage 24.0 (TID 466) +26/04/01 06:11:47 INFO TaskSetManager: Finished task 152.0 in stage 24.0 (TID 462) in 3379 ms on 10.0.0.133 (executor driver) (153/200) +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:47 INFO Executor: Finished task 153.0 in stage 24.0 (TID 463). 9201 bytes result sent to driver +26/04/01 06:11:47 INFO TaskSetManager: Starting task 157.0 in stage 24.0 (TID 467) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:47 INFO Executor: Running task 157.0 in stage 24.0 (TID 467) +26/04/01 06:11:47 INFO TaskSetManager: Finished task 153.0 in stage 24.0 (TID 463) in 3358 ms on 10.0.0.133 (executor driver) (154/200) +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:49 INFO Executor: Finished task 155.0 in stage 24.0 (TID 465). 8470 bytes result sent to driver +26/04/01 06:11:49 INFO TaskSetManager: Starting task 158.0 in stage 24.0 (TID 468) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:49 INFO Executor: Running task 158.0 in stage 24.0 (TID 468) +26/04/01 06:11:49 INFO TaskSetManager: Finished task 155.0 in stage 24.0 (TID 465) in 3456 ms on 10.0.0.133 (executor driver) (155/200) +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:49 INFO Executor: Finished task 154.0 in stage 24.0 (TID 464). 8470 bytes result sent to driver +26/04/01 06:11:49 INFO TaskSetManager: Starting task 159.0 in stage 24.0 (TID 469) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:49 INFO Executor: Running task 159.0 in stage 24.0 (TID 469) +26/04/01 06:11:49 INFO TaskSetManager: Finished task 154.0 in stage 24.0 (TID 464) in 3475 ms on 10.0.0.133 (executor driver) (156/200) +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:50 INFO Executor: Finished task 156.0 in stage 24.0 (TID 466). 8470 bytes result sent to driver +26/04/01 06:11:50 INFO TaskSetManager: Starting task 160.0 in stage 24.0 (TID 470) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:50 INFO Executor: Running task 160.0 in stage 24.0 (TID 470) +26/04/01 06:11:50 INFO TaskSetManager: Finished task 156.0 in stage 24.0 (TID 466) in 3361 ms on 10.0.0.133 (executor driver) (157/200) +26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:51 INFO Executor: Finished task 157.0 in stage 24.0 (TID 467). 8470 bytes result sent to driver +26/04/01 06:11:51 INFO TaskSetManager: Starting task 161.0 in stage 24.0 (TID 471) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:51 INFO Executor: Running task 161.0 in stage 24.0 (TID 471) +26/04/01 06:11:51 INFO TaskSetManager: Finished task 157.0 in stage 24.0 (TID 467) in 3354 ms on 10.0.0.133 (executor driver) (158/200) +26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:53 INFO Executor: Finished task 159.0 in stage 24.0 (TID 469). 8470 bytes result sent to driver +26/04/01 06:11:53 INFO TaskSetManager: Starting task 162.0 in stage 24.0 (TID 472) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:53 INFO Executor: Running task 162.0 in stage 24.0 (TID 472) +26/04/01 06:11:53 INFO TaskSetManager: Finished task 159.0 in stage 24.0 (TID 469) in 3401 ms on 10.0.0.133 (executor driver) (159/200) +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:53 INFO Executor: Finished task 158.0 in stage 24.0 (TID 468). 8470 bytes result sent to driver +26/04/01 06:11:53 INFO TaskSetManager: Starting task 163.0 in stage 24.0 (TID 473) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:53 INFO Executor: Running task 163.0 in stage 24.0 (TID 473) +26/04/01 06:11:53 INFO TaskSetManager: Finished task 158.0 in stage 24.0 (TID 468) in 3422 ms on 10.0.0.133 (executor driver) (160/200) +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:54 INFO Executor: Finished task 160.0 in stage 24.0 (TID 470). 8470 bytes result sent to driver +26/04/01 06:11:54 INFO TaskSetManager: Starting task 164.0 in stage 24.0 (TID 474) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:54 INFO Executor: Running task 164.0 in stage 24.0 (TID 474) +26/04/01 06:11:54 INFO TaskSetManager: Finished task 160.0 in stage 24.0 (TID 470) in 3348 ms on 10.0.0.133 (executor driver) (161/200) +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:54 INFO Executor: Finished task 161.0 in stage 24.0 (TID 471). 8470 bytes result sent to driver +26/04/01 06:11:54 INFO TaskSetManager: Starting task 165.0 in stage 24.0 (TID 475) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:54 INFO Executor: Running task 165.0 in stage 24.0 (TID 475) +26/04/01 06:11:54 INFO TaskSetManager: Finished task 161.0 in stage 24.0 (TID 471) in 3352 ms on 10.0.0.133 (executor driver) (162/200) +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:56 INFO Executor: Finished task 163.0 in stage 24.0 (TID 473). 9201 bytes result sent to driver +26/04/01 06:11:56 INFO TaskSetManager: Starting task 166.0 in stage 24.0 (TID 476) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:56 INFO Executor: Running task 166.0 in stage 24.0 (TID 476) +26/04/01 06:11:56 INFO TaskSetManager: Finished task 163.0 in stage 24.0 (TID 473) in 3404 ms on 10.0.0.133 (executor driver) (163/200) +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:56 INFO Executor: Finished task 162.0 in stage 24.0 (TID 472). 9201 bytes result sent to driver +26/04/01 06:11:56 INFO TaskSetManager: Starting task 167.0 in stage 24.0 (TID 477) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:56 INFO Executor: Running task 167.0 in stage 24.0 (TID 477) +26/04/01 06:11:56 INFO TaskSetManager: Finished task 162.0 in stage 24.0 (TID 472) in 3420 ms on 10.0.0.133 (executor driver) (164/200) +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:57 INFO Executor: Finished task 164.0 in stage 24.0 (TID 474). 9201 bytes result sent to driver +26/04/01 06:11:57 INFO TaskSetManager: Starting task 168.0 in stage 24.0 (TID 478) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:57 INFO Executor: Running task 168.0 in stage 24.0 (TID 478) +26/04/01 06:11:57 INFO TaskSetManager: Finished task 164.0 in stage 24.0 (TID 474) in 3345 ms on 10.0.0.133 (executor driver) (165/200) +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:57 INFO Executor: Finished task 165.0 in stage 24.0 (TID 475). 9201 bytes result sent to driver +26/04/01 06:11:57 INFO TaskSetManager: Starting task 169.0 in stage 24.0 (TID 479) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9428 bytes) +26/04/01 06:11:57 INFO Executor: Running task 169.0 in stage 24.0 (TID 479) +26/04/01 06:11:57 INFO TaskSetManager: Finished task 165.0 in stage 24.0 (TID 475) in 3348 ms on 10.0.0.133 (executor driver) (166/200) +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:11:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:11:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:00 INFO Executor: Finished task 166.0 in stage 24.0 (TID 476). 8470 bytes result sent to driver +26/04/01 06:12:00 INFO TaskSetManager: Starting task 170.0 in stage 24.0 (TID 480) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:00 INFO Executor: Running task 170.0 in stage 24.0 (TID 480) +26/04/01 06:12:00 INFO TaskSetManager: Finished task 166.0 in stage 24.0 (TID 476) in 3452 ms on 10.0.0.133 (executor driver) (167/200) +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:00 INFO Executor: Finished task 167.0 in stage 24.0 (TID 477). 8470 bytes result sent to driver +26/04/01 06:12:00 INFO TaskSetManager: Starting task 171.0 in stage 24.0 (TID 481) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:00 INFO Executor: Running task 171.0 in stage 24.0 (TID 481) +26/04/01 06:12:00 INFO TaskSetManager: Finished task 167.0 in stage 24.0 (TID 477) in 3453 ms on 10.0.0.133 (executor driver) (168/200) +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:00 INFO Executor: Finished task 168.0 in stage 24.0 (TID 478). 8470 bytes result sent to driver +26/04/01 06:12:00 INFO TaskSetManager: Starting task 172.0 in stage 24.0 (TID 482) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:00 INFO Executor: Running task 172.0 in stage 24.0 (TID 482) +26/04/01 06:12:00 INFO TaskSetManager: Finished task 168.0 in stage 24.0 (TID 478) in 3403 ms on 10.0.0.133 (executor driver) (169/200) +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:01 INFO Executor: Finished task 169.0 in stage 24.0 (TID 479). 8470 bytes result sent to driver +26/04/01 06:12:01 INFO TaskSetManager: Starting task 173.0 in stage 24.0 (TID 483) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:01 INFO Executor: Running task 173.0 in stage 24.0 (TID 483) +26/04/01 06:12:01 INFO TaskSetManager: Finished task 169.0 in stage 24.0 (TID 479) in 3387 ms on 10.0.0.133 (executor driver) (170/200) +26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:03 INFO Executor: Finished task 171.0 in stage 24.0 (TID 481). 8470 bytes result sent to driver +26/04/01 06:12:03 INFO TaskSetManager: Starting task 174.0 in stage 24.0 (TID 484) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:03 INFO Executor: Running task 174.0 in stage 24.0 (TID 484) +26/04/01 06:12:03 INFO TaskSetManager: Finished task 171.0 in stage 24.0 (TID 481) in 3420 ms on 10.0.0.133 (executor driver) (171/200) +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:03 INFO Executor: Finished task 170.0 in stage 24.0 (TID 480). 8470 bytes result sent to driver +26/04/01 06:12:03 INFO TaskSetManager: Starting task 175.0 in stage 24.0 (TID 485) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:03 INFO Executor: Running task 175.0 in stage 24.0 (TID 485) +26/04/01 06:12:03 INFO TaskSetManager: Finished task 170.0 in stage 24.0 (TID 480) in 3432 ms on 10.0.0.133 (executor driver) (172/200) +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:04 INFO Executor: Finished task 172.0 in stage 24.0 (TID 482). 8470 bytes result sent to driver +26/04/01 06:12:04 INFO TaskSetManager: Starting task 176.0 in stage 24.0 (TID 486) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:04 INFO Executor: Running task 176.0 in stage 24.0 (TID 486) +26/04/01 06:12:04 INFO TaskSetManager: Finished task 172.0 in stage 24.0 (TID 482) in 3423 ms on 10.0.0.133 (executor driver) (173/200) +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:04 INFO Executor: Finished task 173.0 in stage 24.0 (TID 483). 8470 bytes result sent to driver +26/04/01 06:12:04 INFO TaskSetManager: Starting task 177.0 in stage 24.0 (TID 487) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:04 INFO Executor: Running task 177.0 in stage 24.0 (TID 487) +26/04/01 06:12:04 INFO TaskSetManager: Finished task 173.0 in stage 24.0 (TID 483) in 3371 ms on 10.0.0.133 (executor driver) (174/200) +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:07 INFO Executor: Finished task 175.0 in stage 24.0 (TID 485). 9201 bytes result sent to driver +26/04/01 06:12:07 INFO TaskSetManager: Starting task 178.0 in stage 24.0 (TID 488) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:07 INFO TaskSetManager: Finished task 175.0 in stage 24.0 (TID 485) in 3420 ms on 10.0.0.133 (executor driver) (175/200) +26/04/01 06:12:07 INFO Executor: Running task 178.0 in stage 24.0 (TID 488) +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:07 INFO Executor: Finished task 174.0 in stage 24.0 (TID 484). 9201 bytes result sent to driver +26/04/01 06:12:07 INFO TaskSetManager: Starting task 179.0 in stage 24.0 (TID 489) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:07 INFO Executor: Running task 179.0 in stage 24.0 (TID 489) +26/04/01 06:12:07 INFO TaskSetManager: Finished task 174.0 in stage 24.0 (TID 484) in 3438 ms on 10.0.0.133 (executor driver) (176/200) +26/04/01 06:12:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:07 INFO Executor: Finished task 176.0 in stage 24.0 (TID 486). 9201 bytes result sent to driver +26/04/01 06:12:07 INFO TaskSetManager: Starting task 180.0 in stage 24.0 (TID 490) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:07 INFO Executor: Running task 180.0 in stage 24.0 (TID 490) +26/04/01 06:12:07 INFO TaskSetManager: Finished task 176.0 in stage 24.0 (TID 486) in 3429 ms on 10.0.0.133 (executor driver) (177/200) +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:08 INFO Executor: Finished task 177.0 in stage 24.0 (TID 487). 9201 bytes result sent to driver +26/04/01 06:12:08 INFO TaskSetManager: Starting task 181.0 in stage 24.0 (TID 491) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:08 INFO Executor: Running task 181.0 in stage 24.0 (TID 491) +26/04/01 06:12:08 INFO TaskSetManager: Finished task 177.0 in stage 24.0 (TID 487) in 3370 ms on 10.0.0.133 (executor driver) (178/200) +26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:10 INFO Executor: Finished task 179.0 in stage 24.0 (TID 489). 8470 bytes result sent to driver +26/04/01 06:12:10 INFO TaskSetManager: Starting task 182.0 in stage 24.0 (TID 492) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:10 INFO Executor: Running task 182.0 in stage 24.0 (TID 492) +26/04/01 06:12:10 INFO TaskSetManager: Finished task 179.0 in stage 24.0 (TID 489) in 3423 ms on 10.0.0.133 (executor driver) (179/200) +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:10 INFO Executor: Finished task 178.0 in stage 24.0 (TID 488). 8470 bytes result sent to driver +26/04/01 06:12:10 INFO TaskSetManager: Starting task 183.0 in stage 24.0 (TID 493) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:10 INFO Executor: Running task 183.0 in stage 24.0 (TID 493) +26/04/01 06:12:10 INFO TaskSetManager: Finished task 178.0 in stage 24.0 (TID 488) in 3435 ms on 10.0.0.133 (executor driver) (180/200) +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:11 INFO Executor: Finished task 180.0 in stage 24.0 (TID 490). 8470 bytes result sent to driver +26/04/01 06:12:11 INFO TaskSetManager: Starting task 184.0 in stage 24.0 (TID 494) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:11 INFO Executor: Running task 184.0 in stage 24.0 (TID 494) +26/04/01 06:12:11 INFO TaskSetManager: Finished task 180.0 in stage 24.0 (TID 490) in 3420 ms on 10.0.0.133 (executor driver) (181/200) +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:11 INFO Executor: Finished task 181.0 in stage 24.0 (TID 491). 8470 bytes result sent to driver +26/04/01 06:12:11 INFO TaskSetManager: Starting task 185.0 in stage 24.0 (TID 495) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:11 INFO Executor: Running task 185.0 in stage 24.0 (TID 495) +26/04/01 06:12:11 INFO TaskSetManager: Finished task 181.0 in stage 24.0 (TID 491) in 3373 ms on 10.0.0.133 (executor driver) (182/200) +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:13 INFO Executor: Finished task 183.0 in stage 24.0 (TID 493). 8470 bytes result sent to driver +26/04/01 06:12:13 INFO TaskSetManager: Starting task 186.0 in stage 24.0 (TID 496) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:13 INFO Executor: Running task 186.0 in stage 24.0 (TID 496) +26/04/01 06:12:13 INFO TaskSetManager: Finished task 183.0 in stage 24.0 (TID 493) in 3428 ms on 10.0.0.133 (executor driver) (183/200) +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:13 INFO Executor: Finished task 182.0 in stage 24.0 (TID 492). 8470 bytes result sent to driver +26/04/01 06:12:13 INFO TaskSetManager: Starting task 187.0 in stage 24.0 (TID 497) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:13 INFO TaskSetManager: Finished task 182.0 in stage 24.0 (TID 492) in 3435 ms on 10.0.0.133 (executor driver) (184/200) +26/04/01 06:12:13 INFO Executor: Running task 187.0 in stage 24.0 (TID 497) +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:14 INFO Executor: Finished task 184.0 in stage 24.0 (TID 494). 8470 bytes result sent to driver +26/04/01 06:12:14 INFO TaskSetManager: Starting task 188.0 in stage 24.0 (TID 498) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:14 INFO Executor: Running task 188.0 in stage 24.0 (TID 498) +26/04/01 06:12:14 INFO TaskSetManager: Finished task 184.0 in stage 24.0 (TID 494) in 3373 ms on 10.0.0.133 (executor driver) (185/200) +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:14 INFO Executor: Finished task 185.0 in stage 24.0 (TID 495). 8470 bytes result sent to driver +26/04/01 06:12:14 INFO TaskSetManager: Starting task 189.0 in stage 24.0 (TID 499) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:14 INFO TaskSetManager: Finished task 185.0 in stage 24.0 (TID 495) in 3384 ms on 10.0.0.133 (executor driver) (186/200) +26/04/01 06:12:14 INFO Executor: Running task 189.0 in stage 24.0 (TID 499) +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:17 INFO Executor: Finished task 187.0 in stage 24.0 (TID 497). 9201 bytes result sent to driver +26/04/01 06:12:17 INFO TaskSetManager: Starting task 190.0 in stage 24.0 (TID 500) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:17 INFO Executor: Running task 190.0 in stage 24.0 (TID 500) +26/04/01 06:12:17 INFO TaskSetManager: Finished task 187.0 in stage 24.0 (TID 497) in 3431 ms on 10.0.0.133 (executor driver) (187/200) +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:17 INFO Executor: Finished task 186.0 in stage 24.0 (TID 496). 9201 bytes result sent to driver +26/04/01 06:12:17 INFO TaskSetManager: Starting task 191.0 in stage 24.0 (TID 501) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:17 INFO Executor: Running task 191.0 in stage 24.0 (TID 501) +26/04/01 06:12:17 INFO TaskSetManager: Finished task 186.0 in stage 24.0 (TID 496) in 3450 ms on 10.0.0.133 (executor driver) (188/200) +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:17 INFO Executor: Finished task 188.0 in stage 24.0 (TID 498). 9201 bytes result sent to driver +26/04/01 06:12:17 INFO TaskSetManager: Starting task 192.0 in stage 24.0 (TID 502) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:17 INFO Executor: Running task 192.0 in stage 24.0 (TID 502) +26/04/01 06:12:17 INFO TaskSetManager: Finished task 188.0 in stage 24.0 (TID 498) in 3344 ms on 10.0.0.133 (executor driver) (189/200) +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:18 INFO Executor: Finished task 189.0 in stage 24.0 (TID 499). 9201 bytes result sent to driver +26/04/01 06:12:18 INFO TaskSetManager: Starting task 193.0 in stage 24.0 (TID 503) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:18 INFO Executor: Running task 193.0 in stage 24.0 (TID 503) +26/04/01 06:12:18 INFO TaskSetManager: Finished task 189.0 in stage 24.0 (TID 499) in 3357 ms on 10.0.0.133 (executor driver) (190/200) +26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:20 INFO Executor: Finished task 191.0 in stage 24.0 (TID 501). 8470 bytes result sent to driver +26/04/01 06:12:20 INFO TaskSetManager: Starting task 194.0 in stage 24.0 (TID 504) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:20 INFO Executor: Running task 194.0 in stage 24.0 (TID 504) +26/04/01 06:12:20 INFO TaskSetManager: Finished task 191.0 in stage 24.0 (TID 501) in 3399 ms on 10.0.0.133 (executor driver) (191/200) +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:20 INFO Executor: Finished task 190.0 in stage 24.0 (TID 500). 8470 bytes result sent to driver +26/04/01 06:12:20 INFO TaskSetManager: Starting task 195.0 in stage 24.0 (TID 505) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:20 INFO Executor: Running task 195.0 in stage 24.0 (TID 505) +26/04/01 06:12:20 INFO TaskSetManager: Finished task 190.0 in stage 24.0 (TID 500) in 3423 ms on 10.0.0.133 (executor driver) (192/200) +26/04/01 06:12:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:21 INFO Executor: Finished task 192.0 in stage 24.0 (TID 502). 8470 bytes result sent to driver +26/04/01 06:12:21 INFO TaskSetManager: Starting task 196.0 in stage 24.0 (TID 506) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:21 INFO Executor: Running task 196.0 in stage 24.0 (TID 506) +26/04/01 06:12:21 INFO TaskSetManager: Finished task 192.0 in stage 24.0 (TID 502) in 3359 ms on 10.0.0.133 (executor driver) (193/200) +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:21 INFO Executor: Finished task 193.0 in stage 24.0 (TID 503). 8470 bytes result sent to driver +26/04/01 06:12:21 INFO TaskSetManager: Starting task 197.0 in stage 24.0 (TID 507) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:21 INFO Executor: Running task 197.0 in stage 24.0 (TID 507) +26/04/01 06:12:21 INFO TaskSetManager: Finished task 193.0 in stage 24.0 (TID 503) in 3362 ms on 10.0.0.133 (executor driver) (194/200) +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:24 INFO Executor: Finished task 195.0 in stage 24.0 (TID 505). 8470 bytes result sent to driver +26/04/01 06:12:24 INFO TaskSetManager: Starting task 198.0 in stage 24.0 (TID 508) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:24 INFO Executor: Running task 198.0 in stage 24.0 (TID 508) +26/04/01 06:12:24 INFO TaskSetManager: Finished task 195.0 in stage 24.0 (TID 505) in 3416 ms on 10.0.0.133 (executor driver) (195/200) +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:24 INFO Executor: Finished task 194.0 in stage 24.0 (TID 504). 8470 bytes result sent to driver +26/04/01 06:12:24 INFO TaskSetManager: Starting task 199.0 in stage 24.0 (TID 509) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9428 bytes) +26/04/01 06:12:24 INFO TaskSetManager: Finished task 194.0 in stage 24.0 (TID 504) in 3439 ms on 10.0.0.133 (executor driver) (196/200) +26/04/01 06:12:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:24 INFO Executor: Running task 199.0 in stage 24.0 (TID 509) +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:24 INFO Executor: Finished task 196.0 in stage 24.0 (TID 506). 8470 bytes result sent to driver +26/04/01 06:12:24 INFO TaskSetManager: Finished task 196.0 in stage 24.0 (TID 506) in 3363 ms on 10.0.0.133 (executor driver) (197/200) +26/04/01 06:12:24 INFO Executor: Finished task 197.0 in stage 24.0 (TID 507). 8470 bytes result sent to driver +26/04/01 06:12:24 INFO TaskSetManager: Finished task 197.0 in stage 24.0 (TID 507) in 3351 ms on 10.0.0.133 (executor driver) (198/200) +26/04/01 06:12:27 INFO Executor: Finished task 199.0 in stage 24.0 (TID 509). 9201 bytes result sent to driver +26/04/01 06:12:27 INFO TaskSetManager: Finished task 199.0 in stage 24.0 (TID 509) in 3184 ms on 10.0.0.133 (executor driver) (199/200) +26/04/01 06:12:27 INFO Executor: Finished task 198.0 in stage 24.0 (TID 508). 9201 bytes result sent to driver +26/04/01 06:12:27 INFO TaskSetManager: Finished task 198.0 in stage 24.0 (TID 508) in 3202 ms on 10.0.0.133 (executor driver) (200/200) +26/04/01 06:12:27 INFO TaskSchedulerImpl: Removed TaskSet 24.0, whose tasks have all completed, from pool +26/04/01 06:12:27 INFO DAGScheduler: ShuffleMapStage 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 172.132 s +26/04/01 06:12:27 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:12:27 INFO DAGScheduler: running: Set() +26/04/01 06:12:27 INFO DAGScheduler: waiting: Set() +26/04/01 06:12:27 INFO DAGScheduler: failed: Set() +26/04/01 06:12:27 INFO ShufflePartitionsUtil: For shuffle(6, 4), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 06:12:27 INFO DAGScheduler: Registering RDD 54 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 7 +26/04/01 06:12:27 INFO DAGScheduler: Got map stage job 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 25 output partitions +26/04/01 06:12:27 INFO DAGScheduler: Final stage: ShuffleMapStage 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:12:27 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 30, ShuffleMapStage 29) +26/04/01 06:12:27 INFO DAGScheduler: Missing parents: List() +26/04/01 06:12:27 INFO DAGScheduler: Submitting ShuffleMapStage 31 (MapPartitionsRDD[54] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:12:27 INFO MemoryStore: Block broadcast_27 stored as values in memory (estimated size 39.3 KiB, free 8.6 GiB) +26/04/01 06:12:27 INFO MemoryStore: Block broadcast_27_piece0 stored as bytes in memory (estimated size 12.3 KiB, free 8.6 GiB) +26/04/01 06:12:27 INFO BlockManagerInfo: Added broadcast_27_piece0 in memory on 10.0.0.133:58259 (size: 12.3 KiB, free: 8.6 GiB) +26/04/01 06:12:27 INFO SparkContext: Created broadcast 27 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:12:27 INFO DAGScheduler: Submitting 25 missing tasks from ShuffleMapStage 31 (MapPartitionsRDD[54] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 06:12:27 INFO TaskSchedulerImpl: Adding task set 31.0 with 25 tasks resource profile 0 +26/04/01 06:12:27 INFO TaskSetManager: Starting task 0.0 in stage 31.0 (TID 510) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:27 INFO TaskSetManager: Starting task 1.0 in stage 31.0 (TID 511) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:27 INFO TaskSetManager: Starting task 2.0 in stage 31.0 (TID 512) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:27 INFO TaskSetManager: Starting task 3.0 in stage 31.0 (TID 513) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:27 INFO Executor: Running task 0.0 in stage 31.0 (TID 510) +26/04/01 06:12:27 INFO Executor: Running task 2.0 in stage 31.0 (TID 512) +26/04/01 06:12:27 INFO Executor: Running task 3.0 in stage 31.0 (TID 513) +26/04/01 06:12:27 INFO Executor: Running task 1.0 in stage 31.0 (TID 511) +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (320.9 KiB) non-empty blocks including 4 (320.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (320.9 KiB) non-empty blocks including 4 (320.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:36 INFO Executor: Finished task 3.0 in stage 31.0 (TID 513). 11071 bytes result sent to driver +26/04/01 06:12:36 INFO TaskSetManager: Starting task 4.0 in stage 31.0 (TID 514) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:36 INFO TaskSetManager: Finished task 3.0 in stage 31.0 (TID 513) in 8867 ms on 10.0.0.133 (executor driver) (1/25) +26/04/01 06:12:36 INFO Executor: Running task 4.0 in stage 31.0 (TID 514) +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:36 INFO Executor: Finished task 1.0 in stage 31.0 (TID 511). 11071 bytes result sent to driver +26/04/01 06:12:36 INFO TaskSetManager: Starting task 5.0 in stage 31.0 (TID 515) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:36 INFO Executor: Running task 5.0 in stage 31.0 (TID 515) +26/04/01 06:12:36 INFO TaskSetManager: Finished task 1.0 in stage 31.0 (TID 511) in 8885 ms on 10.0.0.133 (executor driver) (2/25) +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:36 INFO Executor: Finished task 0.0 in stage 31.0 (TID 510). 11071 bytes result sent to driver +26/04/01 06:12:36 INFO TaskSetManager: Starting task 6.0 in stage 31.0 (TID 516) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:36 INFO Executor: Running task 6.0 in stage 31.0 (TID 516) +26/04/01 06:12:36 INFO TaskSetManager: Finished task 0.0 in stage 31.0 (TID 510) in 8900 ms on 10.0.0.133 (executor driver) (3/25) +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO Executor: Finished task 2.0 in stage 31.0 (TID 512). 11071 bytes result sent to driver +26/04/01 06:12:36 INFO TaskSetManager: Starting task 7.0 in stage 31.0 (TID 517) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:36 INFO TaskSetManager: Finished task 2.0 in stage 31.0 (TID 512) in 8905 ms on 10.0.0.133 (executor driver) (4/25) +26/04/01 06:12:36 INFO Executor: Running task 7.0 in stage 31.0 (TID 517) +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:45 INFO Executor: Finished task 4.0 in stage 31.0 (TID 514). 10340 bytes result sent to driver +26/04/01 06:12:45 INFO TaskSetManager: Starting task 8.0 in stage 31.0 (TID 518) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:45 INFO Executor: Running task 8.0 in stage 31.0 (TID 518) +26/04/01 06:12:45 INFO TaskSetManager: Finished task 4.0 in stage 31.0 (TID 514) in 8783 ms on 10.0.0.133 (executor driver) (5/25) +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:45 INFO Executor: Finished task 5.0 in stage 31.0 (TID 515). 10340 bytes result sent to driver +26/04/01 06:12:45 INFO TaskSetManager: Starting task 9.0 in stage 31.0 (TID 519) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:45 INFO Executor: Running task 9.0 in stage 31.0 (TID 519) +26/04/01 06:12:45 INFO TaskSetManager: Finished task 5.0 in stage 31.0 (TID 515) in 8814 ms on 10.0.0.133 (executor driver) (6/25) +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (317.2 KiB) non-empty blocks including 4 (317.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (317.2 KiB) non-empty blocks including 4 (317.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:45 INFO Executor: Finished task 6.0 in stage 31.0 (TID 516). 10340 bytes result sent to driver +26/04/01 06:12:45 INFO Executor: Finished task 7.0 in stage 31.0 (TID 517). 10340 bytes result sent to driver +26/04/01 06:12:45 INFO TaskSetManager: Starting task 10.0 in stage 31.0 (TID 520) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:45 INFO Executor: Running task 10.0 in stage 31.0 (TID 520) +26/04/01 06:12:45 INFO TaskSetManager: Starting task 11.0 in stage 31.0 (TID 521) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:45 INFO TaskSetManager: Finished task 6.0 in stage 31.0 (TID 516) in 8837 ms on 10.0.0.133 (executor driver) (7/25) +26/04/01 06:12:45 INFO Executor: Running task 11.0 in stage 31.0 (TID 521) +26/04/01 06:12:45 INFO TaskSetManager: Finished task 7.0 in stage 31.0 (TID 517) in 8832 ms on 10.0.0.133 (executor driver) (8/25) +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:53 INFO Executor: Finished task 8.0 in stage 31.0 (TID 518). 11071 bytes result sent to driver +26/04/01 06:12:53 INFO TaskSetManager: Starting task 12.0 in stage 31.0 (TID 522) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:53 INFO Executor: Running task 12.0 in stage 31.0 (TID 522) +26/04/01 06:12:53 INFO TaskSetManager: Finished task 8.0 in stage 31.0 (TID 518) in 8774 ms on 10.0.0.133 (executor driver) (9/25) +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:53 INFO Executor: Finished task 9.0 in stage 31.0 (TID 519). 11071 bytes result sent to driver +26/04/01 06:12:53 INFO TaskSetManager: Starting task 13.0 in stage 31.0 (TID 523) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:53 INFO TaskSetManager: Finished task 9.0 in stage 31.0 (TID 519) in 8745 ms on 10.0.0.133 (executor driver) (10/25) +26/04/01 06:12:53 INFO Executor: Running task 13.0 in stage 31.0 (TID 523) +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (322.7 KiB) non-empty blocks including 4 (322.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (322.7 KiB) non-empty blocks including 4 (322.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:53 INFO Executor: Finished task 11.0 in stage 31.0 (TID 521). 11071 bytes result sent to driver +26/04/01 06:12:53 INFO TaskSetManager: Starting task 14.0 in stage 31.0 (TID 524) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:53 INFO Executor: Running task 14.0 in stage 31.0 (TID 524) +26/04/01 06:12:53 INFO TaskSetManager: Finished task 11.0 in stage 31.0 (TID 521) in 8755 ms on 10.0.0.133 (executor driver) (11/25) +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO Executor: Finished task 10.0 in stage 31.0 (TID 520). 11071 bytes result sent to driver +26/04/01 06:12:53 INFO TaskSetManager: Starting task 15.0 in stage 31.0 (TID 525) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9719 bytes) +26/04/01 06:12:53 INFO Executor: Running task 15.0 in stage 31.0 (TID 525) +26/04/01 06:12:53 INFO TaskSetManager: Finished task 10.0 in stage 31.0 (TID 520) in 8763 ms on 10.0.0.133 (executor driver) (12/25) +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:02 INFO Executor: Finished task 12.0 in stage 31.0 (TID 522). 11071 bytes result sent to driver +26/04/01 06:13:02 INFO TaskSetManager: Starting task 16.0 in stage 31.0 (TID 526) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9719 bytes) +26/04/01 06:13:02 INFO Executor: Running task 16.0 in stage 31.0 (TID 526) +26/04/01 06:13:02 INFO TaskSetManager: Finished task 12.0 in stage 31.0 (TID 522) in 8748 ms on 10.0.0.133 (executor driver) (13/25) +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:02 INFO Executor: Finished task 13.0 in stage 31.0 (TID 523). 11071 bytes result sent to driver +26/04/01 06:13:02 INFO TaskSetManager: Starting task 17.0 in stage 31.0 (TID 527) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9719 bytes) +26/04/01 06:13:02 INFO TaskSetManager: Finished task 13.0 in stage 31.0 (TID 523) in 8774 ms on 10.0.0.133 (executor driver) (14/25) +26/04/01 06:13:02 INFO Executor: Running task 17.0 in stage 31.0 (TID 527) +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:02 INFO Executor: Finished task 15.0 in stage 31.0 (TID 525). 11071 bytes result sent to driver +26/04/01 06:13:02 INFO TaskSetManager: Starting task 18.0 in stage 31.0 (TID 528) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9719 bytes) +26/04/01 06:13:02 INFO Executor: Running task 18.0 in stage 31.0 (TID 528) +26/04/01 06:13:02 INFO TaskSetManager: Finished task 15.0 in stage 31.0 (TID 525) in 8795 ms on 10.0.0.133 (executor driver) (15/25) +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:02 INFO Executor: Finished task 14.0 in stage 31.0 (TID 524). 11071 bytes result sent to driver +26/04/01 06:13:02 INFO TaskSetManager: Starting task 19.0 in stage 31.0 (TID 529) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9719 bytes) +26/04/01 06:13:02 INFO TaskSetManager: Finished task 14.0 in stage 31.0 (TID 524) in 8816 ms on 10.0.0.133 (executor driver) (16/25) +26/04/01 06:13:02 INFO Executor: Running task 19.0 in stage 31.0 (TID 529) +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:11 INFO Executor: Finished task 16.0 in stage 31.0 (TID 526). 11071 bytes result sent to driver +26/04/01 06:13:11 INFO TaskSetManager: Starting task 20.0 in stage 31.0 (TID 530) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9719 bytes) +26/04/01 06:13:11 INFO Executor: Running task 20.0 in stage 31.0 (TID 530) +26/04/01 06:13:11 INFO TaskSetManager: Finished task 16.0 in stage 31.0 (TID 526) in 8752 ms on 10.0.0.133 (executor driver) (17/25) +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:11 INFO Executor: Finished task 17.0 in stage 31.0 (TID 527). 11071 bytes result sent to driver +26/04/01 06:13:11 INFO TaskSetManager: Starting task 21.0 in stage 31.0 (TID 531) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9719 bytes) +26/04/01 06:13:11 INFO Executor: Running task 21.0 in stage 31.0 (TID 531) +26/04/01 06:13:11 INFO TaskSetManager: Finished task 17.0 in stage 31.0 (TID 527) in 8756 ms on 10.0.0.133 (executor driver) (18/25) +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:11 INFO Executor: Finished task 18.0 in stage 31.0 (TID 528). 11071 bytes result sent to driver +26/04/01 06:13:11 INFO TaskSetManager: Starting task 22.0 in stage 31.0 (TID 532) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9719 bytes) +26/04/01 06:13:11 INFO TaskSetManager: Finished task 18.0 in stage 31.0 (TID 528) in 8700 ms on 10.0.0.133 (executor driver) (19/25) +26/04/01 06:13:11 INFO Executor: Running task 22.0 in stage 31.0 (TID 532) +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:11 INFO Executor: Finished task 19.0 in stage 31.0 (TID 529). 11071 bytes result sent to driver +26/04/01 06:13:11 INFO TaskSetManager: Starting task 23.0 in stage 31.0 (TID 533) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9719 bytes) +26/04/01 06:13:11 INFO Executor: Running task 23.0 in stage 31.0 (TID 533) +26/04/01 06:13:11 INFO TaskSetManager: Finished task 19.0 in stage 31.0 (TID 529) in 8726 ms on 10.0.0.133 (executor driver) (20/25) +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (322.8 KiB) non-empty blocks including 4 (322.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (322.8 KiB) non-empty blocks including 4 (322.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:20 INFO Executor: Finished task 20.0 in stage 31.0 (TID 530). 11071 bytes result sent to driver +26/04/01 06:13:20 INFO TaskSetManager: Starting task 24.0 in stage 31.0 (TID 534) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9719 bytes) +26/04/01 06:13:20 INFO Executor: Running task 24.0 in stage 31.0 (TID 534) +26/04/01 06:13:20 INFO TaskSetManager: Finished task 20.0 in stage 31.0 (TID 530) in 8794 ms on 10.0.0.133 (executor driver) (21/25) +26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:20 INFO Executor: Finished task 21.0 in stage 31.0 (TID 531). 11071 bytes result sent to driver +26/04/01 06:13:20 INFO TaskSetManager: Finished task 21.0 in stage 31.0 (TID 531) in 8760 ms on 10.0.0.133 (executor driver) (22/25) +26/04/01 06:13:20 INFO Executor: Finished task 22.0 in stage 31.0 (TID 532). 11071 bytes result sent to driver +26/04/01 06:13:20 INFO TaskSetManager: Finished task 22.0 in stage 31.0 (TID 532) in 8754 ms on 10.0.0.133 (executor driver) (23/25) +26/04/01 06:13:20 INFO Executor: Finished task 23.0 in stage 31.0 (TID 533). 11071 bytes result sent to driver +26/04/01 06:13:20 INFO TaskSetManager: Finished task 23.0 in stage 31.0 (TID 533) in 8731 ms on 10.0.0.133 (executor driver) (24/25) +26/04/01 06:13:26 INFO Executor: Finished task 24.0 in stage 31.0 (TID 534). 11071 bytes result sent to driver +26/04/01 06:13:26 INFO TaskSetManager: Finished task 24.0 in stage 31.0 (TID 534) in 6142 ms on 10.0.0.133 (executor driver) (25/25) +26/04/01 06:13:26 INFO TaskSchedulerImpl: Removed TaskSet 31.0, whose tasks have all completed, from pool +26/04/01 06:13:26 INFO DAGScheduler: ShuffleMapStage 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 58.860 s +26/04/01 06:13:26 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:13:26 INFO DAGScheduler: running: Set() +26/04/01 06:13:26 INFO DAGScheduler: waiting: Set() +26/04/01 06:13:26 INFO DAGScheduler: failed: Set() +26/04/01 06:13:26 INFO ShufflePartitionsUtil: For shuffle(7), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 06:13:26 INFO CodeGenerator: Code generated in 56.085125 ms +26/04/01 06:13:26 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:13:26 INFO DAGScheduler: Got job 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:13:26 INFO DAGScheduler: Final stage: ResultStage 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:13:26 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 38) +26/04/01 06:13:26 INFO DAGScheduler: Missing parents: List() +26/04/01 06:13:26 INFO DAGScheduler: Submitting ResultStage 39 (MapPartitionsRDD[59] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:13:26 INFO MemoryStore: Block broadcast_28 stored as values in memory (estimated size 22.7 KiB, free 8.6 GiB) +26/04/01 06:13:26 INFO MemoryStore: Block broadcast_28_piece0 stored as bytes in memory (estimated size 8.4 KiB, free 8.6 GiB) +26/04/01 06:13:26 INFO BlockManagerInfo: Added broadcast_28_piece0 in memory on 10.0.0.133:58259 (size: 8.4 KiB, free: 8.6 GiB) +26/04/01 06:13:26 INFO SparkContext: Created broadcast 28 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:13:26 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 39 (MapPartitionsRDD[59] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:13:26 INFO TaskSchedulerImpl: Adding task set 39.0 with 1 tasks resource profile 0 +26/04/01 06:13:26 INFO TaskSetManager: Starting task 0.0 in stage 39.0 (TID 535) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) +26/04/01 06:13:26 INFO Executor: Running task 0.0 in stage 39.0 (TID 535) +26/04/01 06:13:26 INFO ShuffleBlockFetcherIterator: Getting 25 (54.3 KiB) non-empty blocks including 25 (54.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:26 INFO CodeGenerator: Code generated in 5.687333 ms +26/04/01 06:13:26 INFO Executor: Finished task 0.0 in stage 39.0 (TID 535). 5177 bytes result sent to driver +26/04/01 06:13:26 INFO TaskSetManager: Finished task 0.0 in stage 39.0 (TID 535) in 48 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:13:26 INFO TaskSchedulerImpl: Removed TaskSet 39.0, whose tasks have all completed, from pool +26/04/01 06:13:26 INFO DAGScheduler: ResultStage 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.049 s +26/04/01 06:13:26 INFO DAGScheduler: Job 19 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:13:26 INFO TaskSchedulerImpl: Killing all running tasks in stage 39: Stage finished +26/04/01 06:13:26 INFO DAGScheduler: Job 19 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.051501 s +26/04/01 06:13:26 INFO DAGScheduler: Registering RDD 60 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 8 +26/04/01 06:13:26 INFO DAGScheduler: Got map stage job 20 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:13:26 INFO DAGScheduler: Final stage: ShuffleMapStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:13:26 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 46) +26/04/01 06:13:26 INFO DAGScheduler: Missing parents: List() +26/04/01 06:13:26 INFO DAGScheduler: Submitting ShuffleMapStage 47 (MapPartitionsRDD[60] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:13:26 INFO MemoryStore: Block broadcast_29 stored as values in memory (estimated size 25.0 KiB, free 8.6 GiB) +26/04/01 06:13:26 INFO MemoryStore: Block broadcast_29_piece0 stored as bytes in memory (estimated size 9.3 KiB, free 8.6 GiB) +26/04/01 06:13:26 INFO BlockManagerInfo: Added broadcast_29_piece0 in memory on 10.0.0.133:58259 (size: 9.3 KiB, free: 8.6 GiB) +26/04/01 06:13:26 INFO SparkContext: Created broadcast 29 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:13:26 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 47 (MapPartitionsRDD[60] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:13:26 INFO TaskSchedulerImpl: Adding task set 47.0 with 1 tasks resource profile 0 +26/04/01 06:13:26 INFO TaskSetManager: Starting task 0.0 in stage 47.0 (TID 536) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9404 bytes) +26/04/01 06:13:26 INFO Executor: Running task 0.0 in stage 47.0 (TID 536) +26/04/01 06:13:26 INFO CodeGenerator: Code generated in 3.263208 ms +26/04/01 06:13:26 INFO ShuffleBlockFetcherIterator: Getting 25 (54.3 KiB) non-empty blocks including 25 (54.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:26 INFO Executor: Finished task 0.0 in stage 47.0 (TID 536). 6566 bytes result sent to driver +26/04/01 06:13:26 INFO TaskSetManager: Finished task 0.0 in stage 47.0 (TID 536) in 47 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:13:26 INFO TaskSchedulerImpl: Removed TaskSet 47.0, whose tasks have all completed, from pool +26/04/01 06:13:26 INFO DAGScheduler: ShuffleMapStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.051 s +26/04/01 06:13:26 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:13:26 INFO DAGScheduler: running: Set() +26/04/01 06:13:26 INFO DAGScheduler: waiting: Set() +26/04/01 06:13:26 INFO DAGScheduler: failed: Set() +26/04/01 06:13:26 INFO ShufflePartitionsUtil: For shuffle(8), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 06:13:26 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:13:26 INFO DAGScheduler: Got job 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:13:26 INFO DAGScheduler: Final stage: ResultStage 56 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:13:26 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 55) +26/04/01 06:13:26 INFO DAGScheduler: Missing parents: List() +26/04/01 06:13:26 INFO DAGScheduler: Submitting ResultStage 56 (MapPartitionsRDD[64] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:13:26 INFO MemoryStore: Block broadcast_30 stored as values in memory (estimated size 18.6 KiB, free 8.6 GiB) +26/04/01 06:13:26 INFO MemoryStore: Block broadcast_30_piece0 stored as bytes in memory (estimated size 8.8 KiB, free 8.6 GiB) +26/04/01 06:13:26 INFO BlockManagerInfo: Added broadcast_30_piece0 in memory on 10.0.0.133:58259 (size: 8.8 KiB, free: 8.6 GiB) +26/04/01 06:13:26 INFO SparkContext: Created broadcast 30 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:13:26 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 56 (MapPartitionsRDD[64] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:13:26 INFO TaskSchedulerImpl: Adding task set 56.0 with 1 tasks resource profile 0 +26/04/01 06:13:26 INFO TaskSetManager: Starting task 0.0 in stage 56.0 (TID 537) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) +26/04/01 06:13:26 INFO Executor: Running task 0.0 in stage 56.0 (TID 537) +26/04/01 06:13:26 INFO ShuffleBlockFetcherIterator: Getting 1 (1620.0 B) non-empty blocks including 1 (1620.0 B) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:13:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:13:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB +26/04/01 06:13:26 INFO Executor: Finished task 0.0 in stage 56.0 (TID 537). 5280 bytes result sent to driver +26/04/01 06:13:26 INFO TaskSetManager: Finished task 0.0 in stage 56.0 (TID 537) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:13:26 INFO TaskSchedulerImpl: Removed TaskSet 56.0, whose tasks have all completed, from pool +26/04/01 06:13:26 INFO DAGScheduler: ResultStage 56 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.007 s +26/04/01 06:13:26 INFO DAGScheduler: Job 21 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:13:26 INFO TaskSchedulerImpl: Killing all running tasks in stage 56: Stage finished +26/04/01 06:13:26 INFO DAGScheduler: Job 21 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.008645 s +26/04/01 06:13:26 INFO SparkContext: SparkContext is stopping with exitCode 0. +26/04/01 06:13:26 INFO CometDriverPlugin: CometDriverPlugin shutdown +26/04/01 06:13:26 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! +26/04/01 06:13:26 INFO MemoryStore: MemoryStore cleared +26/04/01 06:13:26 INFO BlockManager: BlockManager stopped +26/04/01 06:13:26 INFO BlockManagerMaster: BlockManagerMaster stopped +26/04/01 06:13:26 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! +26/04/01 06:13:26 INFO SparkContext: Successfully stopped SparkContext +26/04/01 06:13:27 INFO ShutdownHookManager: Shutdown hook called +26/04/01 06:13:27 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-73a4ef89-b8ea-48f7-92a1-8593182866f3/pyspark-c6973fe6-e3dd-4d80-9cfd-2e99d29d47fe +26/04/01 06:13:27 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-74acaa95-d245-4d4b-ab22-d5a168d8ad37 +26/04/01 06:13:27 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-73a4ef89-b8ea-48f7-92a1-8593182866f3 + 488.48 real 2097.74 user 21.12 sys + 5803589632 maximum resident set size + 0 average shared memory size + 0 average unshared data size + 0 average unshared stack size + 380771 page reclaims + 36 page faults + 0 swaps + 0 block input operations + 0 block output operations + 1276 messages sent + 2361 messages received + 17 signals received + 36202 voluntary context switches + 773327 involuntary context switches + 33328818155221 instructions retired + 7704033477817 cycles elapsed + 2546407040 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.log b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.log new file mode 100644 index 0000000000..0dec6c4c4b --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.log @@ -0,0 +1,281 @@ +Registering table customer from /opt/tpch/sf100/customer +Registering table lineitem from /opt/tpch/sf100/lineitem +Registering table nation from /opt/tpch/sf100/nation +Registering table orders from /opt/tpch/sf100/orders +Registering table part from /opt/tpch/sf100/part +Registering table partsupp from /opt/tpch/sf100/partsupp +Registering table region from /opt/tpch/sf100/region +Registering table supplier from /opt/tpch/sf100/supplier + +============================================================ +Starting iteration 1 of 1 +============================================================ + +Running query 9 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q9.sql +Executing: -- CometBench-H query 9 derived from TPC-H query 9 under the terms of the TPC Fair Use Policy. +-- TP... +== Physical Plan == +AdaptiveSparkPlan (46) ++- CometSort (45) + +- CometExchange (44) + +- CometHashAggregate (43) + +- CometExchange (42) + +- CometHashAggregate (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (35) + : +- CometSortMergeJoin (34) + : :- CometSort (29) + : : +- CometExchange (28) + : : +- CometProject (27) + : : +- CometSortMergeJoin (26) + : : :- CometSort (21) + : : : +- CometExchange (20) + : : : +- CometProject (19) + : : : +- CometSortMergeJoin (18) + : : : :- CometSort (13) + : : : : +- CometExchange (12) + : : : : +- CometProject (11) + : : : : +- CometSortMergeJoin (10) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometNativeScan parquet (1) + : : : : +- CometSort (9) + : : : : +- CometExchange (8) + : : : : +- CometFilter (7) + : : : : +- CometNativeScan parquet (6) + : : : +- CometSort (17) + : : : +- CometExchange (16) + : : : +- CometFilter (15) + : : : +- CometNativeScan parquet (14) + : : +- CometSort (25) + : : +- CometExchange (24) + : : +- CometFilter (23) + : : +- CometNativeScan parquet (22) + : +- CometSort (33) + : +- CometExchange (32) + : +- CometFilter (31) + : +- CometNativeScan parquet (30) + +- CometBroadcastExchange (38) + +- CometFilter (37) + +- CometNativeScan parquet (36) + + +(1) CometNativeScan parquet +Output [2]: [p_partkey#74L, p_name#75] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/part] +PushedFilters: [IsNotNull(p_name), StringContains(p_name,moccasin), IsNotNull(p_partkey)] +ReadSchema: struct + +(2) CometFilter +Input [2]: [p_partkey#74L, p_name#75] +Condition : ((isnotnull(p_name#75) AND Contains(p_name#75, moccasin)) AND isnotnull(p_partkey#74L)) + +(3) CometProject +Input [2]: [p_partkey#74L, p_name#75] +Arguments: [p_partkey#74L], [p_partkey#74L] + +(4) CometExchange +Input [1]: [p_partkey#74L] +Arguments: hashpartitioning(p_partkey#74L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=376] + +(5) CometSort +Input [1]: [p_partkey#74L] +Arguments: [p_partkey#74L], [p_partkey#74L ASC NULLS FIRST] + +(6) CometNativeScan parquet +Output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] +PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] +ReadSchema: struct + +(7) CometFilter +Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Condition : ((isnotnull(l_partkey#17L) AND isnotnull(l_suppkey#18L)) AND isnotnull(l_orderkey#16L)) + +(8) CometExchange +Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: hashpartitioning(l_partkey#17L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=380] + +(9) CometSort +Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22], [l_partkey#17L ASC NULLS FIRST] + +(10) CometSortMergeJoin +Left output [1]: [p_partkey#74L] +Right output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: [p_partkey#74L], [l_partkey#17L], Inner + +(11) CometProject +Input [7]: [p_partkey#74L, l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22], [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] + +(12) CometExchange +Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: hashpartitioning(l_suppkey#18L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=384] + +(13) CometSort +Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22], [l_suppkey#18L ASC NULLS FIRST] + +(14) CometNativeScan parquet +Output [2]: [s_suppkey#108L, s_nationkey#111L] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/supplier] +PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] +ReadSchema: struct + +(15) CometFilter +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Condition : (isnotnull(s_suppkey#108L) AND isnotnull(s_nationkey#111L)) + +(16) CometExchange +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: hashpartitioning(s_suppkey#108L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=388] + +(17) CometSort +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: [s_suppkey#108L, s_nationkey#111L], [s_suppkey#108L ASC NULLS FIRST] + +(18) CometSortMergeJoin +Left output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Right output [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: [l_suppkey#18L], [s_suppkey#108L], Inner + +(19) CometProject +Input [8]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_suppkey#108L, s_nationkey#111L] +Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L], [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] + +(20) CometExchange +Input [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] +Arguments: hashpartitioning(l_suppkey#18L, l_partkey#17L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=392] + +(21) CometSort +Input [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] +Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L], [l_suppkey#18L ASC NULLS FIRST, l_partkey#17L ASC NULLS FIRST] + +(22) CometNativeScan parquet +Output [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/partsupp] +PushedFilters: [IsNotNull(ps_suppkey), IsNotNull(ps_partkey)] +ReadSchema: struct + +(23) CometFilter +Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Condition : (isnotnull(ps_suppkey#93L) AND isnotnull(ps_partkey#92L)) + +(24) CometExchange +Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Arguments: hashpartitioning(ps_suppkey#93L, ps_partkey#92L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=396] + +(25) CometSort +Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Arguments: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95], [ps_suppkey#93L ASC NULLS FIRST, ps_partkey#92L ASC NULLS FIRST] + +(26) CometSortMergeJoin +Left output [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] +Right output [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Arguments: [l_suppkey#18L, l_partkey#17L], [ps_suppkey#93L, ps_partkey#92L], Inner + +(27) CometProject +Input [10]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Arguments: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95], [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] + +(28) CometExchange +Input [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] +Arguments: hashpartitioning(l_orderkey#16L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=400] + +(29) CometSort +Input [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] +Arguments: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95], [l_orderkey#16L ASC NULLS FIRST] + +(30) CometNativeScan parquet +Output [2]: [o_orderkey#56L, o_orderdate#60] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/orders] +PushedFilters: [IsNotNull(o_orderkey)] +ReadSchema: struct + +(31) CometFilter +Input [2]: [o_orderkey#56L, o_orderdate#60] +Condition : isnotnull(o_orderkey#56L) + +(32) CometExchange +Input [2]: [o_orderkey#56L, o_orderdate#60] +Arguments: hashpartitioning(o_orderkey#56L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=404] + +(33) CometSort +Input [2]: [o_orderkey#56L, o_orderdate#60] +Arguments: [o_orderkey#56L, o_orderdate#60], [o_orderkey#56L ASC NULLS FIRST] + +(34) CometSortMergeJoin +Left output [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] +Right output [2]: [o_orderkey#56L, o_orderdate#60] +Arguments: [l_orderkey#16L], [o_orderkey#56L], Inner + +(35) CometProject +Input [8]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderkey#56L, o_orderdate#60] +Arguments: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60], [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60] + +(36) CometNativeScan parquet +Output [2]: [n_nationkey#48L, n_name#49] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/nation] +PushedFilters: [IsNotNull(n_nationkey)] +ReadSchema: struct + +(37) CometFilter +Input [2]: [n_nationkey#48L, n_name#49] +Condition : isnotnull(n_nationkey#48L) + +(38) CometBroadcastExchange +Input [2]: [n_nationkey#48L, n_name#49] +Arguments: [n_nationkey#48L, n_name#49] + +(39) CometBroadcastHashJoin +Left output [6]: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60] +Right output [2]: [n_nationkey#48L, n_name#49] +Arguments: [s_nationkey#111L], [n_nationkey#48L], Inner, BuildRight + +(40) CometProject +Input [8]: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60, n_nationkey#48L, n_name#49] +Arguments: [nation#122, o_year#123, amount#124], [n_name#49 AS nation#122, year(o_orderdate#60) AS o_year#123, ((l_extendedprice#21 * (1 - l_discount#22)) - (ps_supplycost#95 * l_quantity#20)) AS amount#124] + +(41) CometHashAggregate +Input [3]: [nation#122, o_year#123, amount#124] +Keys [2]: [nation#122, o_year#123] +Functions [1]: [partial_sum(amount#124)] + +(42) CometExchange +Input [4]: [nation#122, o_year#123, sum#132, isEmpty#133] +Arguments: hashpartitioning(nation#122, o_year#123, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=413] + +(43) CometHashAggregate +Input [4]: [nation#122, o_year#123, sum#132, isEmpty#133] +Keys [2]: [nation#122, o_year#123] +Functions [1]: [sum(amount#124)] + +(44) CometExchange +Input [3]: [nation#122, o_year#123, sum_profit#125] +Arguments: rangepartitioning(nation#122 ASC NULLS FIRST, o_year#123 DESC NULLS LAST, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=415] + +(45) CometSort +Input [3]: [nation#122, o_year#123, sum_profit#125] +Arguments: [nation#122, o_year#123, sum_profit#125], [nation#122 ASC NULLS FIRST, o_year#123 DESC NULLS LAST] + +(46) AdaptiveSparkPlan +Output [3]: [nation#122, o_year#123, sum_profit#125] +Arguments: isFinalPlan=false + + +Query 9 returned 175 rows, hash=fff4ba3023e74505f304fa3243cbeeb6 +Query 9 took 375.36 seconds + +Iteration 1 took 375.36 seconds + +Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/comet-offheap4g-q9-tpch-1775054750948.json diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.time b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.time new file mode 100644 index 0000000000..7aff1629d6 --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.time @@ -0,0 +1,6872 @@ +26/04/01 08:39:30 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) +26/04/01 08:39:30 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address +26/04/01 08:39:30 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +26/04/01 08:39:31 INFO SparkContext: Running Spark version 3.5.8 +26/04/01 08:39:31 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 08:39:31 INFO SparkContext: Java version 17.0.17 +26/04/01 08:39:31 INFO ResourceUtils: ============================================================== +26/04/01 08:39:31 INFO ResourceUtils: No custom resources configured for spark.driver. +26/04/01 08:39:31 INFO ResourceUtils: ============================================================== +26/04/01 08:39:31 INFO SparkContext: Submitted application: comet-offheap4g-q9 benchmark derived from tpch +26/04/01 08:39:31 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 4096, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) +26/04/01 08:39:31 INFO ResourceProfile: Limiting resource is cpu +26/04/01 08:39:31 INFO ResourceProfileManager: Added ResourceProfile id: 0 +26/04/01 08:39:31 INFO SecurityManager: Changing view acls to: andy +26/04/01 08:39:31 INFO SecurityManager: Changing modify acls to: andy +26/04/01 08:39:31 INFO SecurityManager: Changing view acls groups to: +26/04/01 08:39:31 INFO SecurityManager: Changing modify acls groups to: +26/04/01 08:39:31 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY +26/04/01 08:39:31 INFO Utils: Successfully started service 'sparkDriver' on port 59301. +26/04/01 08:39:31 INFO SparkEnv: Registering MapOutputTracker +26/04/01 08:39:31 INFO SparkEnv: Registering BlockManagerMaster +26/04/01 08:39:31 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information +26/04/01 08:39:31 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up +26/04/01 08:39:31 INFO SparkEnv: Registering BlockManagerMasterHeartbeat +26/04/01 08:39:31 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-bde69578-45f2-4d2d-97b9-bf70e14e1e93 +26/04/01 08:39:31 INFO MemoryStore: MemoryStore started with capacity 8.6 GiB +26/04/01 08:39:31 INFO SparkEnv: Registering OutputCommitCoordinator +26/04/01 08:39:31 INFO SparkContext: Added JAR file:///Users/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar at spark://10.0.0.133:59301/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775054371206 +26/04/01 08:39:31 INFO CometDriverPlugin: CometDriverPlugin init +26/04/01 08:39:31 INFO CometDriverPlugin: Setting spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions +26/04/01 08:39:31 INFO CometDriverPlugin: Comet metrics reporting is disabled. Set spark.comet.metrics.enabled=true to enable. +26/04/01 08:39:31 INFO CometDriverPlugin: Comet is running in unified memory mode and sharing off-heap memory with Spark +26/04/01 08:39:31 INFO DriverPluginContainer: Initialized driver component for plugin org.apache.spark.CometPlugin. +26/04/01 08:39:31 INFO Executor: Starting executor ID driver on host 10.0.0.133 +26/04/01 08:39:31 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 08:39:31 INFO Executor: Java version 17.0.17 +26/04/01 08:39:31 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' +26/04/01 08:39:31 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@5a3b465a for default. +26/04/01 08:39:31 INFO Executor: Fetching spark://10.0.0.133:59301/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775054371206 +26/04/01 08:39:31 INFO TransportClientFactory: Successfully created connection to /10.0.0.133:59301 after 8 ms (0 ms spent in bootstraps) +26/04/01 08:39:31 INFO Utils: Fetching spark://10.0.0.133:59301/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-cae9fc0e-ad8f-4143-bb25-84be77849f6a/userFiles-42cf6306-f997-4b97-99fd-212819ed4a9d/fetchFileTemp5223428367132204682.tmp +26/04/01 08:39:31 INFO Executor: Adding file:/private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-cae9fc0e-ad8f-4143-bb25-84be77849f6a/userFiles-42cf6306-f997-4b97-99fd-212819ed4a9d/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to class loader default +26/04/01 08:39:31 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 59303. +26/04/01 08:39:31 INFO NettyBlockTransferService: Server created on 10.0.0.133:59303 +26/04/01 08:39:31 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy +26/04/01 08:39:31 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 59303, None) +26/04/01 08:39:31 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:59303 with 8.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 59303, None) +26/04/01 08:39:31 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 59303, None) +26/04/01 08:39:31 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 59303, None) +26/04/01 08:39:31 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. +26/04/01 08:39:31 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. +26/04/01 08:39:31 INFO InMemoryFileIndex: It took 17 ms to list leaf files for 1 paths. +26/04/01 08:39:32 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:39:32 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:39:32 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:39:32 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:39:32 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:32 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:39:32 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 08:39:32 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 08:39:32 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:32 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:32 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:39:32 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 +26/04/01 08:39:32 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 08:39:32 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) +26/04/01 08:39:32 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 2041 bytes result sent to driver +26/04/01 08:39:32 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 127 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:39:32 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool +26/04/01 08:39:32 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.400 s +26/04/01 08:39:32 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:39:32 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished +26/04/01 08:39:32 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.411248 s +26/04/01 08:39:35 INFO NativeBase: Couldn't locate log file from either COMET_CONF_DIR or comet.log.file.path. Using default log configuration with INFO log level which emits to stderr +26/04/01 08:39:35 INFO core/src/lib.rs: Comet native library version 0.15.0 initialized +26/04/01 08:39:35 INFO NativeBase: Setting system property arrow.enable_unsafe_memory_access to true +26/04/01 08:39:35 INFO NativeBase: Setting system property arrow.enable_null_check_for_get to false +26/04/01 08:39:35 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. +26/04/01 08:39:35 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:39:35 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:39:35 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:39:35 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:39:35 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:35 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:39:35 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 08:39:35 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 08:39:35 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:35 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:35 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:39:35 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 +26/04/01 08:39:35 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 08:39:35 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) +26/04/01 08:39:35 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2344 bytes result sent to driver +26/04/01 08:39:35 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 17 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:39:35 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool +26/04/01 08:39:35 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.025 s +26/04/01 08:39:35 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:39:35 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished +26/04/01 08:39:35 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.027290 s +26/04/01 08:39:35 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 08:39:35 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:39:35 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:39:35 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:39:35 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:39:35 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:35 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:39:35 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 08:39:35 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 08:39:35 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:35 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:35 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:39:35 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 +26/04/01 08:39:35 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 08:39:35 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) +26/04/01 08:39:35 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver +26/04/01 08:39:35 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 7 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:39:35 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool +26/04/01 08:39:35 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s +26/04/01 08:39:35 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:39:35 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished +26/04/01 08:39:35 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.015439 s +26/04/01 08:39:35 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. +26/04/01 08:39:35 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:39:35 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:39:35 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:39:35 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:39:35 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:35 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:39:35 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 08:39:35 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 08:39:35 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:35 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:35 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:39:35 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 +26/04/01 08:39:35 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 08:39:35 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) +26/04/01 08:39:35 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver +26/04/01 08:39:35 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 7 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:39:35 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool +26/04/01 08:39:35 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s +26/04/01 08:39:35 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:39:35 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished +26/04/01 08:39:35 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.015823 s +26/04/01 08:39:35 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. +26/04/01 08:39:35 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:39:35 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:39:35 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:39:35 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:39:35 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:35 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:39:35 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 08:39:35 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 08:39:35 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:35 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:35 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:39:35 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 +26/04/01 08:39:35 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9299 bytes) +26/04/01 08:39:35 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) +26/04/01 08:39:35 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver +26/04/01 08:39:35 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:39:35 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool +26/04/01 08:39:35 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s +26/04/01 08:39:35 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:39:35 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished +26/04/01 08:39:35 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.014553 s +26/04/01 08:39:35 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 08:39:35 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:39:35 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:39:35 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:39:35 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:39:35 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:35 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:39:35 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 08:39:35 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 08:39:35 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:35 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:35 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:39:35 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 +26/04/01 08:39:35 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 08:39:35 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) +26/04/01 08:39:35 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver +26/04/01 08:39:35 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:39:35 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool +26/04/01 08:39:35 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 08:39:35 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:39:35 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished +26/04/01 08:39:35 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.014503 s +26/04/01 08:39:35 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. +26/04/01 08:39:35 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:39:35 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:39:35 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:39:35 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:39:35 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:35 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:39:35 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 08:39:35 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 08:39:35 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:35 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:35 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:39:35 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 +26/04/01 08:39:35 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 08:39:35 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) +26/04/01 08:39:35 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver +26/04/01 08:39:35 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:39:35 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool +26/04/01 08:39:35 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.011 s +26/04/01 08:39:35 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:39:35 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished +26/04/01 08:39:35 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.012746 s +26/04/01 08:39:35 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 08:39:35 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:39:35 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:39:35 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:39:35 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:39:35 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:35 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:39:35 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) +26/04/01 08:39:35 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) +26/04/01 08:39:35 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:35 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:35 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:39:35 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 +26/04/01 08:39:35 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 08:39:35 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) +26/04/01 08:39:35 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver +26/04/01 08:39:35 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:39:35 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool +26/04/01 08:39:35 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.012 s +26/04/01 08:39:35 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:39:35 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished +26/04/01 08:39:35 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.013823 s +26/04/01 08:39:35 INFO FileSourceStrategy: Pushed Filters: IsNotNull(p_name),StringContains(p_name,moccasin),IsNotNull(p_partkey) +26/04/01 08:39:35 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(p_name#75),Contains(p_name#75, moccasin),isnotnull(p_partkey#74L) +26/04/01 08:39:35 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_partkey),IsNotNull(l_suppkey),IsNotNull(l_orderkey) +26/04/01 08:39:35 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_partkey#17L),isnotnull(l_suppkey#18L),isnotnull(l_orderkey#16L) +26/04/01 08:39:35 INFO FileSourceStrategy: Pushed Filters: IsNotNull(s_suppkey),IsNotNull(s_nationkey) +26/04/01 08:39:35 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(s_suppkey#108L),isnotnull(s_nationkey#111L) +26/04/01 08:39:35 INFO FileSourceStrategy: Pushed Filters: IsNotNull(ps_suppkey),IsNotNull(ps_partkey) +26/04/01 08:39:35 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(ps_suppkey#93L),isnotnull(ps_partkey#92L) +26/04/01 08:39:35 INFO FileSourceStrategy: Pushed Filters: IsNotNull(o_orderkey) +26/04/01 08:39:35 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(o_orderkey#56L) +26/04/01 08:39:35 INFO FileSourceStrategy: Pushed Filters: IsNotNull(n_nationkey) +26/04/01 08:39:35 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(n_nationkey#48L) +26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) +26/04/01 08:39:35 INFO CometScanExec: Planning scan with bin packing, max size: 102898790 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:35 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:35 INFO CometScanExec: Planning scan with bin packing, max size: 27960063 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:35 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:35 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:35 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:36 INFO CometScanExec: Planning scan with bin packing, max size: 102898790 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:36 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:59303 (size: 35.4 KiB, free: 8.6 GiB) +26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:59303 (size: 35.4 KiB, free: 8.6 GiB) +26/04/01 08:39:36 INFO SparkContext: Created broadcast 9 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:39:36 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:39:36 INFO FileSourceScanExec: Planning scan with bin packing, max size: 102898790 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:36 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:36 INFO DAGScheduler: Registering RDD 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 +26/04/01 08:39:36 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:36 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 8 output partitions +26/04/01 08:39:36 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:39:36 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:39:36 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:36 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[21] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:39:36 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 20.5 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 8.1 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:59303 (size: 8.1 KiB, free: 8.6 GiB) +26/04/01 08:39:36 INFO SparkContext: Created broadcast 11 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:36 INFO DAGScheduler: Submitting 8 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[21] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7)) +26/04/01 08:39:36 INFO TaskSchedulerImpl: Adding task set 8.0 with 8 tasks resource profile 0 +26/04/01 08:39:36 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 08:39:36 INFO DAGScheduler: Final stage: ResultStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:39:36 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:39:36 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:36 INFO DAGScheduler: Submitting ResultStage 9 (MapPartitionsRDD[20] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9429 bytes) +26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:59303 (size: 35.4 KiB, free: 8.6 GiB) +26/04/01 08:39:36 INFO SparkContext: Created broadcast 10 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:39:36 INFO TaskSetManager: Starting task 1.0 in stage 8.0 (TID 9) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9431 bytes) +26/04/01 08:39:36 INFO TaskSetManager: Starting task 2.0 in stage 8.0 (TID 10) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9431 bytes) +26/04/01 08:39:36 INFO TaskSetManager: Starting task 3.0 in stage 8.0 (TID 11) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9431 bytes) +26/04/01 08:39:36 INFO TaskSetManager: Starting task 4.0 in stage 8.0 (TID 12) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9430 bytes) +26/04/01 08:39:36 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 10.8 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO TaskSetManager: Starting task 5.0 in stage 8.0 (TID 13) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9432 bytes) +26/04/01 08:39:36 INFO TaskSetManager: Starting task 6.0 in stage 8.0 (TID 14) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9431 bytes) +26/04/01 08:39:36 INFO TaskSetManager: Starting task 7.0 in stage 8.0 (TID 15) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9432 bytes) +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 5.3 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) +26/04/01 08:39:36 INFO Executor: Running task 1.0 in stage 8.0 (TID 9) +26/04/01 08:39:36 INFO Executor: Running task 2.0 in stage 8.0 (TID 10) +26/04/01 08:39:36 INFO Executor: Running task 3.0 in stage 8.0 (TID 11) +26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:59303 (size: 5.3 KiB, free: 8.6 GiB) +26/04/01 08:39:36 INFO Executor: Running task 4.0 in stage 8.0 (TID 12) +26/04/01 08:39:36 INFO Executor: Running task 5.0 in stage 8.0 (TID 13) +26/04/01 08:39:36 INFO SparkContext: Created broadcast 12 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:36 INFO Executor: Running task 6.0 in stage 8.0 (TID 14) +26/04/01 08:39:36 INFO Executor: Running task 7.0 in stage 8.0 (TID 15) +26/04/01 08:39:36 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 9 (MapPartitionsRDD[20] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:39:36 INFO TaskSchedulerImpl: Adding task set 9.0 with 1 tasks resource profile 0 +26/04/01 08:39:36 INFO DAGScheduler: Registering RDD 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 +26/04/01 08:39:36 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions +26/04/01 08:39:36 INFO DAGScheduler: Final stage: ShuffleMapStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:39:36 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:39:36 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:36 INFO CometScanExec: Planning scan with bin packing, max size: 27960063 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:36 INFO DAGScheduler: Submitting ShuffleMapStage 10 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_13 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_14 stored as values in memory (estimated size 20.6 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_14_piece0 stored as bytes in memory (estimated size 8.2 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_14_piece0 in memory on 10.0.0.133:59303 (size: 8.2 KiB, free: 8.6 GiB) +26/04/01 08:39:36 INFO SparkContext: Created broadcast 14 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:36 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 10 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:39:36 INFO TaskSchedulerImpl: Adding task set 10.0 with 208 tasks resource profile 0 +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_13_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_13_piece0 in memory on 10.0.0.133:59303 (size: 35.4 KiB, free: 8.6 GiB) +26/04/01 08:39:36 INFO SparkContext: Created broadcast 13 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:39:36 INFO FileSourceScanExec: Planning scan with bin packing, max size: 27960063 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:36 INFO DAGScheduler: Registering RDD 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 2 +26/04/01 08:39:36 INFO DAGScheduler: Got map stage job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 8 output partitions +26/04/01 08:39:36 INFO DAGScheduler: Final stage: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:39:36 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:39:36 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:36 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:36 INFO DAGScheduler: Submitting ShuffleMapStage 11 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_15 stored as values in memory (estimated size 18.0 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_15_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_15_piece0 in memory on 10.0.0.133:59303 (size: 7.9 KiB, free: 8.6 GiB) +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_16 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO SparkContext: Created broadcast 15 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:36 INFO DAGScheduler: Submitting 8 missing tasks from ShuffleMapStage 11 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7)) +26/04/01 08:39:36 INFO TaskSchedulerImpl: Adding task set 11.0 with 8 tasks resource profile 0 +26/04/01 08:39:36 INFO BaseAllocator: Debug mode disabled. Enable with the VM option -Darrow.memory.debug.allocator=true. +26/04/01 08:39:36 INFO DefaultAllocationManagerOption: allocation manager type not specified, using netty as the default type +26/04/01 08:39:36 INFO CheckAllocator: Using DefaultAllocationManager at memory/unsafe/DefaultAllocationManagerFactory.class +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_16_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_16_piece0 in memory on 10.0.0.133:59303 (size: 35.4 KiB, free: 8.6 GiB) +26/04/01 08:39:36 INFO SparkContext: Created broadcast 16 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:39:36 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:36 INFO DAGScheduler: Registering RDD 30 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 3 +26/04/01 08:39:36 INFO DAGScheduler: Got map stage job 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 37 output partitions +26/04/01 08:39:36 INFO DAGScheduler: Final stage: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:39:36 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:39:36 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:36 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:36 INFO DAGScheduler: Submitting ShuffleMapStage 12 (MapPartitionsRDD[30] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_17 stored as values in memory (estimated size 18.2 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_17_piece0 stored as bytes in memory (estimated size 8.0 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_17_piece0 in memory on 10.0.0.133:59303 (size: 8.0 KiB, free: 8.6 GiB) +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_18 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:36 INFO SparkContext: Created broadcast 17 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:36 INFO DAGScheduler: Submitting 37 missing tasks from ShuffleMapStage 12 (MapPartitionsRDD[30] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:39:36 INFO TaskSchedulerImpl: Adding task set 12.0 with 37 tasks resource profile 0 +26/04/01 08:39:36 INFO core/src/execution/jni_api.rs: Comet tokio runtime: using spark.executor.cores=8 worker threads +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_18_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_18_piece0 in memory on 10.0.0.133:59303 (size: 35.4 KiB, free: 8.6 GiB) +26/04/01 08:39:36 INFO SparkContext: Created broadcast 18 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:39:36 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:39:36 INFO DAGScheduler: Registering RDD 33 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 4 +26/04/01 08:39:36 INFO DAGScheduler: Got map stage job 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 64 output partitions +26/04/01 08:39:36 INFO DAGScheduler: Final stage: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:39:36 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:39:36 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:36 INFO DAGScheduler: Submitting ShuffleMapStage 13 (MapPartitionsRDD[33] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_19 stored as values in memory (estimated size 16.6 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO MemoryStore: Block broadcast_19_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 8.6 GiB) +26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_19_piece0 in memory on 10.0.0.133:59303 (size: 7.9 KiB, free: 8.6 GiB) +26/04/01 08:39:36 INFO SparkContext: Created broadcast 19 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:36 INFO DAGScheduler: Submitting 64 missing tasks from ShuffleMapStage 13 (MapPartitionsRDD[33] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:39:36 INFO TaskSchedulerImpl: Adding task set 13.0 with 64 tasks resource profile 0 +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO Executor: Finished task 7.0 in stage 8.0 (TID 15). 6761 bytes result sent to driver +26/04/01 08:39:38 INFO Executor: Finished task 3.0 in stage 8.0 (TID 11). 6761 bytes result sent to driver +26/04/01 08:39:38 INFO Executor: Finished task 5.0 in stage 8.0 (TID 13). 6761 bytes result sent to driver +26/04/01 08:39:38 INFO Executor: Finished task 1.0 in stage 8.0 (TID 9). 6761 bytes result sent to driver +26/04/01 08:39:38 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 6761 bytes result sent to driver +26/04/01 08:39:38 INFO TaskSetManager: Starting task 0.0 in stage 9.0 (TID 16) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:39:38 INFO Executor: Running task 0.0 in stage 9.0 (TID 16) +26/04/01 08:39:38 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 17) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:39:38 INFO Executor: Running task 0.0 in stage 10.0 (TID 17) +26/04/01 08:39:38 INFO TaskSetManager: Starting task 1.0 in stage 10.0 (TID 18) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:39:38 INFO TaskSetManager: Finished task 7.0 in stage 8.0 (TID 15) in 2439 ms on 10.0.0.133 (executor driver) (1/8) +26/04/01 08:39:38 INFO TaskSetManager: Finished task 1.0 in stage 8.0 (TID 9) in 2440 ms on 10.0.0.133 (executor driver) (2/8) +26/04/01 08:39:38 INFO Executor: Finished task 4.0 in stage 8.0 (TID 12). 6761 bytes result sent to driver +26/04/01 08:39:38 INFO Executor: Running task 1.0 in stage 10.0 (TID 18) +26/04/01 08:39:38 INFO Executor: Finished task 2.0 in stage 8.0 (TID 10). 6761 bytes result sent to driver +26/04/01 08:39:38 INFO TaskSetManager: Starting task 2.0 in stage 10.0 (TID 19) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:39:38 INFO Executor: Finished task 6.0 in stage 8.0 (TID 14). 6761 bytes result sent to driver +26/04/01 08:39:38 INFO Executor: Running task 2.0 in stage 10.0 (TID 19) +26/04/01 08:39:38 INFO TaskSetManager: Finished task 3.0 in stage 8.0 (TID 11) in 2441 ms on 10.0.0.133 (executor driver) (3/8) +26/04/01 08:39:38 INFO TaskSetManager: Starting task 3.0 in stage 10.0 (TID 20) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:39:38 INFO Executor: Running task 3.0 in stage 10.0 (TID 20) +26/04/01 08:39:38 INFO TaskSetManager: Starting task 4.0 in stage 10.0 (TID 21) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:39:38 INFO TaskSetManager: Finished task 5.0 in stage 8.0 (TID 13) in 2442 ms on 10.0.0.133 (executor driver) (4/8) +26/04/01 08:39:38 INFO Executor: Running task 4.0 in stage 10.0 (TID 21) +26/04/01 08:39:38 INFO TaskSetManager: Starting task 5.0 in stage 10.0 (TID 22) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:39:38 INFO Executor: Running task 5.0 in stage 10.0 (TID 22) +26/04/01 08:39:38 INFO TaskSetManager: Starting task 6.0 in stage 10.0 (TID 23) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:39:38 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 2443 ms on 10.0.0.133 (executor driver) (5/8) +26/04/01 08:39:38 INFO Executor: Running task 6.0 in stage 10.0 (TID 23) +26/04/01 08:39:38 INFO TaskSetManager: Finished task 4.0 in stage 8.0 (TID 12) in 2443 ms on 10.0.0.133 (executor driver) (6/8) +26/04/01 08:39:38 INFO TaskSetManager: Finished task 2.0 in stage 8.0 (TID 10) in 2443 ms on 10.0.0.133 (executor driver) (7/8) +26/04/01 08:39:38 INFO TaskSetManager: Finished task 6.0 in stage 8.0 (TID 14) in 2443 ms on 10.0.0.133 (executor driver) (8/8) +26/04/01 08:39:38 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool +26/04/01 08:39:38 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.453 s +26/04/01 08:39:38 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:39:38 INFO DAGScheduler: running: Set(ShuffleMapStage 12, ResultStage 9, ShuffleMapStage 13, ShuffleMapStage 10, ShuffleMapStage 11) +26/04/01 08:39:38 INFO DAGScheduler: waiting: Set() +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO DAGScheduler: failed: Set() +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO ShufflePartitionsUtil: For shuffle(0), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 08:39:38 INFO Executor: Finished task 0.0 in stage 9.0 (TID 16). 4762 bytes result sent to driver +26/04/01 08:39:38 INFO TaskSetManager: Starting task 7.0 in stage 10.0 (TID 24) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:39:38 INFO Executor: Running task 7.0 in stage 10.0 (TID 24) +26/04/01 08:39:38 INFO TaskSetManager: Finished task 0.0 in stage 9.0 (TID 16) in 40 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:39:38 INFO TaskSchedulerImpl: Removed TaskSet 9.0, whose tasks have all completed, from pool +26/04/01 08:39:38 INFO DAGScheduler: ResultStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.479 s +26/04/01 08:39:38 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:39:38 INFO TaskSchedulerImpl: Killing all running tasks in stage 9: Stage finished +26/04/01 08:39:38 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 2.487185 s +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:38 INFO Utils: Coalesced 1 broadcast batches into 1 (25 rows) +26/04/01 08:39:38 INFO MemoryStore: Block broadcast_20 stored as values in memory (estimated size 880.0 B, free 8.6 GiB) +26/04/01 08:39:38 INFO MemoryStore: Block broadcast_20_piece0 stored as bytes in memory (estimated size 831.0 B, free 8.6 GiB) +26/04/01 08:39:38 INFO BlockManagerInfo: Added broadcast_20_piece0 in memory on 10.0.0.133:59303 (size: 831.0 B, free: 8.6 GiB) +26/04/01 08:39:38 INFO SparkContext: Created broadcast 20 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:39:38 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:39:38 INFO DAGScheduler: Got job 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions +26/04/01 08:39:38 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:39:38 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 14) +26/04/01 08:39:38 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:38 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[35] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:39:38 INFO MemoryStore: Block broadcast_21 stored as values in memory (estimated size 14.5 KiB, free 8.6 GiB) +26/04/01 08:39:38 INFO MemoryStore: Block broadcast_21_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 8.6 GiB) +26/04/01 08:39:38 INFO BlockManagerInfo: Added broadcast_21_piece0 in memory on 10.0.0.133:59303 (size: 7.9 KiB, free: 8.6 GiB) +26/04/01 08:39:38 INFO SparkContext: Created broadcast 21 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:38 INFO DAGScheduler: Submitting 200 missing tasks from ResultStage 15 (MapPartitionsRDD[35] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:39:38 INFO TaskSchedulerImpl: Adding task set 15.0 with 200 tasks resource profile 0 +26/04/01 08:39:46 INFO Executor: Finished task 4.0 in stage 10.0 (TID 21). 6649 bytes result sent to driver +26/04/01 08:39:46 INFO TaskSetManager: Starting task 8.0 in stage 10.0 (TID 25) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:39:46 INFO Executor: Finished task 3.0 in stage 10.0 (TID 20). 6649 bytes result sent to driver +26/04/01 08:39:46 INFO Executor: Running task 8.0 in stage 10.0 (TID 25) +26/04/01 08:39:46 INFO TaskSetManager: Finished task 4.0 in stage 10.0 (TID 21) in 7994 ms on 10.0.0.133 (executor driver) (1/208) +26/04/01 08:39:46 INFO TaskSetManager: Starting task 9.0 in stage 10.0 (TID 26) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:39:46 INFO TaskSetManager: Finished task 3.0 in stage 10.0 (TID 20) in 7995 ms on 10.0.0.133 (executor driver) (2/208) +26/04/01 08:39:46 INFO Executor: Running task 9.0 in stage 10.0 (TID 26) +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:46 INFO Executor: Finished task 0.0 in stage 10.0 (TID 17). 6649 bytes result sent to driver +26/04/01 08:39:46 INFO TaskSetManager: Starting task 10.0 in stage 10.0 (TID 27) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:46 INFO Executor: Running task 10.0 in stage 10.0 (TID 27) +26/04/01 08:39:46 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 17) in 8001 ms on 10.0.0.133 (executor driver) (3/208) +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:46 INFO Executor: Finished task 1.0 in stage 10.0 (TID 18). 6649 bytes result sent to driver +26/04/01 08:39:46 INFO TaskSetManager: Starting task 11.0 in stage 10.0 (TID 28) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:39:46 INFO Executor: Running task 11.0 in stage 10.0 (TID 28) +26/04/01 08:39:46 INFO TaskSetManager: Finished task 1.0 in stage 10.0 (TID 18) in 8005 ms on 10.0.0.133 (executor driver) (4/208) +26/04/01 08:39:46 INFO Executor: Finished task 5.0 in stage 10.0 (TID 22). 6649 bytes result sent to driver +26/04/01 08:39:46 INFO TaskSetManager: Starting task 12.0 in stage 10.0 (TID 29) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:39:46 INFO Executor: Running task 12.0 in stage 10.0 (TID 29) +26/04/01 08:39:46 INFO TaskSetManager: Finished task 5.0 in stage 10.0 (TID 22) in 8004 ms on 10.0.0.133 (executor driver) (5/208) +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:46 INFO Executor: Finished task 6.0 in stage 10.0 (TID 23). 6649 bytes result sent to driver +26/04/01 08:39:46 INFO TaskSetManager: Starting task 13.0 in stage 10.0 (TID 30) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:39:46 INFO TaskSetManager: Finished task 6.0 in stage 10.0 (TID 23) in 8019 ms on 10.0.0.133 (executor driver) (6/208) +26/04/01 08:39:46 INFO Executor: Running task 13.0 in stage 10.0 (TID 30) +26/04/01 08:39:46 INFO Executor: Finished task 2.0 in stage 10.0 (TID 19). 6649 bytes result sent to driver +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:46 INFO TaskSetManager: Starting task 14.0 in stage 10.0 (TID 31) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:39:46 INFO TaskSetManager: Finished task 2.0 in stage 10.0 (TID 19) in 8023 ms on 10.0.0.133 (executor driver) (7/208) +26/04/01 08:39:46 INFO Executor: Running task 14.0 in stage 10.0 (TID 31) +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:46 INFO Executor: Finished task 7.0 in stage 10.0 (TID 24). 6649 bytes result sent to driver +26/04/01 08:39:46 INFO TaskSetManager: Starting task 15.0 in stage 10.0 (TID 32) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:39:46 INFO Executor: Running task 15.0 in stage 10.0 (TID 32) +26/04/01 08:39:46 INFO TaskSetManager: Finished task 7.0 in stage 10.0 (TID 24) in 7990 ms on 10.0.0.133 (executor driver) (8/208) +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO Executor: Finished task 9.0 in stage 10.0 (TID 26). 6606 bytes result sent to driver +26/04/01 08:39:54 INFO TaskSetManager: Starting task 16.0 in stage 10.0 (TID 33) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:39:54 INFO TaskSetManager: Finished task 9.0 in stage 10.0 (TID 26) in 7941 ms on 10.0.0.133 (executor driver) (9/208) +26/04/01 08:39:54 INFO Executor: Running task 16.0 in stage 10.0 (TID 33) +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO Executor: Finished task 12.0 in stage 10.0 (TID 29). 6563 bytes result sent to driver +26/04/01 08:39:54 INFO TaskSetManager: Starting task 17.0 in stage 10.0 (TID 34) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:39:54 INFO Executor: Running task 17.0 in stage 10.0 (TID 34) +26/04/01 08:39:54 INFO TaskSetManager: Finished task 12.0 in stage 10.0 (TID 29) in 7947 ms on 10.0.0.133 (executor driver) (10/208) +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO Executor: Finished task 10.0 in stage 10.0 (TID 27). 6606 bytes result sent to driver +26/04/01 08:39:54 INFO TaskSetManager: Starting task 18.0 in stage 10.0 (TID 35) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:39:54 INFO TaskSetManager: Finished task 10.0 in stage 10.0 (TID 27) in 7959 ms on 10.0.0.133 (executor driver) (11/208) +26/04/01 08:39:54 INFO Executor: Running task 18.0 in stage 10.0 (TID 35) +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO Executor: Finished task 15.0 in stage 10.0 (TID 32). 6606 bytes result sent to driver +26/04/01 08:39:54 INFO TaskSetManager: Starting task 19.0 in stage 10.0 (TID 36) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:39:54 INFO TaskSetManager: Finished task 15.0 in stage 10.0 (TID 32) in 7934 ms on 10.0.0.133 (executor driver) (12/208) +26/04/01 08:39:54 INFO Executor: Running task 19.0 in stage 10.0 (TID 36) +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO Executor: Finished task 14.0 in stage 10.0 (TID 31). 6563 bytes result sent to driver +26/04/01 08:39:54 INFO TaskSetManager: Starting task 20.0 in stage 10.0 (TID 37) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:39:54 INFO Executor: Running task 20.0 in stage 10.0 (TID 37) +26/04/01 08:39:54 INFO TaskSetManager: Finished task 14.0 in stage 10.0 (TID 31) in 7956 ms on 10.0.0.133 (executor driver) (13/208) +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO Executor: Finished task 8.0 in stage 10.0 (TID 25). 6606 bytes result sent to driver +26/04/01 08:39:54 INFO TaskSetManager: Starting task 21.0 in stage 10.0 (TID 38) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:39:54 INFO TaskSetManager: Finished task 8.0 in stage 10.0 (TID 25) in 7986 ms on 10.0.0.133 (executor driver) (14/208) +26/04/01 08:39:54 INFO Executor: Running task 21.0 in stage 10.0 (TID 38) +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO Executor: Finished task 13.0 in stage 10.0 (TID 30). 6606 bytes result sent to driver +26/04/01 08:39:54 INFO TaskSetManager: Starting task 22.0 in stage 10.0 (TID 39) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:39:54 INFO TaskSetManager: Finished task 13.0 in stage 10.0 (TID 30) in 7979 ms on 10.0.0.133 (executor driver) (15/208) +26/04/01 08:39:54 INFO Executor: Running task 22.0 in stage 10.0 (TID 39) +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO Executor: Finished task 11.0 in stage 10.0 (TID 28). 6606 bytes result sent to driver +26/04/01 08:39:54 INFO TaskSetManager: Starting task 23.0 in stage 10.0 (TID 40) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:39:54 INFO Executor: Running task 23.0 in stage 10.0 (TID 40) +26/04/01 08:39:54 INFO TaskSetManager: Finished task 11.0 in stage 10.0 (TID 28) in 8018 ms on 10.0.0.133 (executor driver) (16/208) +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO Executor: Finished task 16.0 in stage 10.0 (TID 33). 6563 bytes result sent to driver +26/04/01 08:40:02 INFO TaskSetManager: Starting task 24.0 in stage 10.0 (TID 41) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:40:02 INFO TaskSetManager: Finished task 16.0 in stage 10.0 (TID 33) in 7950 ms on 10.0.0.133 (executor driver) (17/208) +26/04/01 08:40:02 INFO Executor: Running task 24.0 in stage 10.0 (TID 41) +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO Executor: Finished task 17.0 in stage 10.0 (TID 34). 6563 bytes result sent to driver +26/04/01 08:40:02 INFO TaskSetManager: Starting task 25.0 in stage 10.0 (TID 42) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:02 INFO TaskSetManager: Finished task 17.0 in stage 10.0 (TID 34) in 7958 ms on 10.0.0.133 (executor driver) (18/208) +26/04/01 08:40:02 INFO Executor: Running task 25.0 in stage 10.0 (TID 42) +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO Executor: Finished task 18.0 in stage 10.0 (TID 35). 6563 bytes result sent to driver +26/04/01 08:40:02 INFO TaskSetManager: Starting task 26.0 in stage 10.0 (TID 43) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:02 INFO TaskSetManager: Finished task 18.0 in stage 10.0 (TID 35) in 7960 ms on 10.0.0.133 (executor driver) (19/208) +26/04/01 08:40:02 INFO Executor: Running task 26.0 in stage 10.0 (TID 43) +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO Executor: Finished task 19.0 in stage 10.0 (TID 36). 6563 bytes result sent to driver +26/04/01 08:40:02 INFO TaskSetManager: Starting task 27.0 in stage 10.0 (TID 44) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:02 INFO Executor: Running task 27.0 in stage 10.0 (TID 44) +26/04/01 08:40:02 INFO TaskSetManager: Finished task 19.0 in stage 10.0 (TID 36) in 7960 ms on 10.0.0.133 (executor driver) (20/208) +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO Executor: Finished task 21.0 in stage 10.0 (TID 38). 6563 bytes result sent to driver +26/04/01 08:40:02 INFO TaskSetManager: Starting task 28.0 in stage 10.0 (TID 45) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:02 INFO Executor: Finished task 20.0 in stage 10.0 (TID 37). 6563 bytes result sent to driver +26/04/01 08:40:02 INFO Executor: Running task 28.0 in stage 10.0 (TID 45) +26/04/01 08:40:02 INFO TaskSetManager: Finished task 21.0 in stage 10.0 (TID 38) in 7975 ms on 10.0.0.133 (executor driver) (21/208) +26/04/01 08:40:02 INFO TaskSetManager: Starting task 29.0 in stage 10.0 (TID 46) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:02 INFO TaskSetManager: Finished task 20.0 in stage 10.0 (TID 37) in 7977 ms on 10.0.0.133 (executor driver) (22/208) +26/04/01 08:40:02 INFO Executor: Running task 29.0 in stage 10.0 (TID 46) +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO Executor: Finished task 22.0 in stage 10.0 (TID 39). 6563 bytes result sent to driver +26/04/01 08:40:02 INFO TaskSetManager: Starting task 30.0 in stage 10.0 (TID 47) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:40:02 INFO TaskSetManager: Finished task 22.0 in stage 10.0 (TID 39) in 7976 ms on 10.0.0.133 (executor driver) (23/208) +26/04/01 08:40:02 INFO Executor: Running task 30.0 in stage 10.0 (TID 47) +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO Executor: Finished task 23.0 in stage 10.0 (TID 40). 6563 bytes result sent to driver +26/04/01 08:40:02 INFO TaskSetManager: Starting task 31.0 in stage 10.0 (TID 48) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:02 INFO Executor: Running task 31.0 in stage 10.0 (TID 48) +26/04/01 08:40:02 INFO TaskSetManager: Finished task 23.0 in stage 10.0 (TID 40) in 7970 ms on 10.0.0.133 (executor driver) (24/208) +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO Executor: Finished task 25.0 in stage 10.0 (TID 42). 6563 bytes result sent to driver +26/04/01 08:40:10 INFO TaskSetManager: Starting task 32.0 in stage 10.0 (TID 49) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:10 INFO Executor: Running task 32.0 in stage 10.0 (TID 49) +26/04/01 08:40:10 INFO TaskSetManager: Finished task 25.0 in stage 10.0 (TID 42) in 7860 ms on 10.0.0.133 (executor driver) (25/208) +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO Executor: Finished task 24.0 in stage 10.0 (TID 41). 6563 bytes result sent to driver +26/04/01 08:40:10 INFO TaskSetManager: Starting task 33.0 in stage 10.0 (TID 50) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:10 INFO Executor: Running task 33.0 in stage 10.0 (TID 50) +26/04/01 08:40:10 INFO TaskSetManager: Finished task 24.0 in stage 10.0 (TID 41) in 7963 ms on 10.0.0.133 (executor driver) (26/208) +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO Executor: Finished task 27.0 in stage 10.0 (TID 44). 6563 bytes result sent to driver +26/04/01 08:40:10 INFO TaskSetManager: Starting task 34.0 in stage 10.0 (TID 51) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:10 INFO Executor: Running task 34.0 in stage 10.0 (TID 51) +26/04/01 08:40:10 INFO TaskSetManager: Finished task 27.0 in stage 10.0 (TID 44) in 7938 ms on 10.0.0.133 (executor driver) (27/208) +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO Executor: Finished task 28.0 in stage 10.0 (TID 45). 6563 bytes result sent to driver +26/04/01 08:40:10 INFO TaskSetManager: Starting task 35.0 in stage 10.0 (TID 52) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:10 INFO TaskSetManager: Finished task 28.0 in stage 10.0 (TID 45) in 7921 ms on 10.0.0.133 (executor driver) (28/208) +26/04/01 08:40:10 INFO Executor: Running task 35.0 in stage 10.0 (TID 52) +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO Executor: Finished task 26.0 in stage 10.0 (TID 43). 6563 bytes result sent to driver +26/04/01 08:40:10 INFO TaskSetManager: Starting task 36.0 in stage 10.0 (TID 53) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:40:10 INFO Executor: Running task 36.0 in stage 10.0 (TID 53) +26/04/01 08:40:10 INFO TaskSetManager: Finished task 26.0 in stage 10.0 (TID 43) in 7971 ms on 10.0.0.133 (executor driver) (29/208) +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO Executor: Finished task 29.0 in stage 10.0 (TID 46). 6563 bytes result sent to driver +26/04/01 08:40:10 INFO TaskSetManager: Starting task 37.0 in stage 10.0 (TID 54) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:10 INFO TaskSetManager: Finished task 29.0 in stage 10.0 (TID 46) in 7942 ms on 10.0.0.133 (executor driver) (30/208) +26/04/01 08:40:10 INFO Executor: Running task 37.0 in stage 10.0 (TID 54) +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO Executor: Finished task 30.0 in stage 10.0 (TID 47). 6563 bytes result sent to driver +26/04/01 08:40:10 INFO TaskSetManager: Starting task 38.0 in stage 10.0 (TID 55) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:10 INFO Executor: Running task 38.0 in stage 10.0 (TID 55) +26/04/01 08:40:10 INFO TaskSetManager: Finished task 30.0 in stage 10.0 (TID 47) in 7955 ms on 10.0.0.133 (executor driver) (31/208) +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO Executor: Finished task 31.0 in stage 10.0 (TID 48). 6563 bytes result sent to driver +26/04/01 08:40:10 INFO TaskSetManager: Starting task 39.0 in stage 10.0 (TID 56) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:10 INFO Executor: Running task 39.0 in stage 10.0 (TID 56) +26/04/01 08:40:10 INFO TaskSetManager: Finished task 31.0 in stage 10.0 (TID 48) in 7951 ms on 10.0.0.133 (executor driver) (32/208) +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO Executor: Finished task 32.0 in stage 10.0 (TID 49). 6606 bytes result sent to driver +26/04/01 08:40:18 INFO TaskSetManager: Starting task 40.0 in stage 10.0 (TID 57) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:18 INFO Executor: Running task 40.0 in stage 10.0 (TID 57) +26/04/01 08:40:18 INFO TaskSetManager: Finished task 32.0 in stage 10.0 (TID 49) in 7967 ms on 10.0.0.133 (executor driver) (33/208) +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO Executor: Finished task 34.0 in stage 10.0 (TID 51). 6606 bytes result sent to driver +26/04/01 08:40:18 INFO TaskSetManager: Starting task 41.0 in stage 10.0 (TID 58) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:18 INFO TaskSetManager: Finished task 34.0 in stage 10.0 (TID 51) in 7944 ms on 10.0.0.133 (executor driver) (34/208) +26/04/01 08:40:18 INFO Executor: Running task 41.0 in stage 10.0 (TID 58) +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO Executor: Finished task 33.0 in stage 10.0 (TID 50). 6606 bytes result sent to driver +26/04/01 08:40:18 INFO TaskSetManager: Starting task 42.0 in stage 10.0 (TID 59) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:40:18 INFO TaskSetManager: Finished task 33.0 in stage 10.0 (TID 50) in 7983 ms on 10.0.0.133 (executor driver) (35/208) +26/04/01 08:40:18 INFO Executor: Running task 42.0 in stage 10.0 (TID 59) +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO Executor: Finished task 35.0 in stage 10.0 (TID 52). 6606 bytes result sent to driver +26/04/01 08:40:18 INFO TaskSetManager: Starting task 43.0 in stage 10.0 (TID 60) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:40:18 INFO TaskSetManager: Finished task 35.0 in stage 10.0 (TID 52) in 7956 ms on 10.0.0.133 (executor driver) (36/208) +26/04/01 08:40:18 INFO Executor: Running task 43.0 in stage 10.0 (TID 60) +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO Executor: Finished task 36.0 in stage 10.0 (TID 53). 6606 bytes result sent to driver +26/04/01 08:40:18 INFO TaskSetManager: Starting task 44.0 in stage 10.0 (TID 61) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:18 INFO TaskSetManager: Finished task 36.0 in stage 10.0 (TID 53) in 7976 ms on 10.0.0.133 (executor driver) (37/208) +26/04/01 08:40:18 INFO Executor: Running task 44.0 in stage 10.0 (TID 61) +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO Executor: Finished task 39.0 in stage 10.0 (TID 56). 6606 bytes result sent to driver +26/04/01 08:40:18 INFO TaskSetManager: Starting task 45.0 in stage 10.0 (TID 62) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:18 INFO Executor: Running task 45.0 in stage 10.0 (TID 62) +26/04/01 08:40:18 INFO TaskSetManager: Finished task 39.0 in stage 10.0 (TID 56) in 7936 ms on 10.0.0.133 (executor driver) (38/208) +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO Executor: Finished task 38.0 in stage 10.0 (TID 55). 6606 bytes result sent to driver +26/04/01 08:40:18 INFO TaskSetManager: Starting task 46.0 in stage 10.0 (TID 63) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:18 INFO TaskSetManager: Finished task 38.0 in stage 10.0 (TID 55) in 7955 ms on 10.0.0.133 (executor driver) (39/208) +26/04/01 08:40:18 INFO Executor: Running task 46.0 in stage 10.0 (TID 63) +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO Executor: Finished task 37.0 in stage 10.0 (TID 54). 6606 bytes result sent to driver +26/04/01 08:40:18 INFO TaskSetManager: Starting task 47.0 in stage 10.0 (TID 64) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:18 INFO TaskSetManager: Finished task 37.0 in stage 10.0 (TID 54) in 8001 ms on 10.0.0.133 (executor driver) (40/208) +26/04/01 08:40:18 INFO Executor: Running task 47.0 in stage 10.0 (TID 64) +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO Executor: Finished task 45.0 in stage 10.0 (TID 62). 6563 bytes result sent to driver +26/04/01 08:40:26 INFO TaskSetManager: Starting task 48.0 in stage 10.0 (TID 65) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:40:26 INFO Executor: Running task 48.0 in stage 10.0 (TID 65) +26/04/01 08:40:26 INFO TaskSetManager: Finished task 45.0 in stage 10.0 (TID 62) in 7649 ms on 10.0.0.133 (executor driver) (41/208) +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO Executor: Finished task 40.0 in stage 10.0 (TID 57). 6563 bytes result sent to driver +26/04/01 08:40:26 INFO TaskSetManager: Starting task 49.0 in stage 10.0 (TID 66) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:26 INFO TaskSetManager: Finished task 40.0 in stage 10.0 (TID 57) in 7972 ms on 10.0.0.133 (executor driver) (42/208) +26/04/01 08:40:26 INFO Executor: Running task 49.0 in stage 10.0 (TID 66) +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO Executor: Finished task 42.0 in stage 10.0 (TID 59). 6563 bytes result sent to driver +26/04/01 08:40:26 INFO TaskSetManager: Starting task 50.0 in stage 10.0 (TID 67) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:26 INFO TaskSetManager: Finished task 42.0 in stage 10.0 (TID 59) in 7920 ms on 10.0.0.133 (executor driver) (43/208) +26/04/01 08:40:26 INFO Executor: Running task 50.0 in stage 10.0 (TID 67) +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO Executor: Finished task 41.0 in stage 10.0 (TID 58). 6563 bytes result sent to driver +26/04/01 08:40:26 INFO TaskSetManager: Starting task 51.0 in stage 10.0 (TID 68) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:26 INFO Executor: Running task 51.0 in stage 10.0 (TID 68) +26/04/01 08:40:26 INFO TaskSetManager: Finished task 41.0 in stage 10.0 (TID 58) in 7969 ms on 10.0.0.133 (executor driver) (44/208) +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO Executor: Finished task 43.0 in stage 10.0 (TID 60). 6563 bytes result sent to driver +26/04/01 08:40:26 INFO TaskSetManager: Starting task 52.0 in stage 10.0 (TID 69) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:26 INFO TaskSetManager: Finished task 43.0 in stage 10.0 (TID 60) in 7948 ms on 10.0.0.133 (executor driver) (45/208) +26/04/01 08:40:26 INFO Executor: Running task 52.0 in stage 10.0 (TID 69) +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO Executor: Finished task 47.0 in stage 10.0 (TID 64). 6563 bytes result sent to driver +26/04/01 08:40:26 INFO TaskSetManager: Starting task 53.0 in stage 10.0 (TID 70) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:26 INFO TaskSetManager: Finished task 47.0 in stage 10.0 (TID 64) in 7952 ms on 10.0.0.133 (executor driver) (46/208) +26/04/01 08:40:26 INFO Executor: Running task 53.0 in stage 10.0 (TID 70) +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO Executor: Finished task 46.0 in stage 10.0 (TID 63). 6563 bytes result sent to driver +26/04/01 08:40:26 INFO TaskSetManager: Starting task 54.0 in stage 10.0 (TID 71) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:40:26 INFO Executor: Running task 54.0 in stage 10.0 (TID 71) +26/04/01 08:40:26 INFO TaskSetManager: Finished task 46.0 in stage 10.0 (TID 63) in 7971 ms on 10.0.0.133 (executor driver) (47/208) +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO Executor: Finished task 44.0 in stage 10.0 (TID 61). 6563 bytes result sent to driver +26/04/01 08:40:26 INFO TaskSetManager: Starting task 55.0 in stage 10.0 (TID 72) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:26 INFO TaskSetManager: Finished task 44.0 in stage 10.0 (TID 61) in 8210 ms on 10.0.0.133 (executor driver) (48/208) +26/04/01 08:40:26 INFO Executor: Running task 55.0 in stage 10.0 (TID 72) +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:33 INFO Executor: Finished task 48.0 in stage 10.0 (TID 65). 6563 bytes result sent to driver +26/04/01 08:40:33 INFO TaskSetManager: Starting task 56.0 in stage 10.0 (TID 73) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:33 INFO TaskSetManager: Finished task 48.0 in stage 10.0 (TID 65) in 7978 ms on 10.0.0.133 (executor driver) (49/208) +26/04/01 08:40:33 INFO Executor: Running task 56.0 in stage 10.0 (TID 73) +26/04/01 08:40:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:34 INFO Executor: Finished task 49.0 in stage 10.0 (TID 66). 6563 bytes result sent to driver +26/04/01 08:40:34 INFO TaskSetManager: Starting task 57.0 in stage 10.0 (TID 74) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:34 INFO TaskSetManager: Finished task 49.0 in stage 10.0 (TID 66) in 7980 ms on 10.0.0.133 (executor driver) (50/208) +26/04/01 08:40:34 INFO Executor: Running task 57.0 in stage 10.0 (TID 74) +26/04/01 08:40:34 INFO Executor: Finished task 50.0 in stage 10.0 (TID 67). 6563 bytes result sent to driver +26/04/01 08:40:34 INFO TaskSetManager: Starting task 58.0 in stage 10.0 (TID 75) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:34 INFO TaskSetManager: Finished task 50.0 in stage 10.0 (TID 67) in 7940 ms on 10.0.0.133 (executor driver) (51/208) +26/04/01 08:40:34 INFO Executor: Running task 58.0 in stage 10.0 (TID 75) +26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:34 INFO Executor: Finished task 51.0 in stage 10.0 (TID 68). 6563 bytes result sent to driver +26/04/01 08:40:34 INFO TaskSetManager: Starting task 59.0 in stage 10.0 (TID 76) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:34 INFO TaskSetManager: Finished task 51.0 in stage 10.0 (TID 68) in 7962 ms on 10.0.0.133 (executor driver) (52/208) +26/04/01 08:40:34 INFO Executor: Running task 59.0 in stage 10.0 (TID 76) +26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:34 INFO Executor: Finished task 52.0 in stage 10.0 (TID 69). 6563 bytes result sent to driver +26/04/01 08:40:34 INFO TaskSetManager: Starting task 60.0 in stage 10.0 (TID 77) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:40:34 INFO Executor: Running task 60.0 in stage 10.0 (TID 77) +26/04/01 08:40:34 INFO TaskSetManager: Finished task 52.0 in stage 10.0 (TID 69) in 7989 ms on 10.0.0.133 (executor driver) (53/208) +26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:34 INFO Executor: Finished task 53.0 in stage 10.0 (TID 70). 6563 bytes result sent to driver +26/04/01 08:40:34 INFO TaskSetManager: Starting task 61.0 in stage 10.0 (TID 78) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:34 INFO TaskSetManager: Finished task 53.0 in stage 10.0 (TID 70) in 7977 ms on 10.0.0.133 (executor driver) (54/208) +26/04/01 08:40:34 INFO Executor: Running task 61.0 in stage 10.0 (TID 78) +26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:34 INFO Executor: Finished task 54.0 in stage 10.0 (TID 71). 6563 bytes result sent to driver +26/04/01 08:40:34 INFO TaskSetManager: Starting task 62.0 in stage 10.0 (TID 79) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:34 INFO TaskSetManager: Finished task 54.0 in stage 10.0 (TID 71) in 7988 ms on 10.0.0.133 (executor driver) (55/208) +26/04/01 08:40:34 INFO Executor: Running task 62.0 in stage 10.0 (TID 79) +26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:34 INFO Executor: Finished task 55.0 in stage 10.0 (TID 72). 6563 bytes result sent to driver +26/04/01 08:40:34 INFO TaskSetManager: Starting task 63.0 in stage 10.0 (TID 80) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:34 INFO TaskSetManager: Finished task 55.0 in stage 10.0 (TID 72) in 7966 ms on 10.0.0.133 (executor driver) (56/208) +26/04/01 08:40:34 INFO Executor: Running task 63.0 in stage 10.0 (TID 80) +26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:41 INFO Executor: Finished task 56.0 in stage 10.0 (TID 73). 6563 bytes result sent to driver +26/04/01 08:40:41 INFO TaskSetManager: Starting task 64.0 in stage 10.0 (TID 81) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:41 INFO TaskSetManager: Finished task 56.0 in stage 10.0 (TID 73) in 7963 ms on 10.0.0.133 (executor driver) (57/208) +26/04/01 08:40:41 INFO Executor: Running task 64.0 in stage 10.0 (TID 81) +26/04/01 08:40:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:42 INFO Executor: Finished task 59.0 in stage 10.0 (TID 76). 6563 bytes result sent to driver +26/04/01 08:40:42 INFO TaskSetManager: Starting task 65.0 in stage 10.0 (TID 82) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:42 INFO TaskSetManager: Finished task 59.0 in stage 10.0 (TID 76) in 7907 ms on 10.0.0.133 (executor driver) (58/208) +26/04/01 08:40:42 INFO Executor: Running task 65.0 in stage 10.0 (TID 82) +26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:42 INFO Executor: Finished task 57.0 in stage 10.0 (TID 74). 6563 bytes result sent to driver +26/04/01 08:40:42 INFO TaskSetManager: Starting task 66.0 in stage 10.0 (TID 83) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:40:42 INFO TaskSetManager: Finished task 57.0 in stage 10.0 (TID 74) in 7958 ms on 10.0.0.133 (executor driver) (59/208) +26/04/01 08:40:42 INFO Executor: Running task 66.0 in stage 10.0 (TID 83) +26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:42 INFO Executor: Finished task 60.0 in stage 10.0 (TID 77). 6563 bytes result sent to driver +26/04/01 08:40:42 INFO TaskSetManager: Starting task 67.0 in stage 10.0 (TID 84) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:40:42 INFO Executor: Running task 67.0 in stage 10.0 (TID 84) +26/04/01 08:40:42 INFO TaskSetManager: Finished task 60.0 in stage 10.0 (TID 77) in 7908 ms on 10.0.0.133 (executor driver) (60/208) +26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:42 INFO Executor: Finished task 58.0 in stage 10.0 (TID 75). 6563 bytes result sent to driver +26/04/01 08:40:42 INFO TaskSetManager: Starting task 68.0 in stage 10.0 (TID 85) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:42 INFO Executor: Running task 68.0 in stage 10.0 (TID 85) +26/04/01 08:40:42 INFO TaskSetManager: Finished task 58.0 in stage 10.0 (TID 75) in 7999 ms on 10.0.0.133 (executor driver) (61/208) +26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:42 INFO Executor: Finished task 61.0 in stage 10.0 (TID 78). 6563 bytes result sent to driver +26/04/01 08:40:42 INFO TaskSetManager: Starting task 69.0 in stage 10.0 (TID 86) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:42 INFO Executor: Running task 69.0 in stage 10.0 (TID 86) +26/04/01 08:40:42 INFO TaskSetManager: Finished task 61.0 in stage 10.0 (TID 78) in 7879 ms on 10.0.0.133 (executor driver) (62/208) +26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:42 INFO Executor: Finished task 62.0 in stage 10.0 (TID 79). 6563 bytes result sent to driver +26/04/01 08:40:42 INFO TaskSetManager: Starting task 70.0 in stage 10.0 (TID 87) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:42 INFO TaskSetManager: Finished task 62.0 in stage 10.0 (TID 79) in 7955 ms on 10.0.0.133 (executor driver) (63/208) +26/04/01 08:40:42 INFO Executor: Running task 70.0 in stage 10.0 (TID 87) +26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:42 INFO Executor: Finished task 63.0 in stage 10.0 (TID 80). 6606 bytes result sent to driver +26/04/01 08:40:42 INFO TaskSetManager: Starting task 71.0 in stage 10.0 (TID 88) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:42 INFO Executor: Running task 71.0 in stage 10.0 (TID 88) +26/04/01 08:40:42 INFO TaskSetManager: Finished task 63.0 in stage 10.0 (TID 80) in 7966 ms on 10.0.0.133 (executor driver) (64/208) +26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:49 INFO Executor: Finished task 64.0 in stage 10.0 (TID 81). 6606 bytes result sent to driver +26/04/01 08:40:49 INFO TaskSetManager: Starting task 72.0 in stage 10.0 (TID 89) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:40:49 INFO TaskSetManager: Finished task 64.0 in stage 10.0 (TID 81) in 7934 ms on 10.0.0.133 (executor driver) (65/208) +26/04/01 08:40:49 INFO Executor: Running task 72.0 in stage 10.0 (TID 89) +26/04/01 08:40:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:50 INFO Executor: Finished task 65.0 in stage 10.0 (TID 82). 6606 bytes result sent to driver +26/04/01 08:40:50 INFO TaskSetManager: Starting task 73.0 in stage 10.0 (TID 90) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:40:50 INFO TaskSetManager: Finished task 65.0 in stage 10.0 (TID 82) in 7957 ms on 10.0.0.133 (executor driver) (66/208) +26/04/01 08:40:50 INFO Executor: Running task 73.0 in stage 10.0 (TID 90) +26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:50 INFO Executor: Finished task 66.0 in stage 10.0 (TID 83). 6606 bytes result sent to driver +26/04/01 08:40:50 INFO TaskSetManager: Starting task 74.0 in stage 10.0 (TID 91) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:50 INFO TaskSetManager: Finished task 66.0 in stage 10.0 (TID 83) in 7956 ms on 10.0.0.133 (executor driver) (67/208) +26/04/01 08:40:50 INFO Executor: Running task 74.0 in stage 10.0 (TID 91) +26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:50 INFO Executor: Finished task 68.0 in stage 10.0 (TID 85). 6606 bytes result sent to driver +26/04/01 08:40:50 INFO TaskSetManager: Starting task 75.0 in stage 10.0 (TID 92) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:50 INFO TaskSetManager: Finished task 68.0 in stage 10.0 (TID 85) in 7918 ms on 10.0.0.133 (executor driver) (68/208) +26/04/01 08:40:50 INFO Executor: Running task 75.0 in stage 10.0 (TID 92) +26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:50 INFO Executor: Finished task 69.0 in stage 10.0 (TID 86). 6606 bytes result sent to driver +26/04/01 08:40:50 INFO TaskSetManager: Starting task 76.0 in stage 10.0 (TID 93) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:50 INFO TaskSetManager: Finished task 69.0 in stage 10.0 (TID 86) in 7907 ms on 10.0.0.133 (executor driver) (69/208) +26/04/01 08:40:50 INFO Executor: Running task 76.0 in stage 10.0 (TID 93) +26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:50 INFO Executor: Finished task 67.0 in stage 10.0 (TID 84). 6606 bytes result sent to driver +26/04/01 08:40:50 INFO TaskSetManager: Starting task 77.0 in stage 10.0 (TID 94) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:50 INFO TaskSetManager: Finished task 67.0 in stage 10.0 (TID 84) in 7954 ms on 10.0.0.133 (executor driver) (70/208) +26/04/01 08:40:50 INFO Executor: Running task 77.0 in stage 10.0 (TID 94) +26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:50 INFO Executor: Finished task 70.0 in stage 10.0 (TID 87). 6606 bytes result sent to driver +26/04/01 08:40:50 INFO TaskSetManager: Starting task 78.0 in stage 10.0 (TID 95) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:40:50 INFO TaskSetManager: Finished task 70.0 in stage 10.0 (TID 87) in 7939 ms on 10.0.0.133 (executor driver) (71/208) +26/04/01 08:40:50 INFO Executor: Running task 78.0 in stage 10.0 (TID 95) +26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:50 INFO Executor: Finished task 71.0 in stage 10.0 (TID 88). 6563 bytes result sent to driver +26/04/01 08:40:50 INFO TaskSetManager: Starting task 79.0 in stage 10.0 (TID 96) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:50 INFO TaskSetManager: Finished task 71.0 in stage 10.0 (TID 88) in 7952 ms on 10.0.0.133 (executor driver) (72/208) +26/04/01 08:40:50 INFO Executor: Running task 79.0 in stage 10.0 (TID 96) +26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:57 INFO Executor: Finished task 72.0 in stage 10.0 (TID 89). 6563 bytes result sent to driver +26/04/01 08:40:57 INFO TaskSetManager: Starting task 80.0 in stage 10.0 (TID 97) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:57 INFO Executor: Running task 80.0 in stage 10.0 (TID 97) +26/04/01 08:40:57 INFO TaskSetManager: Finished task 72.0 in stage 10.0 (TID 89) in 7967 ms on 10.0.0.133 (executor driver) (73/208) +26/04/01 08:40:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:58 INFO Executor: Finished task 76.0 in stage 10.0 (TID 93). 6563 bytes result sent to driver +26/04/01 08:40:58 INFO TaskSetManager: Starting task 81.0 in stage 10.0 (TID 98) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:58 INFO TaskSetManager: Finished task 76.0 in stage 10.0 (TID 93) in 7924 ms on 10.0.0.133 (executor driver) (74/208) +26/04/01 08:40:58 INFO Executor: Running task 81.0 in stage 10.0 (TID 98) +26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:58 INFO Executor: Finished task 75.0 in stage 10.0 (TID 92). 6563 bytes result sent to driver +26/04/01 08:40:58 INFO TaskSetManager: Starting task 82.0 in stage 10.0 (TID 99) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:58 INFO Executor: Running task 82.0 in stage 10.0 (TID 99) +26/04/01 08:40:58 INFO TaskSetManager: Finished task 75.0 in stage 10.0 (TID 92) in 7942 ms on 10.0.0.133 (executor driver) (75/208) +26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:58 INFO Executor: Finished task 73.0 in stage 10.0 (TID 90). 6563 bytes result sent to driver +26/04/01 08:40:58 INFO TaskSetManager: Starting task 83.0 in stage 10.0 (TID 100) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:58 INFO Executor: Running task 83.0 in stage 10.0 (TID 100) +26/04/01 08:40:58 INFO TaskSetManager: Finished task 73.0 in stage 10.0 (TID 90) in 7955 ms on 10.0.0.133 (executor driver) (76/208) +26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:58 INFO Executor: Finished task 74.0 in stage 10.0 (TID 91). 6563 bytes result sent to driver +26/04/01 08:40:58 INFO TaskSetManager: Starting task 84.0 in stage 10.0 (TID 101) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:40:58 INFO TaskSetManager: Finished task 74.0 in stage 10.0 (TID 91) in 7964 ms on 10.0.0.133 (executor driver) (77/208) +26/04/01 08:40:58 INFO Executor: Running task 84.0 in stage 10.0 (TID 101) +26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:58 INFO Executor: Finished task 77.0 in stage 10.0 (TID 94). 6563 bytes result sent to driver +26/04/01 08:40:58 INFO TaskSetManager: Starting task 85.0 in stage 10.0 (TID 102) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:40:58 INFO TaskSetManager: Finished task 77.0 in stage 10.0 (TID 94) in 7940 ms on 10.0.0.133 (executor driver) (78/208) +26/04/01 08:40:58 INFO Executor: Running task 85.0 in stage 10.0 (TID 102) +26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:58 INFO Executor: Finished task 78.0 in stage 10.0 (TID 95). 6563 bytes result sent to driver +26/04/01 08:40:58 INFO TaskSetManager: Starting task 86.0 in stage 10.0 (TID 103) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:58 INFO TaskSetManager: Finished task 78.0 in stage 10.0 (TID 95) in 7968 ms on 10.0.0.133 (executor driver) (79/208) +26/04/01 08:40:58 INFO Executor: Running task 86.0 in stage 10.0 (TID 103) +26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:58 INFO Executor: Finished task 79.0 in stage 10.0 (TID 96). 6563 bytes result sent to driver +26/04/01 08:40:58 INFO TaskSetManager: Starting task 87.0 in stage 10.0 (TID 104) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:40:58 INFO Executor: Running task 87.0 in stage 10.0 (TID 104) +26/04/01 08:40:58 INFO TaskSetManager: Finished task 79.0 in stage 10.0 (TID 96) in 7948 ms on 10.0.0.133 (executor driver) (80/208) +26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:05 INFO Executor: Finished task 80.0 in stage 10.0 (TID 97). 6563 bytes result sent to driver +26/04/01 08:41:05 INFO TaskSetManager: Starting task 88.0 in stage 10.0 (TID 105) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:05 INFO TaskSetManager: Finished task 80.0 in stage 10.0 (TID 97) in 7948 ms on 10.0.0.133 (executor driver) (81/208) +26/04/01 08:41:05 INFO Executor: Running task 88.0 in stage 10.0 (TID 105) +26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:05 INFO Executor: Finished task 82.0 in stage 10.0 (TID 99). 6563 bytes result sent to driver +26/04/01 08:41:05 INFO TaskSetManager: Starting task 89.0 in stage 10.0 (TID 106) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:05 INFO TaskSetManager: Finished task 82.0 in stage 10.0 (TID 99) in 7943 ms on 10.0.0.133 (executor driver) (82/208) +26/04/01 08:41:05 INFO Executor: Running task 89.0 in stage 10.0 (TID 106) +26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:05 INFO Executor: Finished task 84.0 in stage 10.0 (TID 101). 6563 bytes result sent to driver +26/04/01 08:41:05 INFO TaskSetManager: Starting task 90.0 in stage 10.0 (TID 107) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:41:05 INFO TaskSetManager: Finished task 84.0 in stage 10.0 (TID 101) in 7932 ms on 10.0.0.133 (executor driver) (83/208) +26/04/01 08:41:05 INFO Executor: Running task 90.0 in stage 10.0 (TID 107) +26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:05 INFO Executor: Finished task 81.0 in stage 10.0 (TID 98). 6563 bytes result sent to driver +26/04/01 08:41:05 INFO TaskSetManager: Starting task 91.0 in stage 10.0 (TID 108) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:41:05 INFO TaskSetManager: Finished task 81.0 in stage 10.0 (TID 98) in 7964 ms on 10.0.0.133 (executor driver) (84/208) +26/04/01 08:41:05 INFO Executor: Running task 91.0 in stage 10.0 (TID 108) +26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:06 INFO Executor: Finished task 83.0 in stage 10.0 (TID 100). 6563 bytes result sent to driver +26/04/01 08:41:06 INFO TaskSetManager: Starting task 92.0 in stage 10.0 (TID 109) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:06 INFO Executor: Running task 92.0 in stage 10.0 (TID 109) +26/04/01 08:41:06 INFO TaskSetManager: Finished task 83.0 in stage 10.0 (TID 100) in 7970 ms on 10.0.0.133 (executor driver) (85/208) +26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:06 INFO Executor: Finished task 85.0 in stage 10.0 (TID 102). 6563 bytes result sent to driver +26/04/01 08:41:06 INFO TaskSetManager: Starting task 93.0 in stage 10.0 (TID 110) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:06 INFO TaskSetManager: Finished task 85.0 in stage 10.0 (TID 102) in 7989 ms on 10.0.0.133 (executor driver) (86/208) +26/04/01 08:41:06 INFO Executor: Running task 93.0 in stage 10.0 (TID 110) +26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:06 INFO Executor: Finished task 86.0 in stage 10.0 (TID 103). 6563 bytes result sent to driver +26/04/01 08:41:06 INFO TaskSetManager: Starting task 94.0 in stage 10.0 (TID 111) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:06 INFO TaskSetManager: Finished task 86.0 in stage 10.0 (TID 103) in 7968 ms on 10.0.0.133 (executor driver) (87/208) +26/04/01 08:41:06 INFO Executor: Running task 94.0 in stage 10.0 (TID 111) +26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:06 INFO Executor: Finished task 87.0 in stage 10.0 (TID 104). 6563 bytes result sent to driver +26/04/01 08:41:06 INFO TaskSetManager: Starting task 95.0 in stage 10.0 (TID 112) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:06 INFO TaskSetManager: Finished task 87.0 in stage 10.0 (TID 104) in 7952 ms on 10.0.0.133 (executor driver) (88/208) +26/04/01 08:41:06 INFO Executor: Running task 95.0 in stage 10.0 (TID 112) +26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:13 INFO Executor: Finished task 91.0 in stage 10.0 (TID 108). 6563 bytes result sent to driver +26/04/01 08:41:13 INFO TaskSetManager: Starting task 96.0 in stage 10.0 (TID 113) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:41:13 INFO TaskSetManager: Finished task 91.0 in stage 10.0 (TID 108) in 7664 ms on 10.0.0.133 (executor driver) (89/208) +26/04/01 08:41:13 INFO Executor: Running task 96.0 in stage 10.0 (TID 113) +26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:13 INFO Executor: Finished task 88.0 in stage 10.0 (TID 105). 6563 bytes result sent to driver +26/04/01 08:41:13 INFO TaskSetManager: Starting task 97.0 in stage 10.0 (TID 114) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:41:13 INFO Executor: Running task 97.0 in stage 10.0 (TID 114) +26/04/01 08:41:13 INFO TaskSetManager: Finished task 88.0 in stage 10.0 (TID 105) in 7930 ms on 10.0.0.133 (executor driver) (90/208) +26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:13 INFO Executor: Finished task 89.0 in stage 10.0 (TID 106). 6563 bytes result sent to driver +26/04/01 08:41:13 INFO TaskSetManager: Starting task 98.0 in stage 10.0 (TID 115) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:41:13 INFO TaskSetManager: Finished task 89.0 in stage 10.0 (TID 106) in 7922 ms on 10.0.0.133 (executor driver) (91/208) +26/04/01 08:41:13 INFO Executor: Running task 98.0 in stage 10.0 (TID 115) +26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:13 INFO Executor: Finished task 93.0 in stage 10.0 (TID 110). 6563 bytes result sent to driver +26/04/01 08:41:13 INFO TaskSetManager: Starting task 99.0 in stage 10.0 (TID 116) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:41:13 INFO TaskSetManager: Finished task 93.0 in stage 10.0 (TID 110) in 7904 ms on 10.0.0.133 (executor driver) (92/208) +26/04/01 08:41:13 INFO Executor: Running task 99.0 in stage 10.0 (TID 116) +26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:13 INFO Executor: Finished task 92.0 in stage 10.0 (TID 109). 6563 bytes result sent to driver +26/04/01 08:41:13 INFO TaskSetManager: Starting task 100.0 in stage 10.0 (TID 117) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:41:13 INFO TaskSetManager: Finished task 92.0 in stage 10.0 (TID 109) in 7959 ms on 10.0.0.133 (executor driver) (93/208) +26/04/01 08:41:13 INFO Executor: Running task 100.0 in stage 10.0 (TID 117) +26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:14 INFO Executor: Finished task 94.0 in stage 10.0 (TID 111). 6606 bytes result sent to driver +26/04/01 08:41:14 INFO TaskSetManager: Starting task 101.0 in stage 10.0 (TID 118) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:41:14 INFO TaskSetManager: Finished task 94.0 in stage 10.0 (TID 111) in 7963 ms on 10.0.0.133 (executor driver) (94/208) +26/04/01 08:41:14 INFO Executor: Running task 101.0 in stage 10.0 (TID 118) +26/04/01 08:41:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:14 INFO Executor: Finished task 90.0 in stage 10.0 (TID 107). 6606 bytes result sent to driver +26/04/01 08:41:14 INFO TaskSetManager: Starting task 102.0 in stage 10.0 (TID 119) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:41:14 INFO TaskSetManager: Finished task 90.0 in stage 10.0 (TID 107) in 8226 ms on 10.0.0.133 (executor driver) (95/208) +26/04/01 08:41:14 INFO Executor: Running task 102.0 in stage 10.0 (TID 119) +26/04/01 08:41:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:14 INFO Executor: Finished task 95.0 in stage 10.0 (TID 112). 6606 bytes result sent to driver +26/04/01 08:41:14 INFO TaskSetManager: Starting task 103.0 in stage 10.0 (TID 120) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:41:14 INFO Executor: Running task 103.0 in stage 10.0 (TID 120) +26/04/01 08:41:14 INFO TaskSetManager: Finished task 95.0 in stage 10.0 (TID 112) in 7954 ms on 10.0.0.133 (executor driver) (96/208) +26/04/01 08:41:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:21 INFO Executor: Finished task 96.0 in stage 10.0 (TID 113). 6606 bytes result sent to driver +26/04/01 08:41:21 INFO TaskSetManager: Starting task 104.0 in stage 10.0 (TID 121) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:21 INFO TaskSetManager: Finished task 96.0 in stage 10.0 (TID 113) in 7999 ms on 10.0.0.133 (executor driver) (97/208) +26/04/01 08:41:21 INFO Executor: Running task 104.0 in stage 10.0 (TID 121) +26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:21 INFO Executor: Finished task 97.0 in stage 10.0 (TID 114). 6606 bytes result sent to driver +26/04/01 08:41:21 INFO TaskSetManager: Starting task 105.0 in stage 10.0 (TID 122) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:21 INFO TaskSetManager: Finished task 97.0 in stage 10.0 (TID 114) in 7992 ms on 10.0.0.133 (executor driver) (98/208) +26/04/01 08:41:21 INFO Executor: Running task 105.0 in stage 10.0 (TID 122) +26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:21 INFO Executor: Finished task 98.0 in stage 10.0 (TID 115). 6606 bytes result sent to driver +26/04/01 08:41:21 INFO TaskSetManager: Starting task 106.0 in stage 10.0 (TID 123) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:21 INFO Executor: Running task 106.0 in stage 10.0 (TID 123) +26/04/01 08:41:21 INFO TaskSetManager: Finished task 98.0 in stage 10.0 (TID 115) in 7985 ms on 10.0.0.133 (executor driver) (99/208) +26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:21 INFO Executor: Finished task 99.0 in stage 10.0 (TID 116). 6606 bytes result sent to driver +26/04/01 08:41:21 INFO TaskSetManager: Starting task 107.0 in stage 10.0 (TID 124) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:21 INFO TaskSetManager: Finished task 99.0 in stage 10.0 (TID 116) in 7964 ms on 10.0.0.133 (executor driver) (100/208) +26/04/01 08:41:21 INFO Executor: Running task 107.0 in stage 10.0 (TID 124) +26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:21 INFO Executor: Finished task 100.0 in stage 10.0 (TID 117). 6606 bytes result sent to driver +26/04/01 08:41:21 INFO TaskSetManager: Starting task 108.0 in stage 10.0 (TID 125) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:41:21 INFO Executor: Running task 108.0 in stage 10.0 (TID 125) +26/04/01 08:41:21 INFO TaskSetManager: Finished task 100.0 in stage 10.0 (TID 117) in 8000 ms on 10.0.0.133 (executor driver) (101/208) +26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:22 INFO Executor: Finished task 101.0 in stage 10.0 (TID 118). 6563 bytes result sent to driver +26/04/01 08:41:22 INFO TaskSetManager: Starting task 109.0 in stage 10.0 (TID 126) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:41:22 INFO TaskSetManager: Finished task 101.0 in stage 10.0 (TID 118) in 8013 ms on 10.0.0.133 (executor driver) (102/208) +26/04/01 08:41:22 INFO Executor: Running task 109.0 in stage 10.0 (TID 126) +26/04/01 08:41:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:22 INFO Executor: Finished task 102.0 in stage 10.0 (TID 119). 6563 bytes result sent to driver +26/04/01 08:41:22 INFO TaskSetManager: Starting task 110.0 in stage 10.0 (TID 127) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:41:22 INFO TaskSetManager: Finished task 102.0 in stage 10.0 (TID 119) in 7955 ms on 10.0.0.133 (executor driver) (103/208) +26/04/01 08:41:22 INFO Executor: Running task 110.0 in stage 10.0 (TID 127) +26/04/01 08:41:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:22 INFO Executor: Finished task 103.0 in stage 10.0 (TID 120). 6563 bytes result sent to driver +26/04/01 08:41:22 INFO TaskSetManager: Starting task 111.0 in stage 10.0 (TID 128) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:41:22 INFO TaskSetManager: Finished task 103.0 in stage 10.0 (TID 120) in 8005 ms on 10.0.0.133 (executor driver) (104/208) +26/04/01 08:41:22 INFO Executor: Running task 111.0 in stage 10.0 (TID 128) +26/04/01 08:41:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:29 INFO Executor: Finished task 104.0 in stage 10.0 (TID 121). 6563 bytes result sent to driver +26/04/01 08:41:29 INFO TaskSetManager: Starting task 112.0 in stage 10.0 (TID 129) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:41:29 INFO TaskSetManager: Finished task 104.0 in stage 10.0 (TID 121) in 7957 ms on 10.0.0.133 (executor driver) (105/208) +26/04/01 08:41:29 INFO Executor: Running task 112.0 in stage 10.0 (TID 129) +26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:29 INFO Executor: Finished task 105.0 in stage 10.0 (TID 122). 6563 bytes result sent to driver +26/04/01 08:41:29 INFO TaskSetManager: Starting task 113.0 in stage 10.0 (TID 130) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:41:29 INFO TaskSetManager: Finished task 105.0 in stage 10.0 (TID 122) in 7957 ms on 10.0.0.133 (executor driver) (106/208) +26/04/01 08:41:29 INFO Executor: Running task 113.0 in stage 10.0 (TID 130) +26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:29 INFO Executor: Finished task 107.0 in stage 10.0 (TID 124). 6563 bytes result sent to driver +26/04/01 08:41:29 INFO TaskSetManager: Starting task 114.0 in stage 10.0 (TID 131) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:41:29 INFO TaskSetManager: Finished task 107.0 in stage 10.0 (TID 124) in 7949 ms on 10.0.0.133 (executor driver) (107/208) +26/04/01 08:41:29 INFO Executor: Running task 114.0 in stage 10.0 (TID 131) +26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:29 INFO Executor: Finished task 106.0 in stage 10.0 (TID 123). 6563 bytes result sent to driver +26/04/01 08:41:29 INFO TaskSetManager: Starting task 115.0 in stage 10.0 (TID 132) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:41:29 INFO TaskSetManager: Finished task 106.0 in stage 10.0 (TID 123) in 8001 ms on 10.0.0.133 (executor driver) (108/208) +26/04/01 08:41:29 INFO Executor: Running task 115.0 in stage 10.0 (TID 132) +26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:29 INFO Executor: Finished task 108.0 in stage 10.0 (TID 125). 6563 bytes result sent to driver +26/04/01 08:41:29 INFO TaskSetManager: Starting task 116.0 in stage 10.0 (TID 133) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:29 INFO TaskSetManager: Finished task 108.0 in stage 10.0 (TID 125) in 7953 ms on 10.0.0.133 (executor driver) (109/208) +26/04/01 08:41:29 INFO Executor: Running task 116.0 in stage 10.0 (TID 133) +26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:30 INFO Executor: Finished task 109.0 in stage 10.0 (TID 126). 6563 bytes result sent to driver +26/04/01 08:41:30 INFO TaskSetManager: Starting task 117.0 in stage 10.0 (TID 134) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:30 INFO Executor: Running task 117.0 in stage 10.0 (TID 134) +26/04/01 08:41:30 INFO TaskSetManager: Finished task 109.0 in stage 10.0 (TID 126) in 7943 ms on 10.0.0.133 (executor driver) (110/208) +26/04/01 08:41:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:30 INFO Executor: Finished task 110.0 in stage 10.0 (TID 127). 6563 bytes result sent to driver +26/04/01 08:41:30 INFO TaskSetManager: Starting task 118.0 in stage 10.0 (TID 135) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:30 INFO Executor: Running task 118.0 in stage 10.0 (TID 135) +26/04/01 08:41:30 INFO TaskSetManager: Finished task 110.0 in stage 10.0 (TID 127) in 7976 ms on 10.0.0.133 (executor driver) (111/208) +26/04/01 08:41:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:30 INFO Executor: Finished task 111.0 in stage 10.0 (TID 128). 6563 bytes result sent to driver +26/04/01 08:41:30 INFO TaskSetManager: Starting task 119.0 in stage 10.0 (TID 136) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:30 INFO TaskSetManager: Finished task 111.0 in stage 10.0 (TID 128) in 7974 ms on 10.0.0.133 (executor driver) (112/208) +26/04/01 08:41:30 INFO Executor: Running task 119.0 in stage 10.0 (TID 136) +26/04/01 08:41:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:37 INFO Executor: Finished task 112.0 in stage 10.0 (TID 129). 6563 bytes result sent to driver +26/04/01 08:41:37 INFO TaskSetManager: Starting task 120.0 in stage 10.0 (TID 137) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:41:37 INFO TaskSetManager: Finished task 112.0 in stage 10.0 (TID 129) in 7995 ms on 10.0.0.133 (executor driver) (113/208) +26/04/01 08:41:37 INFO Executor: Running task 120.0 in stage 10.0 (TID 137) +26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:37 INFO Executor: Finished task 113.0 in stage 10.0 (TID 130). 6563 bytes result sent to driver +26/04/01 08:41:37 INFO TaskSetManager: Starting task 121.0 in stage 10.0 (TID 138) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:41:37 INFO TaskSetManager: Finished task 113.0 in stage 10.0 (TID 130) in 7981 ms on 10.0.0.133 (executor driver) (114/208) +26/04/01 08:41:37 INFO Executor: Running task 121.0 in stage 10.0 (TID 138) +26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:37 INFO Executor: Finished task 114.0 in stage 10.0 (TID 131). 6563 bytes result sent to driver +26/04/01 08:41:37 INFO TaskSetManager: Starting task 122.0 in stage 10.0 (TID 139) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:37 INFO TaskSetManager: Finished task 114.0 in stage 10.0 (TID 131) in 7991 ms on 10.0.0.133 (executor driver) (115/208) +26/04/01 08:41:37 INFO Executor: Running task 122.0 in stage 10.0 (TID 139) +26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:37 INFO Executor: Finished task 115.0 in stage 10.0 (TID 132). 6563 bytes result sent to driver +26/04/01 08:41:37 INFO TaskSetManager: Starting task 123.0 in stage 10.0 (TID 140) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:37 INFO TaskSetManager: Finished task 115.0 in stage 10.0 (TID 132) in 7978 ms on 10.0.0.133 (executor driver) (116/208) +26/04/01 08:41:37 INFO Executor: Running task 123.0 in stage 10.0 (TID 140) +26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:37 INFO Executor: Finished task 116.0 in stage 10.0 (TID 133). 6563 bytes result sent to driver +26/04/01 08:41:37 INFO TaskSetManager: Starting task 124.0 in stage 10.0 (TID 141) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:37 INFO TaskSetManager: Finished task 116.0 in stage 10.0 (TID 133) in 7968 ms on 10.0.0.133 (executor driver) (117/208) +26/04/01 08:41:37 INFO Executor: Running task 124.0 in stage 10.0 (TID 141) +26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:38 INFO Executor: Finished task 117.0 in stage 10.0 (TID 134). 6563 bytes result sent to driver +26/04/01 08:41:38 INFO TaskSetManager: Starting task 125.0 in stage 10.0 (TID 142) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:38 INFO TaskSetManager: Finished task 117.0 in stage 10.0 (TID 134) in 7972 ms on 10.0.0.133 (executor driver) (118/208) +26/04/01 08:41:38 INFO Executor: Running task 125.0 in stage 10.0 (TID 142) +26/04/01 08:41:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:38 INFO Executor: Finished task 118.0 in stage 10.0 (TID 135). 6563 bytes result sent to driver +26/04/01 08:41:38 INFO TaskSetManager: Starting task 126.0 in stage 10.0 (TID 143) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:41:38 INFO Executor: Running task 126.0 in stage 10.0 (TID 143) +26/04/01 08:41:38 INFO TaskSetManager: Finished task 118.0 in stage 10.0 (TID 135) in 7967 ms on 10.0.0.133 (executor driver) (119/208) +26/04/01 08:41:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:38 INFO Executor: Finished task 119.0 in stage 10.0 (TID 136). 6563 bytes result sent to driver +26/04/01 08:41:38 INFO TaskSetManager: Starting task 127.0 in stage 10.0 (TID 144) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:41:38 INFO TaskSetManager: Finished task 119.0 in stage 10.0 (TID 136) in 7987 ms on 10.0.0.133 (executor driver) (120/208) +26/04/01 08:41:38 INFO Executor: Running task 127.0 in stage 10.0 (TID 144) +26/04/01 08:41:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:45 INFO Executor: Finished task 120.0 in stage 10.0 (TID 137). 6563 bytes result sent to driver +26/04/01 08:41:45 INFO TaskSetManager: Starting task 128.0 in stage 10.0 (TID 145) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:45 INFO TaskSetManager: Finished task 120.0 in stage 10.0 (TID 137) in 7976 ms on 10.0.0.133 (executor driver) (121/208) +26/04/01 08:41:45 INFO Executor: Running task 128.0 in stage 10.0 (TID 145) +26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:45 INFO Executor: Finished task 121.0 in stage 10.0 (TID 138). 6649 bytes result sent to driver +26/04/01 08:41:45 INFO TaskSetManager: Starting task 129.0 in stage 10.0 (TID 146) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:45 INFO TaskSetManager: Finished task 121.0 in stage 10.0 (TID 138) in 7953 ms on 10.0.0.133 (executor driver) (122/208) +26/04/01 08:41:45 INFO Executor: Running task 129.0 in stage 10.0 (TID 146) +26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:45 INFO Executor: Finished task 122.0 in stage 10.0 (TID 139). 6606 bytes result sent to driver +26/04/01 08:41:45 INFO TaskSetManager: Starting task 130.0 in stage 10.0 (TID 147) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:45 INFO TaskSetManager: Finished task 122.0 in stage 10.0 (TID 139) in 7934 ms on 10.0.0.133 (executor driver) (123/208) +26/04/01 08:41:45 INFO Executor: Running task 130.0 in stage 10.0 (TID 147) +26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:45 INFO Executor: Finished task 123.0 in stage 10.0 (TID 140). 6606 bytes result sent to driver +26/04/01 08:41:45 INFO TaskSetManager: Starting task 131.0 in stage 10.0 (TID 148) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:45 INFO TaskSetManager: Finished task 123.0 in stage 10.0 (TID 140) in 7930 ms on 10.0.0.133 (executor driver) (124/208) +26/04/01 08:41:45 INFO Executor: Running task 131.0 in stage 10.0 (TID 148) +26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:45 INFO Executor: Finished task 124.0 in stage 10.0 (TID 141). 6606 bytes result sent to driver +26/04/01 08:41:45 INFO TaskSetManager: Starting task 132.0 in stage 10.0 (TID 149) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:41:45 INFO Executor: Running task 132.0 in stage 10.0 (TID 149) +26/04/01 08:41:45 INFO TaskSetManager: Finished task 124.0 in stage 10.0 (TID 141) in 7962 ms on 10.0.0.133 (executor driver) (125/208) +26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:46 INFO Executor: Finished task 125.0 in stage 10.0 (TID 142). 6606 bytes result sent to driver +26/04/01 08:41:46 INFO TaskSetManager: Starting task 133.0 in stage 10.0 (TID 150) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:41:46 INFO TaskSetManager: Finished task 125.0 in stage 10.0 (TID 142) in 8083 ms on 10.0.0.133 (executor driver) (126/208) +26/04/01 08:41:46 INFO Executor: Running task 133.0 in stage 10.0 (TID 150) +26/04/01 08:41:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:46 INFO Executor: Finished task 127.0 in stage 10.0 (TID 144). 6606 bytes result sent to driver +26/04/01 08:41:46 INFO TaskSetManager: Starting task 134.0 in stage 10.0 (TID 151) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:46 INFO Executor: Running task 134.0 in stage 10.0 (TID 151) +26/04/01 08:41:46 INFO TaskSetManager: Finished task 127.0 in stage 10.0 (TID 144) in 7968 ms on 10.0.0.133 (executor driver) (127/208) +26/04/01 08:41:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:46 INFO Executor: Finished task 126.0 in stage 10.0 (TID 143). 6606 bytes result sent to driver +26/04/01 08:41:46 INFO TaskSetManager: Starting task 135.0 in stage 10.0 (TID 152) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:46 INFO TaskSetManager: Finished task 126.0 in stage 10.0 (TID 143) in 8172 ms on 10.0.0.133 (executor driver) (128/208) +26/04/01 08:41:46 INFO Executor: Running task 135.0 in stage 10.0 (TID 152) +26/04/01 08:41:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:53 INFO Executor: Finished task 128.0 in stage 10.0 (TID 145). 6606 bytes result sent to driver +26/04/01 08:41:53 INFO TaskSetManager: Starting task 136.0 in stage 10.0 (TID 153) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:53 INFO TaskSetManager: Finished task 128.0 in stage 10.0 (TID 145) in 7947 ms on 10.0.0.133 (executor driver) (129/208) +26/04/01 08:41:53 INFO Executor: Running task 136.0 in stage 10.0 (TID 153) +26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:53 INFO Executor: Finished task 129.0 in stage 10.0 (TID 146). 6563 bytes result sent to driver +26/04/01 08:41:53 INFO TaskSetManager: Starting task 137.0 in stage 10.0 (TID 154) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:53 INFO TaskSetManager: Finished task 129.0 in stage 10.0 (TID 146) in 7978 ms on 10.0.0.133 (executor driver) (130/208) +26/04/01 08:41:53 INFO Executor: Running task 137.0 in stage 10.0 (TID 154) +26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:53 INFO Executor: Finished task 130.0 in stage 10.0 (TID 147). 6563 bytes result sent to driver +26/04/01 08:41:53 INFO TaskSetManager: Starting task 138.0 in stage 10.0 (TID 155) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:41:53 INFO TaskSetManager: Finished task 130.0 in stage 10.0 (TID 147) in 7950 ms on 10.0.0.133 (executor driver) (131/208) +26/04/01 08:41:53 INFO Executor: Running task 138.0 in stage 10.0 (TID 155) +26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:53 INFO Executor: Finished task 131.0 in stage 10.0 (TID 148). 6563 bytes result sent to driver +26/04/01 08:41:53 INFO TaskSetManager: Starting task 139.0 in stage 10.0 (TID 156) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:41:53 INFO TaskSetManager: Finished task 131.0 in stage 10.0 (TID 148) in 7957 ms on 10.0.0.133 (executor driver) (132/208) +26/04/01 08:41:53 INFO Executor: Running task 139.0 in stage 10.0 (TID 156) +26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:53 INFO Executor: Finished task 132.0 in stage 10.0 (TID 149). 6563 bytes result sent to driver +26/04/01 08:41:53 INFO TaskSetManager: Starting task 140.0 in stage 10.0 (TID 157) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:53 INFO TaskSetManager: Finished task 132.0 in stage 10.0 (TID 149) in 7951 ms on 10.0.0.133 (executor driver) (133/208) +26/04/01 08:41:53 INFO Executor: Running task 140.0 in stage 10.0 (TID 157) +26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:54 INFO Executor: Finished task 133.0 in stage 10.0 (TID 150). 6563 bytes result sent to driver +26/04/01 08:41:54 INFO TaskSetManager: Starting task 141.0 in stage 10.0 (TID 158) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:54 INFO TaskSetManager: Finished task 133.0 in stage 10.0 (TID 150) in 7933 ms on 10.0.0.133 (executor driver) (134/208) +26/04/01 08:41:54 INFO Executor: Running task 141.0 in stage 10.0 (TID 158) +26/04/01 08:41:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:54 INFO Executor: Finished task 134.0 in stage 10.0 (TID 151). 6563 bytes result sent to driver +26/04/01 08:41:54 INFO TaskSetManager: Starting task 142.0 in stage 10.0 (TID 159) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:54 INFO TaskSetManager: Finished task 134.0 in stage 10.0 (TID 151) in 7884 ms on 10.0.0.133 (executor driver) (135/208) +26/04/01 08:41:54 INFO Executor: Running task 142.0 in stage 10.0 (TID 159) +26/04/01 08:41:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:54 INFO Executor: Finished task 135.0 in stage 10.0 (TID 152). 6563 bytes result sent to driver +26/04/01 08:41:54 INFO TaskSetManager: Starting task 143.0 in stage 10.0 (TID 160) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:41:54 INFO TaskSetManager: Finished task 135.0 in stage 10.0 (TID 152) in 7951 ms on 10.0.0.133 (executor driver) (136/208) +26/04/01 08:41:54 INFO Executor: Running task 143.0 in stage 10.0 (TID 160) +26/04/01 08:41:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:41:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:01 INFO Executor: Finished task 136.0 in stage 10.0 (TID 153). 6563 bytes result sent to driver +26/04/01 08:42:01 INFO TaskSetManager: Starting task 144.0 in stage 10.0 (TID 161) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:42:01 INFO TaskSetManager: Finished task 136.0 in stage 10.0 (TID 153) in 7945 ms on 10.0.0.133 (executor driver) (137/208) +26/04/01 08:42:01 INFO Executor: Running task 144.0 in stage 10.0 (TID 161) +26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:01 INFO Executor: Finished task 137.0 in stage 10.0 (TID 154). 6563 bytes result sent to driver +26/04/01 08:42:01 INFO TaskSetManager: Starting task 145.0 in stage 10.0 (TID 162) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:42:01 INFO TaskSetManager: Finished task 137.0 in stage 10.0 (TID 154) in 7962 ms on 10.0.0.133 (executor driver) (138/208) +26/04/01 08:42:01 INFO Executor: Running task 145.0 in stage 10.0 (TID 162) +26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:01 INFO Executor: Finished task 139.0 in stage 10.0 (TID 156). 6563 bytes result sent to driver +26/04/01 08:42:01 INFO TaskSetManager: Starting task 146.0 in stage 10.0 (TID 163) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:01 INFO TaskSetManager: Finished task 139.0 in stage 10.0 (TID 156) in 7940 ms on 10.0.0.133 (executor driver) (139/208) +26/04/01 08:42:01 INFO Executor: Running task 146.0 in stage 10.0 (TID 163) +26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:01 INFO Executor: Finished task 138.0 in stage 10.0 (TID 155). 6563 bytes result sent to driver +26/04/01 08:42:01 INFO TaskSetManager: Starting task 147.0 in stage 10.0 (TID 164) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:01 INFO TaskSetManager: Finished task 138.0 in stage 10.0 (TID 155) in 7965 ms on 10.0.0.133 (executor driver) (140/208) +26/04/01 08:42:01 INFO Executor: Running task 147.0 in stage 10.0 (TID 164) +26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:01 INFO Executor: Finished task 140.0 in stage 10.0 (TID 157). 6563 bytes result sent to driver +26/04/01 08:42:01 INFO TaskSetManager: Starting task 148.0 in stage 10.0 (TID 165) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:01 INFO TaskSetManager: Finished task 140.0 in stage 10.0 (TID 157) in 7956 ms on 10.0.0.133 (executor driver) (141/208) +26/04/01 08:42:01 INFO Executor: Running task 148.0 in stage 10.0 (TID 165) +26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:02 INFO Executor: Finished task 141.0 in stage 10.0 (TID 158). 6563 bytes result sent to driver +26/04/01 08:42:02 INFO TaskSetManager: Starting task 149.0 in stage 10.0 (TID 166) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:02 INFO TaskSetManager: Finished task 141.0 in stage 10.0 (TID 158) in 7945 ms on 10.0.0.133 (executor driver) (142/208) +26/04/01 08:42:02 INFO Executor: Running task 149.0 in stage 10.0 (TID 166) +26/04/01 08:42:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:02 INFO Executor: Finished task 142.0 in stage 10.0 (TID 159). 6563 bytes result sent to driver +26/04/01 08:42:02 INFO TaskSetManager: Starting task 150.0 in stage 10.0 (TID 167) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:42:02 INFO Executor: Running task 150.0 in stage 10.0 (TID 167) +26/04/01 08:42:02 INFO TaskSetManager: Finished task 142.0 in stage 10.0 (TID 159) in 7920 ms on 10.0.0.133 (executor driver) (143/208) +26/04/01 08:42:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:02 INFO Executor: Finished task 143.0 in stage 10.0 (TID 160). 6563 bytes result sent to driver +26/04/01 08:42:02 INFO TaskSetManager: Starting task 151.0 in stage 10.0 (TID 168) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:02 INFO Executor: Running task 151.0 in stage 10.0 (TID 168) +26/04/01 08:42:02 INFO TaskSetManager: Finished task 143.0 in stage 10.0 (TID 160) in 7924 ms on 10.0.0.133 (executor driver) (144/208) +26/04/01 08:42:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:09 INFO Executor: Finished task 144.0 in stage 10.0 (TID 161). 6563 bytes result sent to driver +26/04/01 08:42:09 INFO TaskSetManager: Starting task 152.0 in stage 10.0 (TID 169) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:09 INFO TaskSetManager: Finished task 144.0 in stage 10.0 (TID 161) in 8031 ms on 10.0.0.133 (executor driver) (145/208) +26/04/01 08:42:09 INFO Executor: Running task 152.0 in stage 10.0 (TID 169) +26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:09 INFO Executor: Finished task 145.0 in stage 10.0 (TID 162). 6563 bytes result sent to driver +26/04/01 08:42:09 INFO TaskSetManager: Starting task 153.0 in stage 10.0 (TID 170) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:09 INFO TaskSetManager: Finished task 145.0 in stage 10.0 (TID 162) in 8011 ms on 10.0.0.133 (executor driver) (146/208) +26/04/01 08:42:09 INFO Executor: Running task 153.0 in stage 10.0 (TID 170) +26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:09 INFO Executor: Finished task 147.0 in stage 10.0 (TID 164). 6563 bytes result sent to driver +26/04/01 08:42:09 INFO TaskSetManager: Starting task 154.0 in stage 10.0 (TID 171) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:09 INFO TaskSetManager: Finished task 147.0 in stage 10.0 (TID 164) in 7995 ms on 10.0.0.133 (executor driver) (147/208) +26/04/01 08:42:09 INFO Executor: Running task 154.0 in stage 10.0 (TID 171) +26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:09 INFO Executor: Finished task 146.0 in stage 10.0 (TID 163). 6563 bytes result sent to driver +26/04/01 08:42:09 INFO TaskSetManager: Starting task 155.0 in stage 10.0 (TID 172) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:09 INFO TaskSetManager: Finished task 146.0 in stage 10.0 (TID 163) in 8012 ms on 10.0.0.133 (executor driver) (148/208) +26/04/01 08:42:09 INFO Executor: Running task 155.0 in stage 10.0 (TID 172) +26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:09 INFO Executor: Finished task 148.0 in stage 10.0 (TID 165). 6563 bytes result sent to driver +26/04/01 08:42:09 INFO TaskSetManager: Starting task 156.0 in stage 10.0 (TID 173) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:42:09 INFO TaskSetManager: Finished task 148.0 in stage 10.0 (TID 165) in 7977 ms on 10.0.0.133 (executor driver) (149/208) +26/04/01 08:42:09 INFO Executor: Running task 156.0 in stage 10.0 (TID 173) +26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:10 INFO Executor: Finished task 149.0 in stage 10.0 (TID 166). 6563 bytes result sent to driver +26/04/01 08:42:10 INFO TaskSetManager: Starting task 157.0 in stage 10.0 (TID 174) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:10 INFO TaskSetManager: Finished task 149.0 in stage 10.0 (TID 166) in 7996 ms on 10.0.0.133 (executor driver) (150/208) +26/04/01 08:42:10 INFO Executor: Running task 157.0 in stage 10.0 (TID 174) +26/04/01 08:42:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:10 INFO Executor: Finished task 150.0 in stage 10.0 (TID 167). 6563 bytes result sent to driver +26/04/01 08:42:10 INFO TaskSetManager: Starting task 158.0 in stage 10.0 (TID 175) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:10 INFO TaskSetManager: Finished task 150.0 in stage 10.0 (TID 167) in 7975 ms on 10.0.0.133 (executor driver) (151/208) +26/04/01 08:42:10 INFO Executor: Running task 158.0 in stage 10.0 (TID 175) +26/04/01 08:42:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:10 INFO Executor: Finished task 151.0 in stage 10.0 (TID 168). 6563 bytes result sent to driver +26/04/01 08:42:10 INFO TaskSetManager: Starting task 159.0 in stage 10.0 (TID 176) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:10 INFO TaskSetManager: Finished task 151.0 in stage 10.0 (TID 168) in 8009 ms on 10.0.0.133 (executor driver) (152/208) +26/04/01 08:42:10 INFO Executor: Running task 159.0 in stage 10.0 (TID 176) +26/04/01 08:42:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:17 INFO Executor: Finished task 152.0 in stage 10.0 (TID 169). 6606 bytes result sent to driver +26/04/01 08:42:17 INFO TaskSetManager: Starting task 160.0 in stage 10.0 (TID 177) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:17 INFO TaskSetManager: Finished task 152.0 in stage 10.0 (TID 169) in 8030 ms on 10.0.0.133 (executor driver) (153/208) +26/04/01 08:42:17 INFO Executor: Running task 160.0 in stage 10.0 (TID 177) +26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:17 INFO Executor: Finished task 153.0 in stage 10.0 (TID 170). 6606 bytes result sent to driver +26/04/01 08:42:17 INFO TaskSetManager: Starting task 161.0 in stage 10.0 (TID 178) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:17 INFO TaskSetManager: Finished task 153.0 in stage 10.0 (TID 170) in 8009 ms on 10.0.0.133 (executor driver) (154/208) +26/04/01 08:42:17 INFO Executor: Running task 161.0 in stage 10.0 (TID 178) +26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:17 INFO Executor: Finished task 154.0 in stage 10.0 (TID 171). 6606 bytes result sent to driver +26/04/01 08:42:17 INFO TaskSetManager: Starting task 162.0 in stage 10.0 (TID 179) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:42:17 INFO TaskSetManager: Finished task 154.0 in stage 10.0 (TID 171) in 8046 ms on 10.0.0.133 (executor driver) (155/208) +26/04/01 08:42:17 INFO Executor: Running task 162.0 in stage 10.0 (TID 179) +26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:17 INFO Executor: Finished task 155.0 in stage 10.0 (TID 172). 6606 bytes result sent to driver +26/04/01 08:42:17 INFO TaskSetManager: Starting task 163.0 in stage 10.0 (TID 180) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:42:17 INFO TaskSetManager: Finished task 155.0 in stage 10.0 (TID 172) in 8048 ms on 10.0.0.133 (executor driver) (156/208) +26/04/01 08:42:17 INFO Executor: Running task 163.0 in stage 10.0 (TID 180) +26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:17 INFO Executor: Finished task 156.0 in stage 10.0 (TID 173). 6606 bytes result sent to driver +26/04/01 08:42:17 INFO TaskSetManager: Starting task 164.0 in stage 10.0 (TID 181) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:17 INFO TaskSetManager: Finished task 156.0 in stage 10.0 (TID 173) in 8036 ms on 10.0.0.133 (executor driver) (157/208) +26/04/01 08:42:17 INFO Executor: Running task 164.0 in stage 10.0 (TID 181) +26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:18 INFO Executor: Finished task 157.0 in stage 10.0 (TID 174). 6606 bytes result sent to driver +26/04/01 08:42:18 INFO TaskSetManager: Starting task 165.0 in stage 10.0 (TID 182) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:18 INFO TaskSetManager: Finished task 157.0 in stage 10.0 (TID 174) in 8078 ms on 10.0.0.133 (executor driver) (158/208) +26/04/01 08:42:18 INFO Executor: Running task 165.0 in stage 10.0 (TID 182) +26/04/01 08:42:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:18 INFO Executor: Finished task 158.0 in stage 10.0 (TID 175). 6606 bytes result sent to driver +26/04/01 08:42:18 INFO TaskSetManager: Starting task 166.0 in stage 10.0 (TID 183) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:18 INFO TaskSetManager: Finished task 158.0 in stage 10.0 (TID 175) in 8103 ms on 10.0.0.133 (executor driver) (159/208) +26/04/01 08:42:18 INFO Executor: Running task 166.0 in stage 10.0 (TID 183) +26/04/01 08:42:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:18 INFO Executor: Finished task 159.0 in stage 10.0 (TID 176). 6606 bytes result sent to driver +26/04/01 08:42:18 INFO TaskSetManager: Starting task 167.0 in stage 10.0 (TID 184) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:18 INFO Executor: Running task 167.0 in stage 10.0 (TID 184) +26/04/01 08:42:18 INFO TaskSetManager: Finished task 159.0 in stage 10.0 (TID 176) in 8053 ms on 10.0.0.133 (executor driver) (160/208) +26/04/01 08:42:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:25 INFO Executor: Finished task 160.0 in stage 10.0 (TID 177). 6563 bytes result sent to driver +26/04/01 08:42:25 INFO TaskSetManager: Starting task 168.0 in stage 10.0 (TID 185) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:42:25 INFO TaskSetManager: Finished task 160.0 in stage 10.0 (TID 177) in 8041 ms on 10.0.0.133 (executor driver) (161/208) +26/04/01 08:42:25 INFO Executor: Running task 168.0 in stage 10.0 (TID 185) +26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:25 INFO Executor: Finished task 161.0 in stage 10.0 (TID 178). 6563 bytes result sent to driver +26/04/01 08:42:25 INFO TaskSetManager: Starting task 169.0 in stage 10.0 (TID 186) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:25 INFO TaskSetManager: Finished task 161.0 in stage 10.0 (TID 178) in 8091 ms on 10.0.0.133 (executor driver) (162/208) +26/04/01 08:42:25 INFO Executor: Running task 169.0 in stage 10.0 (TID 186) +26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:25 INFO Executor: Finished task 162.0 in stage 10.0 (TID 179). 6563 bytes result sent to driver +26/04/01 08:42:25 INFO TaskSetManager: Starting task 170.0 in stage 10.0 (TID 187) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:25 INFO Executor: Running task 170.0 in stage 10.0 (TID 187) +26/04/01 08:42:25 INFO TaskSetManager: Finished task 162.0 in stage 10.0 (TID 179) in 8004 ms on 10.0.0.133 (executor driver) (163/208) +26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:25 INFO Executor: Finished task 164.0 in stage 10.0 (TID 181). 6563 bytes result sent to driver +26/04/01 08:42:25 INFO TaskSetManager: Starting task 171.0 in stage 10.0 (TID 188) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:25 INFO Executor: Finished task 163.0 in stage 10.0 (TID 180). 6563 bytes result sent to driver +26/04/01 08:42:25 INFO Executor: Running task 171.0 in stage 10.0 (TID 188) +26/04/01 08:42:25 INFO TaskSetManager: Finished task 164.0 in stage 10.0 (TID 181) in 8062 ms on 10.0.0.133 (executor driver) (164/208) +26/04/01 08:42:25 INFO TaskSetManager: Starting task 172.0 in stage 10.0 (TID 189) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:25 INFO TaskSetManager: Finished task 163.0 in stage 10.0 (TID 180) in 8082 ms on 10.0.0.133 (executor driver) (165/208) +26/04/01 08:42:25 INFO Executor: Running task 172.0 in stage 10.0 (TID 189) +26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:26 INFO Executor: Finished task 165.0 in stage 10.0 (TID 182). 6563 bytes result sent to driver +26/04/01 08:42:26 INFO TaskSetManager: Starting task 173.0 in stage 10.0 (TID 190) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:26 INFO Executor: Running task 173.0 in stage 10.0 (TID 190) +26/04/01 08:42:26 INFO TaskSetManager: Finished task 165.0 in stage 10.0 (TID 182) in 8053 ms on 10.0.0.133 (executor driver) (166/208) +26/04/01 08:42:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:26 INFO Executor: Finished task 166.0 in stage 10.0 (TID 183). 6563 bytes result sent to driver +26/04/01 08:42:26 INFO TaskSetManager: Starting task 174.0 in stage 10.0 (TID 191) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:42:26 INFO TaskSetManager: Finished task 166.0 in stage 10.0 (TID 183) in 8066 ms on 10.0.0.133 (executor driver) (167/208) +26/04/01 08:42:26 INFO Executor: Running task 174.0 in stage 10.0 (TID 191) +26/04/01 08:42:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:26 INFO Executor: Finished task 167.0 in stage 10.0 (TID 184). 6563 bytes result sent to driver +26/04/01 08:42:26 INFO TaskSetManager: Starting task 175.0 in stage 10.0 (TID 192) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:42:26 INFO Executor: Running task 175.0 in stage 10.0 (TID 192) +26/04/01 08:42:26 INFO TaskSetManager: Finished task 167.0 in stage 10.0 (TID 184) in 8074 ms on 10.0.0.133 (executor driver) (168/208) +26/04/01 08:42:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:33 INFO Executor: Finished task 168.0 in stage 10.0 (TID 185). 6563 bytes result sent to driver +26/04/01 08:42:33 INFO TaskSetManager: Starting task 176.0 in stage 10.0 (TID 193) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:33 INFO TaskSetManager: Finished task 168.0 in stage 10.0 (TID 185) in 8042 ms on 10.0.0.133 (executor driver) (169/208) +26/04/01 08:42:33 INFO Executor: Running task 176.0 in stage 10.0 (TID 193) +26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:33 INFO Executor: Finished task 169.0 in stage 10.0 (TID 186). 6563 bytes result sent to driver +26/04/01 08:42:33 INFO TaskSetManager: Starting task 177.0 in stage 10.0 (TID 194) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:33 INFO TaskSetManager: Finished task 169.0 in stage 10.0 (TID 186) in 8050 ms on 10.0.0.133 (executor driver) (170/208) +26/04/01 08:42:33 INFO Executor: Running task 177.0 in stage 10.0 (TID 194) +26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:33 INFO Executor: Finished task 170.0 in stage 10.0 (TID 187). 6563 bytes result sent to driver +26/04/01 08:42:33 INFO TaskSetManager: Starting task 178.0 in stage 10.0 (TID 195) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:33 INFO TaskSetManager: Finished task 170.0 in stage 10.0 (TID 187) in 8022 ms on 10.0.0.133 (executor driver) (171/208) +26/04/01 08:42:33 INFO Executor: Running task 178.0 in stage 10.0 (TID 195) +26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:33 INFO Executor: Finished task 172.0 in stage 10.0 (TID 189). 6563 bytes result sent to driver +26/04/01 08:42:33 INFO TaskSetManager: Starting task 179.0 in stage 10.0 (TID 196) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:33 INFO TaskSetManager: Finished task 172.0 in stage 10.0 (TID 189) in 8050 ms on 10.0.0.133 (executor driver) (172/208) +26/04/01 08:42:33 INFO Executor: Running task 179.0 in stage 10.0 (TID 196) +26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:33 INFO Executor: Finished task 171.0 in stage 10.0 (TID 188). 6563 bytes result sent to driver +26/04/01 08:42:33 INFO TaskSetManager: Starting task 180.0 in stage 10.0 (TID 197) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:42:33 INFO TaskSetManager: Finished task 171.0 in stage 10.0 (TID 188) in 8109 ms on 10.0.0.133 (executor driver) (173/208) +26/04/01 08:42:33 INFO Executor: Running task 180.0 in stage 10.0 (TID 197) +26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:34 INFO Executor: Finished task 173.0 in stage 10.0 (TID 190). 6563 bytes result sent to driver +26/04/01 08:42:34 INFO TaskSetManager: Starting task 181.0 in stage 10.0 (TID 198) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:34 INFO TaskSetManager: Finished task 173.0 in stage 10.0 (TID 190) in 8109 ms on 10.0.0.133 (executor driver) (174/208) +26/04/01 08:42:34 INFO Executor: Running task 181.0 in stage 10.0 (TID 198) +26/04/01 08:42:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:34 INFO Executor: Finished task 174.0 in stage 10.0 (TID 191). 6563 bytes result sent to driver +26/04/01 08:42:34 INFO TaskSetManager: Starting task 182.0 in stage 10.0 (TID 199) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:34 INFO TaskSetManager: Finished task 174.0 in stage 10.0 (TID 191) in 8140 ms on 10.0.0.133 (executor driver) (175/208) +26/04/01 08:42:34 INFO Executor: Running task 182.0 in stage 10.0 (TID 199) +26/04/01 08:42:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:34 INFO Executor: Finished task 175.0 in stage 10.0 (TID 192). 6563 bytes result sent to driver +26/04/01 08:42:34 INFO TaskSetManager: Starting task 183.0 in stage 10.0 (TID 200) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:34 INFO Executor: Running task 183.0 in stage 10.0 (TID 200) +26/04/01 08:42:34 INFO TaskSetManager: Finished task 175.0 in stage 10.0 (TID 192) in 8171 ms on 10.0.0.133 (executor driver) (176/208) +26/04/01 08:42:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:41 INFO Executor: Finished task 176.0 in stage 10.0 (TID 193). 6563 bytes result sent to driver +26/04/01 08:42:41 INFO TaskSetManager: Starting task 184.0 in stage 10.0 (TID 201) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:41 INFO TaskSetManager: Finished task 176.0 in stage 10.0 (TID 193) in 7973 ms on 10.0.0.133 (executor driver) (177/208) +26/04/01 08:42:41 INFO Executor: Running task 184.0 in stage 10.0 (TID 201) +26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:41 INFO Executor: Finished task 177.0 in stage 10.0 (TID 194). 6563 bytes result sent to driver +26/04/01 08:42:41 INFO TaskSetManager: Starting task 185.0 in stage 10.0 (TID 202) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:41 INFO Executor: Running task 185.0 in stage 10.0 (TID 202) +26/04/01 08:42:41 INFO TaskSetManager: Finished task 177.0 in stage 10.0 (TID 194) in 7972 ms on 10.0.0.133 (executor driver) (178/208) +26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:41 INFO Executor: Finished task 178.0 in stage 10.0 (TID 195). 6563 bytes result sent to driver +26/04/01 08:42:41 INFO TaskSetManager: Starting task 186.0 in stage 10.0 (TID 203) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:42:41 INFO TaskSetManager: Finished task 178.0 in stage 10.0 (TID 195) in 7978 ms on 10.0.0.133 (executor driver) (179/208) +26/04/01 08:42:41 INFO Executor: Running task 186.0 in stage 10.0 (TID 203) +26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:41 INFO Executor: Finished task 179.0 in stage 10.0 (TID 196). 6563 bytes result sent to driver +26/04/01 08:42:41 INFO TaskSetManager: Starting task 187.0 in stage 10.0 (TID 204) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:42:41 INFO TaskSetManager: Finished task 179.0 in stage 10.0 (TID 196) in 7969 ms on 10.0.0.133 (executor driver) (180/208) +26/04/01 08:42:41 INFO Executor: Running task 187.0 in stage 10.0 (TID 204) +26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:41 INFO Executor: Finished task 180.0 in stage 10.0 (TID 197). 6563 bytes result sent to driver +26/04/01 08:42:41 INFO TaskSetManager: Starting task 188.0 in stage 10.0 (TID 205) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:41 INFO Executor: Running task 188.0 in stage 10.0 (TID 205) +26/04/01 08:42:41 INFO TaskSetManager: Finished task 180.0 in stage 10.0 (TID 197) in 7961 ms on 10.0.0.133 (executor driver) (181/208) +26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:42 INFO Executor: Finished task 181.0 in stage 10.0 (TID 198). 6563 bytes result sent to driver +26/04/01 08:42:42 INFO TaskSetManager: Starting task 189.0 in stage 10.0 (TID 206) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:42 INFO TaskSetManager: Finished task 181.0 in stage 10.0 (TID 198) in 7980 ms on 10.0.0.133 (executor driver) (182/208) +26/04/01 08:42:42 INFO Executor: Running task 189.0 in stage 10.0 (TID 206) +26/04/01 08:42:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:42 INFO Executor: Finished task 182.0 in stage 10.0 (TID 199). 6563 bytes result sent to driver +26/04/01 08:42:42 INFO TaskSetManager: Starting task 190.0 in stage 10.0 (TID 207) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:42 INFO Executor: Running task 190.0 in stage 10.0 (TID 207) +26/04/01 08:42:42 INFO TaskSetManager: Finished task 182.0 in stage 10.0 (TID 199) in 7992 ms on 10.0.0.133 (executor driver) (183/208) +26/04/01 08:42:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:42 INFO Executor: Finished task 183.0 in stage 10.0 (TID 200). 6563 bytes result sent to driver +26/04/01 08:42:42 INFO TaskSetManager: Starting task 191.0 in stage 10.0 (TID 208) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:42:42 INFO TaskSetManager: Finished task 183.0 in stage 10.0 (TID 200) in 7964 ms on 10.0.0.133 (executor driver) (184/208) +26/04/01 08:42:42 INFO Executor: Running task 191.0 in stage 10.0 (TID 208) +26/04/01 08:42:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:49 INFO Executor: Finished task 184.0 in stage 10.0 (TID 201). 6606 bytes result sent to driver +26/04/01 08:42:49 INFO TaskSetManager: Starting task 192.0 in stage 10.0 (TID 209) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:42:49 INFO TaskSetManager: Finished task 184.0 in stage 10.0 (TID 201) in 7976 ms on 10.0.0.133 (executor driver) (185/208) +26/04/01 08:42:49 INFO Executor: Running task 192.0 in stage 10.0 (TID 209) +26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:49 INFO Executor: Finished task 185.0 in stage 10.0 (TID 202). 6606 bytes result sent to driver +26/04/01 08:42:49 INFO TaskSetManager: Starting task 193.0 in stage 10.0 (TID 210) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9346 bytes) +26/04/01 08:42:49 INFO TaskSetManager: Finished task 185.0 in stage 10.0 (TID 202) in 7974 ms on 10.0.0.133 (executor driver) (186/208) +26/04/01 08:42:49 INFO Executor: Running task 193.0 in stage 10.0 (TID 210) +26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:49 INFO Executor: Finished task 186.0 in stage 10.0 (TID 203). 6606 bytes result sent to driver +26/04/01 08:42:49 INFO TaskSetManager: Starting task 194.0 in stage 10.0 (TID 211) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:42:49 INFO Executor: Running task 194.0 in stage 10.0 (TID 211) +26/04/01 08:42:49 INFO TaskSetManager: Finished task 186.0 in stage 10.0 (TID 203) in 7969 ms on 10.0.0.133 (executor driver) (187/208) +26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:49 INFO Executor: Finished task 187.0 in stage 10.0 (TID 204). 6606 bytes result sent to driver +26/04/01 08:42:49 INFO TaskSetManager: Starting task 195.0 in stage 10.0 (TID 212) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:42:49 INFO TaskSetManager: Finished task 187.0 in stage 10.0 (TID 204) in 7994 ms on 10.0.0.133 (executor driver) (188/208) +26/04/01 08:42:49 INFO Executor: Running task 195.0 in stage 10.0 (TID 212) +26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:49 INFO Executor: Finished task 188.0 in stage 10.0 (TID 205). 6606 bytes result sent to driver +26/04/01 08:42:49 INFO TaskSetManager: Starting task 196.0 in stage 10.0 (TID 213) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9346 bytes) +26/04/01 08:42:49 INFO TaskSetManager: Finished task 188.0 in stage 10.0 (TID 205) in 7969 ms on 10.0.0.133 (executor driver) (189/208) +26/04/01 08:42:49 INFO Executor: Running task 196.0 in stage 10.0 (TID 213) +26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:50 INFO Executor: Finished task 189.0 in stage 10.0 (TID 206). 6606 bytes result sent to driver +26/04/01 08:42:50 INFO TaskSetManager: Starting task 197.0 in stage 10.0 (TID 214) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9346 bytes) +26/04/01 08:42:50 INFO TaskSetManager: Finished task 189.0 in stage 10.0 (TID 206) in 7975 ms on 10.0.0.133 (executor driver) (190/208) +26/04/01 08:42:50 INFO Executor: Running task 197.0 in stage 10.0 (TID 214) +26/04/01 08:42:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:50 INFO Executor: Finished task 190.0 in stage 10.0 (TID 207). 6606 bytes result sent to driver +26/04/01 08:42:50 INFO TaskSetManager: Starting task 198.0 in stage 10.0 (TID 215) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:42:50 INFO Executor: Running task 198.0 in stage 10.0 (TID 215) +26/04/01 08:42:50 INFO TaskSetManager: Finished task 190.0 in stage 10.0 (TID 207) in 7999 ms on 10.0.0.133 (executor driver) (191/208) +26/04/01 08:42:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:50 INFO Executor: Finished task 191.0 in stage 10.0 (TID 208). 6606 bytes result sent to driver +26/04/01 08:42:50 INFO TaskSetManager: Starting task 199.0 in stage 10.0 (TID 216) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:42:50 INFO TaskSetManager: Finished task 191.0 in stage 10.0 (TID 208) in 7965 ms on 10.0.0.133 (executor driver) (192/208) +26/04/01 08:42:50 INFO Executor: Running task 199.0 in stage 10.0 (TID 216) +26/04/01 08:42:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:54 INFO Executor: Finished task 192.0 in stage 10.0 (TID 209). 6563 bytes result sent to driver +26/04/01 08:42:54 INFO TaskSetManager: Starting task 200.0 in stage 10.0 (TID 217) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9345 bytes) +26/04/01 08:42:54 INFO TaskSetManager: Finished task 192.0 in stage 10.0 (TID 209) in 4979 ms on 10.0.0.133 (executor driver) (193/208) +26/04/01 08:42:54 INFO Executor: Running task 200.0 in stage 10.0 (TID 217) +26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:54 INFO Executor: Finished task 193.0 in stage 10.0 (TID 210). 6563 bytes result sent to driver +26/04/01 08:42:54 INFO TaskSetManager: Starting task 201.0 in stage 10.0 (TID 218) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9346 bytes) +26/04/01 08:42:54 INFO TaskSetManager: Finished task 193.0 in stage 10.0 (TID 210) in 4958 ms on 10.0.0.133 (executor driver) (194/208) +26/04/01 08:42:54 INFO Executor: Running task 201.0 in stage 10.0 (TID 218) +26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:54 INFO Executor: Finished task 194.0 in stage 10.0 (TID 211). 6563 bytes result sent to driver +26/04/01 08:42:54 INFO TaskSetManager: Starting task 202.0 in stage 10.0 (TID 219) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9346 bytes) +26/04/01 08:42:54 INFO TaskSetManager: Finished task 194.0 in stage 10.0 (TID 211) in 4952 ms on 10.0.0.133 (executor driver) (195/208) +26/04/01 08:42:54 INFO Executor: Running task 202.0 in stage 10.0 (TID 219) +26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:54 INFO Executor: Finished task 195.0 in stage 10.0 (TID 212). 6563 bytes result sent to driver +26/04/01 08:42:54 INFO TaskSetManager: Starting task 203.0 in stage 10.0 (TID 220) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:42:54 INFO TaskSetManager: Finished task 195.0 in stage 10.0 (TID 212) in 4956 ms on 10.0.0.133 (executor driver) (196/208) +26/04/01 08:42:54 INFO Executor: Running task 203.0 in stage 10.0 (TID 220) +26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:54 INFO Executor: Finished task 196.0 in stage 10.0 (TID 213). 6563 bytes result sent to driver +26/04/01 08:42:54 INFO TaskSetManager: Starting task 204.0 in stage 10.0 (TID 221) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:42:54 INFO TaskSetManager: Finished task 196.0 in stage 10.0 (TID 213) in 4950 ms on 10.0.0.133 (executor driver) (197/208) +26/04/01 08:42:54 INFO Executor: Running task 204.0 in stage 10.0 (TID 221) +26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:55 INFO Executor: Finished task 197.0 in stage 10.0 (TID 214). 6563 bytes result sent to driver +26/04/01 08:42:55 INFO TaskSetManager: Starting task 205.0 in stage 10.0 (TID 222) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:42:55 INFO TaskSetManager: Finished task 197.0 in stage 10.0 (TID 214) in 4923 ms on 10.0.0.133 (executor driver) (198/208) +26/04/01 08:42:55 INFO Executor: Running task 205.0 in stage 10.0 (TID 222) +26/04/01 08:42:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:55 INFO Executor: Finished task 198.0 in stage 10.0 (TID 215). 6563 bytes result sent to driver +26/04/01 08:42:55 INFO TaskSetManager: Starting task 206.0 in stage 10.0 (TID 223) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9346 bytes) +26/04/01 08:42:55 INFO TaskSetManager: Finished task 198.0 in stage 10.0 (TID 215) in 4924 ms on 10.0.0.133 (executor driver) (199/208) +26/04/01 08:42:55 INFO Executor: Running task 206.0 in stage 10.0 (TID 223) +26/04/01 08:42:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:55 INFO Executor: Finished task 199.0 in stage 10.0 (TID 216). 6563 bytes result sent to driver +26/04/01 08:42:55 INFO TaskSetManager: Starting task 207.0 in stage 10.0 (TID 224) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9346 bytes) +26/04/01 08:42:55 INFO Executor: Running task 207.0 in stage 10.0 (TID 224) +26/04/01 08:42:55 INFO TaskSetManager: Finished task 199.0 in stage 10.0 (TID 216) in 4924 ms on 10.0.0.133 (executor driver) (200/208) +26/04/01 08:42:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO Executor: Finished task 200.0 in stage 10.0 (TID 217). 6563 bytes result sent to driver +26/04/01 08:42:59 INFO TaskSetManager: Starting task 0.0 in stage 11.0 (TID 225) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9463 bytes) +26/04/01 08:42:59 INFO TaskSetManager: Finished task 200.0 in stage 10.0 (TID 217) in 4938 ms on 10.0.0.133 (executor driver) (201/208) +26/04/01 08:42:59 INFO Executor: Running task 0.0 in stage 11.0 (TID 225) +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO Executor: Finished task 201.0 in stage 10.0 (TID 218). 6563 bytes result sent to driver +26/04/01 08:42:59 INFO TaskSetManager: Starting task 1.0 in stage 11.0 (TID 226) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9464 bytes) +26/04/01 08:42:59 INFO TaskSetManager: Finished task 201.0 in stage 10.0 (TID 218) in 4941 ms on 10.0.0.133 (executor driver) (202/208) +26/04/01 08:42:59 INFO Executor: Running task 1.0 in stage 11.0 (TID 226) +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO Executor: Finished task 0.0 in stage 11.0 (TID 225). 6563 bytes result sent to driver +26/04/01 08:42:59 INFO TaskSetManager: Starting task 2.0 in stage 11.0 (TID 227) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9464 bytes) +26/04/01 08:42:59 INFO TaskSetManager: Finished task 0.0 in stage 11.0 (TID 225) in 106 ms on 10.0.0.133 (executor driver) (1/8) +26/04/01 08:42:59 INFO Executor: Running task 2.0 in stage 11.0 (TID 227) +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO Executor: Finished task 202.0 in stage 10.0 (TID 219). 6563 bytes result sent to driver +26/04/01 08:42:59 INFO TaskSetManager: Starting task 3.0 in stage 11.0 (TID 228) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9465 bytes) +26/04/01 08:42:59 INFO TaskSetManager: Finished task 202.0 in stage 10.0 (TID 219) in 4932 ms on 10.0.0.133 (executor driver) (203/208) +26/04/01 08:42:59 INFO Executor: Running task 3.0 in stage 11.0 (TID 228) +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO Executor: Finished task 1.0 in stage 11.0 (TID 226). 6563 bytes result sent to driver +26/04/01 08:42:59 INFO TaskSetManager: Starting task 4.0 in stage 11.0 (TID 229) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9465 bytes) +26/04/01 08:42:59 INFO TaskSetManager: Finished task 1.0 in stage 11.0 (TID 226) in 109 ms on 10.0.0.133 (executor driver) (2/8) +26/04/01 08:42:59 INFO Executor: Running task 4.0 in stage 11.0 (TID 229) +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO Executor: Finished task 2.0 in stage 11.0 (TID 227). 6563 bytes result sent to driver +26/04/01 08:42:59 INFO TaskSetManager: Starting task 5.0 in stage 11.0 (TID 230) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9464 bytes) +26/04/01 08:42:59 INFO Executor: Running task 5.0 in stage 11.0 (TID 230) +26/04/01 08:42:59 INFO TaskSetManager: Finished task 2.0 in stage 11.0 (TID 227) in 107 ms on 10.0.0.133 (executor driver) (3/8) +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO Executor: Finished task 3.0 in stage 11.0 (TID 228). 6563 bytes result sent to driver +26/04/01 08:42:59 INFO TaskSetManager: Starting task 6.0 in stage 11.0 (TID 231) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9463 bytes) +26/04/01 08:42:59 INFO TaskSetManager: Finished task 3.0 in stage 11.0 (TID 228) in 109 ms on 10.0.0.133 (executor driver) (4/8) +26/04/01 08:42:59 INFO Executor: Running task 6.0 in stage 11.0 (TID 231) +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO Executor: Finished task 203.0 in stage 10.0 (TID 220). 6563 bytes result sent to driver +26/04/01 08:42:59 INFO TaskSetManager: Starting task 7.0 in stage 11.0 (TID 232) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9463 bytes) +26/04/01 08:42:59 INFO TaskSetManager: Finished task 203.0 in stage 10.0 (TID 220) in 4931 ms on 10.0.0.133 (executor driver) (204/208) +26/04/01 08:42:59 INFO Executor: Running task 7.0 in stage 11.0 (TID 232) +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO Executor: Finished task 204.0 in stage 10.0 (TID 221). 6563 bytes result sent to driver +26/04/01 08:42:59 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 233) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:42:59 INFO Executor: Running task 0.0 in stage 12.0 (TID 233) +26/04/01 08:42:59 INFO TaskSetManager: Finished task 204.0 in stage 10.0 (TID 221) in 4933 ms on 10.0.0.133 (executor driver) (205/208) +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO Executor: Finished task 4.0 in stage 11.0 (TID 229). 6563 bytes result sent to driver +26/04/01 08:42:59 INFO TaskSetManager: Starting task 1.0 in stage 12.0 (TID 234) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:42:59 INFO Executor: Running task 1.0 in stage 12.0 (TID 234) +26/04/01 08:42:59 INFO TaskSetManager: Finished task 4.0 in stage 11.0 (TID 229) in 111 ms on 10.0.0.133 (executor driver) (5/8) +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO Executor: Finished task 5.0 in stage 11.0 (TID 230). 6563 bytes result sent to driver +26/04/01 08:42:59 INFO TaskSetManager: Starting task 2.0 in stage 12.0 (TID 235) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:42:59 INFO TaskSetManager: Finished task 5.0 in stage 11.0 (TID 230) in 111 ms on 10.0.0.133 (executor driver) (6/8) +26/04/01 08:42:59 INFO Executor: Running task 2.0 in stage 12.0 (TID 235) +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO Executor: Finished task 6.0 in stage 11.0 (TID 231). 6563 bytes result sent to driver +26/04/01 08:42:59 INFO TaskSetManager: Starting task 3.0 in stage 12.0 (TID 236) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:42:59 INFO TaskSetManager: Finished task 6.0 in stage 11.0 (TID 231) in 111 ms on 10.0.0.133 (executor driver) (7/8) +26/04/01 08:42:59 INFO Executor: Running task 3.0 in stage 12.0 (TID 236) +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO Executor: Finished task 7.0 in stage 11.0 (TID 232). 6563 bytes result sent to driver +26/04/01 08:42:59 INFO TaskSetManager: Starting task 4.0 in stage 12.0 (TID 237) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:42:59 INFO TaskSetManager: Finished task 7.0 in stage 11.0 (TID 232) in 114 ms on 10.0.0.133 (executor driver) (8/8) +26/04/01 08:42:59 INFO Executor: Running task 4.0 in stage 12.0 (TID 237) +26/04/01 08:42:59 INFO TaskSchedulerImpl: Removed TaskSet 11.0, whose tasks have all completed, from pool +26/04/01 08:42:59 INFO DAGScheduler: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 203.765 s +26/04/01 08:42:59 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:42:59 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 12, ShuffleMapStage 13, ShuffleMapStage 10) +26/04/01 08:42:59 INFO DAGScheduler: waiting: Set() +26/04/01 08:42:59 INFO DAGScheduler: failed: Set() +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:00 INFO Executor: Finished task 205.0 in stage 10.0 (TID 222). 6563 bytes result sent to driver +26/04/01 08:43:00 INFO TaskSetManager: Starting task 5.0 in stage 12.0 (TID 238) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:00 INFO TaskSetManager: Finished task 205.0 in stage 10.0 (TID 222) in 4909 ms on 10.0.0.133 (executor driver) (206/208) +26/04/01 08:43:00 INFO Executor: Running task 5.0 in stage 12.0 (TID 238) +26/04/01 08:43:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:00 INFO Executor: Finished task 206.0 in stage 10.0 (TID 223). 6563 bytes result sent to driver +26/04/01 08:43:00 INFO TaskSetManager: Starting task 6.0 in stage 12.0 (TID 239) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:00 INFO TaskSetManager: Finished task 206.0 in stage 10.0 (TID 223) in 4920 ms on 10.0.0.133 (executor driver) (207/208) +26/04/01 08:43:00 INFO Executor: Running task 6.0 in stage 12.0 (TID 239) +26/04/01 08:43:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:00 INFO Executor: Finished task 207.0 in stage 10.0 (TID 224). 6563 bytes result sent to driver +26/04/01 08:43:00 INFO TaskSetManager: Starting task 7.0 in stage 12.0 (TID 240) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:00 INFO Executor: Running task 7.0 in stage 12.0 (TID 240) +26/04/01 08:43:00 INFO TaskSetManager: Finished task 207.0 in stage 10.0 (TID 224) in 4944 ms on 10.0.0.133 (executor driver) (208/208) +26/04/01 08:43:00 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool +26/04/01 08:43:00 INFO DAGScheduler: ShuffleMapStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 204.203 s +26/04/01 08:43:00 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:43:00 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 12, ShuffleMapStage 13) +26/04/01 08:43:00 INFO DAGScheduler: waiting: Set() +26/04/01 08:43:00 INFO DAGScheduler: failed: Set() +26/04/01 08:43:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO Executor: Finished task 0.0 in stage 12.0 (TID 233). 6563 bytes result sent to driver +26/04/01 08:43:03 INFO TaskSetManager: Starting task 8.0 in stage 12.0 (TID 241) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:03 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 233) in 3395 ms on 10.0.0.133 (executor driver) (1/37) +26/04/01 08:43:03 INFO Executor: Running task 8.0 in stage 12.0 (TID 241) +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO Executor: Finished task 1.0 in stage 12.0 (TID 234). 6563 bytes result sent to driver +26/04/01 08:43:03 INFO TaskSetManager: Starting task 9.0 in stage 12.0 (TID 242) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:03 INFO Executor: Running task 9.0 in stage 12.0 (TID 242) +26/04/01 08:43:03 INFO TaskSetManager: Finished task 1.0 in stage 12.0 (TID 234) in 3379 ms on 10.0.0.133 (executor driver) (2/37) +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO Executor: Finished task 2.0 in stage 12.0 (TID 235). 6563 bytes result sent to driver +26/04/01 08:43:03 INFO TaskSetManager: Starting task 10.0 in stage 12.0 (TID 243) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:03 INFO TaskSetManager: Finished task 2.0 in stage 12.0 (TID 235) in 3393 ms on 10.0.0.133 (executor driver) (3/37) +26/04/01 08:43:03 INFO Executor: Running task 10.0 in stage 12.0 (TID 243) +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO Executor: Finished task 3.0 in stage 12.0 (TID 236). 6563 bytes result sent to driver +26/04/01 08:43:03 INFO TaskSetManager: Starting task 11.0 in stage 12.0 (TID 244) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:03 INFO TaskSetManager: Finished task 3.0 in stage 12.0 (TID 236) in 3388 ms on 10.0.0.133 (executor driver) (4/37) +26/04/01 08:43:03 INFO Executor: Running task 11.0 in stage 12.0 (TID 244) +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO Executor: Finished task 4.0 in stage 12.0 (TID 237). 6563 bytes result sent to driver +26/04/01 08:43:03 INFO TaskSetManager: Starting task 12.0 in stage 12.0 (TID 245) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:03 INFO TaskSetManager: Finished task 4.0 in stage 12.0 (TID 237) in 3388 ms on 10.0.0.133 (executor driver) (5/37) +26/04/01 08:43:03 INFO Executor: Running task 12.0 in stage 12.0 (TID 245) +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO Executor: Finished task 5.0 in stage 12.0 (TID 238). 6563 bytes result sent to driver +26/04/01 08:43:03 INFO TaskSetManager: Starting task 13.0 in stage 12.0 (TID 246) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:03 INFO TaskSetManager: Finished task 5.0 in stage 12.0 (TID 238) in 3380 ms on 10.0.0.133 (executor driver) (6/37) +26/04/01 08:43:03 INFO Executor: Running task 13.0 in stage 12.0 (TID 246) +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO Executor: Finished task 6.0 in stage 12.0 (TID 239). 6563 bytes result sent to driver +26/04/01 08:43:03 INFO TaskSetManager: Starting task 14.0 in stage 12.0 (TID 247) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:03 INFO TaskSetManager: Finished task 6.0 in stage 12.0 (TID 239) in 3364 ms on 10.0.0.133 (executor driver) (7/37) +26/04/01 08:43:03 INFO Executor: Running task 14.0 in stage 12.0 (TID 247) +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO Executor: Finished task 7.0 in stage 12.0 (TID 240). 6606 bytes result sent to driver +26/04/01 08:43:03 INFO TaskSetManager: Starting task 15.0 in stage 12.0 (TID 248) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:03 INFO TaskSetManager: Finished task 7.0 in stage 12.0 (TID 240) in 3386 ms on 10.0.0.133 (executor driver) (8/37) +26/04/01 08:43:03 INFO Executor: Running task 15.0 in stage 12.0 (TID 248) +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:06 INFO Executor: Finished task 8.0 in stage 12.0 (TID 241). 6563 bytes result sent to driver +26/04/01 08:43:06 INFO TaskSetManager: Starting task 16.0 in stage 12.0 (TID 249) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:06 INFO TaskSetManager: Finished task 8.0 in stage 12.0 (TID 241) in 3383 ms on 10.0.0.133 (executor driver) (9/37) +26/04/01 08:43:06 INFO Executor: Running task 16.0 in stage 12.0 (TID 249) +26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:06 INFO Executor: Finished task 9.0 in stage 12.0 (TID 242). 6563 bytes result sent to driver +26/04/01 08:43:06 INFO TaskSetManager: Starting task 17.0 in stage 12.0 (TID 250) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:06 INFO TaskSetManager: Finished task 9.0 in stage 12.0 (TID 242) in 3393 ms on 10.0.0.133 (executor driver) (10/37) +26/04/01 08:43:06 INFO Executor: Running task 17.0 in stage 12.0 (TID 250) +26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:06 INFO Executor: Finished task 10.0 in stage 12.0 (TID 243). 6563 bytes result sent to driver +26/04/01 08:43:06 INFO TaskSetManager: Starting task 18.0 in stage 12.0 (TID 251) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:06 INFO TaskSetManager: Finished task 10.0 in stage 12.0 (TID 243) in 3410 ms on 10.0.0.133 (executor driver) (11/37) +26/04/01 08:43:06 INFO Executor: Running task 18.0 in stage 12.0 (TID 251) +26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:06 INFO Executor: Finished task 11.0 in stage 12.0 (TID 244). 6563 bytes result sent to driver +26/04/01 08:43:06 INFO TaskSetManager: Starting task 19.0 in stage 12.0 (TID 252) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:06 INFO TaskSetManager: Finished task 11.0 in stage 12.0 (TID 244) in 3418 ms on 10.0.0.133 (executor driver) (12/37) +26/04/01 08:43:06 INFO Executor: Running task 19.0 in stage 12.0 (TID 252) +26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:06 INFO Executor: Finished task 12.0 in stage 12.0 (TID 245). 6563 bytes result sent to driver +26/04/01 08:43:06 INFO TaskSetManager: Starting task 20.0 in stage 12.0 (TID 253) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:06 INFO TaskSetManager: Finished task 12.0 in stage 12.0 (TID 245) in 3442 ms on 10.0.0.133 (executor driver) (13/37) +26/04/01 08:43:06 INFO Executor: Running task 20.0 in stage 12.0 (TID 253) +26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:06 INFO Executor: Finished task 13.0 in stage 12.0 (TID 246). 6563 bytes result sent to driver +26/04/01 08:43:06 INFO TaskSetManager: Starting task 21.0 in stage 12.0 (TID 254) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:06 INFO TaskSetManager: Finished task 13.0 in stage 12.0 (TID 246) in 3420 ms on 10.0.0.133 (executor driver) (14/37) +26/04/01 08:43:06 INFO Executor: Running task 21.0 in stage 12.0 (TID 254) +26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:06 INFO Executor: Finished task 14.0 in stage 12.0 (TID 247). 6563 bytes result sent to driver +26/04/01 08:43:06 INFO TaskSetManager: Starting task 22.0 in stage 12.0 (TID 255) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:06 INFO TaskSetManager: Finished task 14.0 in stage 12.0 (TID 247) in 3407 ms on 10.0.0.133 (executor driver) (15/37) +26/04/01 08:43:06 INFO Executor: Running task 22.0 in stage 12.0 (TID 255) +26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:07 INFO Executor: Finished task 15.0 in stage 12.0 (TID 248). 6563 bytes result sent to driver +26/04/01 08:43:07 INFO TaskSetManager: Starting task 23.0 in stage 12.0 (TID 256) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:07 INFO TaskSetManager: Finished task 15.0 in stage 12.0 (TID 248) in 3408 ms on 10.0.0.133 (executor driver) (16/37) +26/04/01 08:43:07 INFO Executor: Running task 23.0 in stage 12.0 (TID 256) +26/04/01 08:43:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:09 INFO Executor: Finished task 16.0 in stage 12.0 (TID 249). 6563 bytes result sent to driver +26/04/01 08:43:09 INFO TaskSetManager: Starting task 24.0 in stage 12.0 (TID 257) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:09 INFO TaskSetManager: Finished task 16.0 in stage 12.0 (TID 249) in 3324 ms on 10.0.0.133 (executor driver) (17/37) +26/04/01 08:43:09 INFO Executor: Running task 24.0 in stage 12.0 (TID 257) +26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:09 INFO Executor: Finished task 17.0 in stage 12.0 (TID 250). 6563 bytes result sent to driver +26/04/01 08:43:09 INFO TaskSetManager: Starting task 25.0 in stage 12.0 (TID 258) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:09 INFO TaskSetManager: Finished task 17.0 in stage 12.0 (TID 250) in 3314 ms on 10.0.0.133 (executor driver) (18/37) +26/04/01 08:43:09 INFO Executor: Running task 25.0 in stage 12.0 (TID 258) +26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:09 INFO Executor: Finished task 18.0 in stage 12.0 (TID 251). 6563 bytes result sent to driver +26/04/01 08:43:09 INFO TaskSetManager: Starting task 26.0 in stage 12.0 (TID 259) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:09 INFO TaskSetManager: Finished task 18.0 in stage 12.0 (TID 251) in 3334 ms on 10.0.0.133 (executor driver) (19/37) +26/04/01 08:43:09 INFO Executor: Running task 26.0 in stage 12.0 (TID 259) +26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:09 INFO Executor: Finished task 19.0 in stage 12.0 (TID 252). 6563 bytes result sent to driver +26/04/01 08:43:09 INFO TaskSetManager: Starting task 27.0 in stage 12.0 (TID 260) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:09 INFO TaskSetManager: Finished task 19.0 in stage 12.0 (TID 252) in 3335 ms on 10.0.0.133 (executor driver) (20/37) +26/04/01 08:43:09 INFO Executor: Running task 27.0 in stage 12.0 (TID 260) +26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:10 INFO Executor: Finished task 20.0 in stage 12.0 (TID 253). 6563 bytes result sent to driver +26/04/01 08:43:10 INFO TaskSetManager: Starting task 28.0 in stage 12.0 (TID 261) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:10 INFO TaskSetManager: Finished task 20.0 in stage 12.0 (TID 253) in 3328 ms on 10.0.0.133 (executor driver) (21/37) +26/04/01 08:43:10 INFO Executor: Running task 28.0 in stage 12.0 (TID 261) +26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:10 INFO Executor: Finished task 21.0 in stage 12.0 (TID 254). 6563 bytes result sent to driver +26/04/01 08:43:10 INFO TaskSetManager: Starting task 29.0 in stage 12.0 (TID 262) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:10 INFO TaskSetManager: Finished task 21.0 in stage 12.0 (TID 254) in 3329 ms on 10.0.0.133 (executor driver) (22/37) +26/04/01 08:43:10 INFO Executor: Running task 29.0 in stage 12.0 (TID 262) +26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:10 INFO Executor: Finished task 22.0 in stage 12.0 (TID 255). 6563 bytes result sent to driver +26/04/01 08:43:10 INFO TaskSetManager: Starting task 30.0 in stage 12.0 (TID 263) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:10 INFO TaskSetManager: Finished task 22.0 in stage 12.0 (TID 255) in 3349 ms on 10.0.0.133 (executor driver) (23/37) +26/04/01 08:43:10 INFO Executor: Running task 30.0 in stage 12.0 (TID 263) +26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:10 INFO Executor: Finished task 23.0 in stage 12.0 (TID 256). 6563 bytes result sent to driver +26/04/01 08:43:10 INFO TaskSetManager: Starting task 31.0 in stage 12.0 (TID 264) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:10 INFO TaskSetManager: Finished task 23.0 in stage 12.0 (TID 256) in 3324 ms on 10.0.0.133 (executor driver) (24/37) +26/04/01 08:43:10 INFO Executor: Running task 31.0 in stage 12.0 (TID 264) +26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO Executor: Finished task 24.0 in stage 12.0 (TID 257). 6563 bytes result sent to driver +26/04/01 08:43:13 INFO TaskSetManager: Starting task 32.0 in stage 12.0 (TID 265) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9694 bytes) +26/04/01 08:43:13 INFO Executor: Running task 32.0 in stage 12.0 (TID 265) +26/04/01 08:43:13 INFO TaskSetManager: Finished task 24.0 in stage 12.0 (TID 257) in 3350 ms on 10.0.0.133 (executor driver) (25/37) +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO Executor: Finished task 25.0 in stage 12.0 (TID 258). 6563 bytes result sent to driver +26/04/01 08:43:13 INFO TaskSetManager: Starting task 33.0 in stage 12.0 (TID 266) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9695 bytes) +26/04/01 08:43:13 INFO TaskSetManager: Finished task 25.0 in stage 12.0 (TID 258) in 3344 ms on 10.0.0.133 (executor driver) (26/37) +26/04/01 08:43:13 INFO Executor: Running task 33.0 in stage 12.0 (TID 266) +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO Executor: Finished task 26.0 in stage 12.0 (TID 259). 6563 bytes result sent to driver +26/04/01 08:43:13 INFO TaskSetManager: Starting task 34.0 in stage 12.0 (TID 267) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9691 bytes) +26/04/01 08:43:13 INFO TaskSetManager: Finished task 26.0 in stage 12.0 (TID 259) in 3339 ms on 10.0.0.133 (executor driver) (27/37) +26/04/01 08:43:13 INFO Executor: Running task 34.0 in stage 12.0 (TID 267) +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO Executor: Finished task 27.0 in stage 12.0 (TID 260). 6563 bytes result sent to driver +26/04/01 08:43:13 INFO TaskSetManager: Starting task 35.0 in stage 12.0 (TID 268) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9694 bytes) +26/04/01 08:43:13 INFO TaskSetManager: Finished task 27.0 in stage 12.0 (TID 260) in 3352 ms on 10.0.0.133 (executor driver) (28/37) +26/04/01 08:43:13 INFO Executor: Running task 35.0 in stage 12.0 (TID 268) +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO Executor: Finished task 28.0 in stage 12.0 (TID 261). 6563 bytes result sent to driver +26/04/01 08:43:13 INFO TaskSetManager: Starting task 36.0 in stage 12.0 (TID 269) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9482 bytes) +26/04/01 08:43:13 INFO TaskSetManager: Finished task 28.0 in stage 12.0 (TID 261) in 3357 ms on 10.0.0.133 (executor driver) (29/37) +26/04/01 08:43:13 INFO Executor: Running task 36.0 in stage 12.0 (TID 269) +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO Executor: Finished task 29.0 in stage 12.0 (TID 262). 6563 bytes result sent to driver +26/04/01 08:43:13 INFO TaskSetManager: Starting task 0.0 in stage 13.0 (TID 270) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:13 INFO TaskSetManager: Finished task 29.0 in stage 12.0 (TID 262) in 3351 ms on 10.0.0.133 (executor driver) (30/37) +26/04/01 08:43:13 INFO Executor: Running task 0.0 in stage 13.0 (TID 270) +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO BlockManagerInfo: Removed broadcast_15_piece0 on 10.0.0.133:59303 in memory (size: 7.9 KiB, free: 8.6 GiB) +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO Executor: Finished task 30.0 in stage 12.0 (TID 263). 6606 bytes result sent to driver +26/04/01 08:43:13 INFO TaskSetManager: Starting task 1.0 in stage 13.0 (TID 271) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:43:13 INFO Executor: Running task 1.0 in stage 13.0 (TID 271) +26/04/01 08:43:13 INFO TaskSetManager: Finished task 30.0 in stage 12.0 (TID 263) in 3328 ms on 10.0.0.133 (executor driver) (31/37) +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO Executor: Finished task 31.0 in stage 12.0 (TID 264). 6606 bytes result sent to driver +26/04/01 08:43:13 INFO TaskSetManager: Starting task 2.0 in stage 13.0 (TID 272) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:13 INFO Executor: Running task 2.0 in stage 13.0 (TID 272) +26/04/01 08:43:13 INFO TaskSetManager: Finished task 31.0 in stage 12.0 (TID 264) in 3321 ms on 10.0.0.133 (executor driver) (32/37) +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:14 INFO Executor: Finished task 36.0 in stage 12.0 (TID 269). 6606 bytes result sent to driver +26/04/01 08:43:14 INFO TaskSetManager: Starting task 3.0 in stage 13.0 (TID 273) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:14 INFO Executor: Running task 3.0 in stage 13.0 (TID 273) +26/04/01 08:43:14 INFO TaskSetManager: Finished task 36.0 in stage 12.0 (TID 269) in 1146 ms on 10.0.0.133 (executor driver) (33/37) +26/04/01 08:43:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:15 INFO Executor: Finished task 33.0 in stage 12.0 (TID 266). 6606 bytes result sent to driver +26/04/01 08:43:15 INFO TaskSetManager: Starting task 4.0 in stage 13.0 (TID 274) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:15 INFO TaskSetManager: Finished task 33.0 in stage 12.0 (TID 266) in 2037 ms on 10.0.0.133 (executor driver) (34/37) +26/04/01 08:43:15 INFO Executor: Running task 4.0 in stage 13.0 (TID 274) +26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:15 INFO Executor: Finished task 32.0 in stage 12.0 (TID 265). 6606 bytes result sent to driver +26/04/01 08:43:15 INFO TaskSetManager: Starting task 5.0 in stage 13.0 (TID 275) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:43:15 INFO TaskSetManager: Finished task 32.0 in stage 12.0 (TID 265) in 2050 ms on 10.0.0.133 (executor driver) (35/37) +26/04/01 08:43:15 INFO Executor: Running task 5.0 in stage 13.0 (TID 275) +26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:15 INFO Executor: Finished task 34.0 in stage 12.0 (TID 267). 6606 bytes result sent to driver +26/04/01 08:43:15 INFO TaskSetManager: Starting task 6.0 in stage 13.0 (TID 276) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:15 INFO TaskSetManager: Finished task 34.0 in stage 12.0 (TID 267) in 2038 ms on 10.0.0.133 (executor driver) (36/37) +26/04/01 08:43:15 INFO Executor: Running task 6.0 in stage 13.0 (TID 276) +26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:15 INFO Executor: Finished task 35.0 in stage 12.0 (TID 268). 6606 bytes result sent to driver +26/04/01 08:43:15 INFO TaskSetManager: Starting task 7.0 in stage 13.0 (TID 277) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:15 INFO TaskSetManager: Finished task 35.0 in stage 12.0 (TID 268) in 2043 ms on 10.0.0.133 (executor driver) (37/37) +26/04/01 08:43:15 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool +26/04/01 08:43:15 INFO Executor: Running task 7.0 in stage 13.0 (TID 277) +26/04/01 08:43:15 INFO DAGScheduler: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 219.261 s +26/04/01 08:43:15 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:43:15 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 13) +26/04/01 08:43:15 INFO DAGScheduler: waiting: Set() +26/04/01 08:43:15 INFO DAGScheduler: failed: Set() +26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:15 INFO Executor: Finished task 0.0 in stage 13.0 (TID 270). 6649 bytes result sent to driver +26/04/01 08:43:15 INFO TaskSetManager: Starting task 8.0 in stage 13.0 (TID 278) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:43:15 INFO TaskSetManager: Finished task 0.0 in stage 13.0 (TID 270) in 2270 ms on 10.0.0.133 (executor driver) (1/64) +26/04/01 08:43:15 INFO Executor: Running task 8.0 in stage 13.0 (TID 278) +26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:15 INFO Executor: Finished task 1.0 in stage 13.0 (TID 271). 6563 bytes result sent to driver +26/04/01 08:43:15 INFO TaskSetManager: Starting task 9.0 in stage 13.0 (TID 279) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:15 INFO Executor: Running task 9.0 in stage 13.0 (TID 279) +26/04/01 08:43:15 INFO TaskSetManager: Finished task 1.0 in stage 13.0 (TID 271) in 2277 ms on 10.0.0.133 (executor driver) (2/64) +26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:15 INFO Executor: Finished task 2.0 in stage 13.0 (TID 272). 6563 bytes result sent to driver +26/04/01 08:43:15 INFO TaskSetManager: Starting task 10.0 in stage 13.0 (TID 280) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:43:15 INFO TaskSetManager: Finished task 2.0 in stage 13.0 (TID 272) in 2263 ms on 10.0.0.133 (executor driver) (3/64) +26/04/01 08:43:15 INFO Executor: Running task 10.0 in stage 13.0 (TID 280) +26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:16 INFO Executor: Finished task 3.0 in stage 13.0 (TID 273). 6563 bytes result sent to driver +26/04/01 08:43:16 INFO TaskSetManager: Starting task 11.0 in stage 13.0 (TID 281) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:16 INFO TaskSetManager: Finished task 3.0 in stage 13.0 (TID 273) in 2248 ms on 10.0.0.133 (executor driver) (4/64) +26/04/01 08:43:16 INFO Executor: Running task 11.0 in stage 13.0 (TID 281) +26/04/01 08:43:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:17 INFO Executor: Finished task 4.0 in stage 13.0 (TID 274). 6563 bytes result sent to driver +26/04/01 08:43:17 INFO TaskSetManager: Starting task 12.0 in stage 13.0 (TID 282) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:17 INFO TaskSetManager: Finished task 4.0 in stage 13.0 (TID 274) in 2280 ms on 10.0.0.133 (executor driver) (5/64) +26/04/01 08:43:17 INFO Executor: Running task 12.0 in stage 13.0 (TID 282) +26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:17 INFO Executor: Finished task 5.0 in stage 13.0 (TID 275). 6563 bytes result sent to driver +26/04/01 08:43:17 INFO TaskSetManager: Starting task 13.0 in stage 13.0 (TID 283) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:17 INFO Executor: Running task 13.0 in stage 13.0 (TID 283) +26/04/01 08:43:17 INFO TaskSetManager: Finished task 5.0 in stage 13.0 (TID 275) in 2295 ms on 10.0.0.133 (executor driver) (6/64) +26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:17 INFO Executor: Finished task 6.0 in stage 13.0 (TID 276). 6563 bytes result sent to driver +26/04/01 08:43:17 INFO TaskSetManager: Starting task 14.0 in stage 13.0 (TID 284) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:17 INFO TaskSetManager: Finished task 6.0 in stage 13.0 (TID 276) in 2291 ms on 10.0.0.133 (executor driver) (7/64) +26/04/01 08:43:17 INFO Executor: Running task 14.0 in stage 13.0 (TID 284) +26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:17 INFO Executor: Finished task 7.0 in stage 13.0 (TID 277). 6563 bytes result sent to driver +26/04/01 08:43:17 INFO TaskSetManager: Starting task 15.0 in stage 13.0 (TID 285) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:17 INFO TaskSetManager: Finished task 7.0 in stage 13.0 (TID 277) in 2306 ms on 10.0.0.133 (executor driver) (8/64) +26/04/01 08:43:17 INFO Executor: Running task 15.0 in stage 13.0 (TID 285) +26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:18 INFO Executor: Finished task 8.0 in stage 13.0 (TID 278). 6563 bytes result sent to driver +26/04/01 08:43:18 INFO TaskSetManager: Starting task 16.0 in stage 13.0 (TID 286) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:43:18 INFO TaskSetManager: Finished task 8.0 in stage 13.0 (TID 278) in 2307 ms on 10.0.0.133 (executor driver) (9/64) +26/04/01 08:43:18 INFO Executor: Running task 16.0 in stage 13.0 (TID 286) +26/04/01 08:43:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:18 INFO Executor: Finished task 9.0 in stage 13.0 (TID 279). 6563 bytes result sent to driver +26/04/01 08:43:18 INFO TaskSetManager: Starting task 17.0 in stage 13.0 (TID 287) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:18 INFO TaskSetManager: Finished task 9.0 in stage 13.0 (TID 279) in 2314 ms on 10.0.0.133 (executor driver) (10/64) +26/04/01 08:43:18 INFO Executor: Running task 17.0 in stage 13.0 (TID 287) +26/04/01 08:43:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:18 INFO Executor: Finished task 10.0 in stage 13.0 (TID 280). 6563 bytes result sent to driver +26/04/01 08:43:18 INFO TaskSetManager: Starting task 18.0 in stage 13.0 (TID 288) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:43:18 INFO TaskSetManager: Finished task 10.0 in stage 13.0 (TID 280) in 2286 ms on 10.0.0.133 (executor driver) (11/64) +26/04/01 08:43:18 INFO Executor: Running task 18.0 in stage 13.0 (TID 288) +26/04/01 08:43:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:19 INFO Executor: Finished task 11.0 in stage 13.0 (TID 281). 6563 bytes result sent to driver +26/04/01 08:43:19 INFO TaskSetManager: Starting task 19.0 in stage 13.0 (TID 289) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:19 INFO TaskSetManager: Finished task 11.0 in stage 13.0 (TID 281) in 2280 ms on 10.0.0.133 (executor driver) (12/64) +26/04/01 08:43:19 INFO Executor: Running task 19.0 in stage 13.0 (TID 289) +26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:19 INFO Executor: Finished task 12.0 in stage 13.0 (TID 282). 6563 bytes result sent to driver +26/04/01 08:43:19 INFO TaskSetManager: Starting task 20.0 in stage 13.0 (TID 290) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:19 INFO Executor: Running task 20.0 in stage 13.0 (TID 290) +26/04/01 08:43:19 INFO TaskSetManager: Finished task 12.0 in stage 13.0 (TID 282) in 2301 ms on 10.0.0.133 (executor driver) (13/64) +26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:19 INFO Executor: Finished task 13.0 in stage 13.0 (TID 283). 6563 bytes result sent to driver +26/04/01 08:43:19 INFO TaskSetManager: Starting task 21.0 in stage 13.0 (TID 291) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:43:19 INFO Executor: Running task 21.0 in stage 13.0 (TID 291) +26/04/01 08:43:19 INFO TaskSetManager: Finished task 13.0 in stage 13.0 (TID 283) in 2295 ms on 10.0.0.133 (executor driver) (14/64) +26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:19 INFO Executor: Finished task 14.0 in stage 13.0 (TID 284). 6563 bytes result sent to driver +26/04/01 08:43:19 INFO TaskSetManager: Starting task 22.0 in stage 13.0 (TID 292) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:19 INFO TaskSetManager: Finished task 14.0 in stage 13.0 (TID 284) in 2283 ms on 10.0.0.133 (executor driver) (15/64) +26/04/01 08:43:19 INFO Executor: Running task 22.0 in stage 13.0 (TID 292) +26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:19 INFO Executor: Finished task 15.0 in stage 13.0 (TID 285). 6563 bytes result sent to driver +26/04/01 08:43:19 INFO TaskSetManager: Starting task 23.0 in stage 13.0 (TID 293) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:19 INFO TaskSetManager: Finished task 15.0 in stage 13.0 (TID 285) in 2288 ms on 10.0.0.133 (executor driver) (16/64) +26/04/01 08:43:19 INFO Executor: Running task 23.0 in stage 13.0 (TID 293) +26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:20 INFO Executor: Finished task 16.0 in stage 13.0 (TID 286). 6563 bytes result sent to driver +26/04/01 08:43:20 INFO TaskSetManager: Starting task 24.0 in stage 13.0 (TID 294) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:43:20 INFO Executor: Running task 24.0 in stage 13.0 (TID 294) +26/04/01 08:43:20 INFO TaskSetManager: Finished task 16.0 in stage 13.0 (TID 286) in 2261 ms on 10.0.0.133 (executor driver) (17/64) +26/04/01 08:43:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:20 INFO Executor: Finished task 17.0 in stage 13.0 (TID 287). 6563 bytes result sent to driver +26/04/01 08:43:20 INFO TaskSetManager: Starting task 25.0 in stage 13.0 (TID 295) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:20 INFO TaskSetManager: Finished task 17.0 in stage 13.0 (TID 287) in 2283 ms on 10.0.0.133 (executor driver) (18/64) +26/04/01 08:43:20 INFO Executor: Running task 25.0 in stage 13.0 (TID 295) +26/04/01 08:43:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:20 INFO Executor: Finished task 18.0 in stage 13.0 (TID 288). 6563 bytes result sent to driver +26/04/01 08:43:20 INFO TaskSetManager: Starting task 26.0 in stage 13.0 (TID 296) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:20 INFO TaskSetManager: Finished task 18.0 in stage 13.0 (TID 288) in 2265 ms on 10.0.0.133 (executor driver) (19/64) +26/04/01 08:43:20 INFO Executor: Running task 26.0 in stage 13.0 (TID 296) +26/04/01 08:43:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:21 INFO Executor: Finished task 19.0 in stage 13.0 (TID 289). 6563 bytes result sent to driver +26/04/01 08:43:21 INFO TaskSetManager: Starting task 27.0 in stage 13.0 (TID 297) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:21 INFO TaskSetManager: Finished task 19.0 in stage 13.0 (TID 289) in 2253 ms on 10.0.0.133 (executor driver) (20/64) +26/04/01 08:43:21 INFO Executor: Running task 27.0 in stage 13.0 (TID 297) +26/04/01 08:43:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:22 INFO Executor: Finished task 21.0 in stage 13.0 (TID 291). 6563 bytes result sent to driver +26/04/01 08:43:22 INFO TaskSetManager: Starting task 28.0 in stage 13.0 (TID 298) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:22 INFO Executor: Running task 28.0 in stage 13.0 (TID 298) +26/04/01 08:43:22 INFO TaskSetManager: Finished task 21.0 in stage 13.0 (TID 291) in 2272 ms on 10.0.0.133 (executor driver) (21/64) +26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:22 INFO Executor: Finished task 20.0 in stage 13.0 (TID 290). 6563 bytes result sent to driver +26/04/01 08:43:22 INFO TaskSetManager: Starting task 29.0 in stage 13.0 (TID 299) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:22 INFO Executor: Running task 29.0 in stage 13.0 (TID 299) +26/04/01 08:43:22 INFO TaskSetManager: Finished task 20.0 in stage 13.0 (TID 290) in 2296 ms on 10.0.0.133 (executor driver) (22/64) +26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:22 INFO Executor: Finished task 22.0 in stage 13.0 (TID 292). 6563 bytes result sent to driver +26/04/01 08:43:22 INFO TaskSetManager: Starting task 30.0 in stage 13.0 (TID 300) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:43:22 INFO TaskSetManager: Finished task 22.0 in stage 13.0 (TID 292) in 2279 ms on 10.0.0.133 (executor driver) (23/64) +26/04/01 08:43:22 INFO Executor: Running task 30.0 in stage 13.0 (TID 300) +26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:22 INFO Executor: Finished task 23.0 in stage 13.0 (TID 293). 6563 bytes result sent to driver +26/04/01 08:43:22 INFO TaskSetManager: Starting task 31.0 in stage 13.0 (TID 301) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:43:22 INFO TaskSetManager: Finished task 23.0 in stage 13.0 (TID 293) in 2289 ms on 10.0.0.133 (executor driver) (24/64) +26/04/01 08:43:22 INFO Executor: Running task 31.0 in stage 13.0 (TID 301) +26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:22 INFO Executor: Finished task 24.0 in stage 13.0 (TID 294). 6563 bytes result sent to driver +26/04/01 08:43:22 INFO TaskSetManager: Starting task 32.0 in stage 13.0 (TID 302) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:22 INFO TaskSetManager: Finished task 24.0 in stage 13.0 (TID 294) in 2267 ms on 10.0.0.133 (executor driver) (25/64) +26/04/01 08:43:22 INFO Executor: Running task 32.0 in stage 13.0 (TID 302) +26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:22 INFO Executor: Finished task 25.0 in stage 13.0 (TID 295). 6563 bytes result sent to driver +26/04/01 08:43:22 INFO TaskSetManager: Starting task 33.0 in stage 13.0 (TID 303) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:22 INFO TaskSetManager: Finished task 25.0 in stage 13.0 (TID 295) in 2261 ms on 10.0.0.133 (executor driver) (26/64) +26/04/01 08:43:22 INFO Executor: Running task 33.0 in stage 13.0 (TID 303) +26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:22 INFO Executor: Finished task 26.0 in stage 13.0 (TID 296). 6563 bytes result sent to driver +26/04/01 08:43:22 INFO TaskSetManager: Starting task 34.0 in stage 13.0 (TID 304) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:22 INFO TaskSetManager: Finished task 26.0 in stage 13.0 (TID 296) in 2269 ms on 10.0.0.133 (executor driver) (27/64) +26/04/01 08:43:22 INFO Executor: Running task 34.0 in stage 13.0 (TID 304) +26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:23 INFO Executor: Finished task 27.0 in stage 13.0 (TID 297). 6563 bytes result sent to driver +26/04/01 08:43:23 INFO TaskSetManager: Starting task 35.0 in stage 13.0 (TID 305) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:23 INFO TaskSetManager: Finished task 27.0 in stage 13.0 (TID 297) in 2245 ms on 10.0.0.133 (executor driver) (28/64) +26/04/01 08:43:23 INFO Executor: Running task 35.0 in stage 13.0 (TID 305) +26/04/01 08:43:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:24 INFO Executor: Finished task 32.0 in stage 13.0 (TID 302). 6563 bytes result sent to driver +26/04/01 08:43:24 INFO TaskSetManager: Starting task 36.0 in stage 13.0 (TID 306) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:24 INFO Executor: Running task 36.0 in stage 13.0 (TID 306) +26/04/01 08:43:24 INFO TaskSetManager: Finished task 32.0 in stage 13.0 (TID 302) in 1596 ms on 10.0.0.133 (executor driver) (29/64) +26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:24 INFO Executor: Finished task 33.0 in stage 13.0 (TID 303). 6563 bytes result sent to driver +26/04/01 08:43:24 INFO TaskSetManager: Starting task 37.0 in stage 13.0 (TID 307) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:24 INFO TaskSetManager: Finished task 33.0 in stage 13.0 (TID 303) in 1595 ms on 10.0.0.133 (executor driver) (30/64) +26/04/01 08:43:24 INFO Executor: Running task 37.0 in stage 13.0 (TID 307) +26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:24 INFO Executor: Finished task 34.0 in stage 13.0 (TID 304). 6563 bytes result sent to driver +26/04/01 08:43:24 INFO TaskSetManager: Starting task 38.0 in stage 13.0 (TID 308) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:24 INFO TaskSetManager: Finished task 34.0 in stage 13.0 (TID 304) in 1585 ms on 10.0.0.133 (executor driver) (31/64) +26/04/01 08:43:24 INFO Executor: Running task 38.0 in stage 13.0 (TID 308) +26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:24 INFO Executor: Finished task 29.0 in stage 13.0 (TID 299). 6563 bytes result sent to driver +26/04/01 08:43:24 INFO TaskSetManager: Starting task 39.0 in stage 13.0 (TID 309) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:24 INFO TaskSetManager: Finished task 29.0 in stage 13.0 (TID 299) in 2290 ms on 10.0.0.133 (executor driver) (32/64) +26/04/01 08:43:24 INFO Executor: Running task 39.0 in stage 13.0 (TID 309) +26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:24 INFO Executor: Finished task 28.0 in stage 13.0 (TID 298). 6563 bytes result sent to driver +26/04/01 08:43:24 INFO TaskSetManager: Starting task 40.0 in stage 13.0 (TID 310) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:24 INFO TaskSetManager: Finished task 28.0 in stage 13.0 (TID 298) in 2301 ms on 10.0.0.133 (executor driver) (33/64) +26/04/01 08:43:24 INFO Executor: Running task 40.0 in stage 13.0 (TID 310) +26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:24 INFO Executor: Finished task 30.0 in stage 13.0 (TID 300). 6563 bytes result sent to driver +26/04/01 08:43:24 INFO TaskSetManager: Starting task 41.0 in stage 13.0 (TID 311) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:24 INFO TaskSetManager: Finished task 30.0 in stage 13.0 (TID 300) in 2294 ms on 10.0.0.133 (executor driver) (34/64) +26/04/01 08:43:24 INFO Executor: Running task 41.0 in stage 13.0 (TID 311) +26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:24 INFO Executor: Finished task 31.0 in stage 13.0 (TID 301). 6563 bytes result sent to driver +26/04/01 08:43:24 INFO TaskSetManager: Starting task 42.0 in stage 13.0 (TID 312) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:24 INFO TaskSetManager: Finished task 31.0 in stage 13.0 (TID 301) in 2285 ms on 10.0.0.133 (executor driver) (35/64) +26/04/01 08:43:24 INFO Executor: Running task 42.0 in stage 13.0 (TID 312) +26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:25 INFO Executor: Finished task 35.0 in stage 13.0 (TID 305). 6563 bytes result sent to driver +26/04/01 08:43:25 INFO TaskSetManager: Starting task 43.0 in stage 13.0 (TID 313) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:25 INFO TaskSetManager: Finished task 35.0 in stage 13.0 (TID 305) in 1570 ms on 10.0.0.133 (executor driver) (36/64) +26/04/01 08:43:25 INFO Executor: Running task 43.0 in stage 13.0 (TID 313) +26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:25 INFO Executor: Finished task 36.0 in stage 13.0 (TID 306). 6563 bytes result sent to driver +26/04/01 08:43:25 INFO TaskSetManager: Starting task 44.0 in stage 13.0 (TID 314) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:25 INFO Executor: Running task 44.0 in stage 13.0 (TID 314) +26/04/01 08:43:25 INFO TaskSetManager: Finished task 36.0 in stage 13.0 (TID 306) in 1597 ms on 10.0.0.133 (executor driver) (37/64) +26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:25 INFO Executor: Finished task 37.0 in stage 13.0 (TID 307). 6563 bytes result sent to driver +26/04/01 08:43:25 INFO TaskSetManager: Starting task 45.0 in stage 13.0 (TID 315) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:25 INFO Executor: Running task 45.0 in stage 13.0 (TID 315) +26/04/01 08:43:25 INFO TaskSetManager: Finished task 37.0 in stage 13.0 (TID 307) in 1612 ms on 10.0.0.133 (executor driver) (38/64) +26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:25 INFO Executor: Finished task 38.0 in stage 13.0 (TID 308). 6563 bytes result sent to driver +26/04/01 08:43:25 INFO TaskSetManager: Starting task 46.0 in stage 13.0 (TID 316) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:25 INFO TaskSetManager: Finished task 38.0 in stage 13.0 (TID 308) in 1605 ms on 10.0.0.133 (executor driver) (39/64) +26/04/01 08:43:25 INFO Executor: Running task 46.0 in stage 13.0 (TID 316) +26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:26 INFO Executor: Finished task 40.0 in stage 13.0 (TID 310). 6563 bytes result sent to driver +26/04/01 08:43:26 INFO TaskSetManager: Starting task 47.0 in stage 13.0 (TID 317) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:26 INFO Executor: Running task 47.0 in stage 13.0 (TID 317) +26/04/01 08:43:26 INFO TaskSetManager: Finished task 40.0 in stage 13.0 (TID 310) in 1595 ms on 10.0.0.133 (executor driver) (40/64) +26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:26 INFO Executor: Finished task 39.0 in stage 13.0 (TID 309). 6563 bytes result sent to driver +26/04/01 08:43:26 INFO TaskSetManager: Starting task 48.0 in stage 13.0 (TID 318) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:26 INFO TaskSetManager: Finished task 39.0 in stage 13.0 (TID 309) in 1599 ms on 10.0.0.133 (executor driver) (41/64) +26/04/01 08:43:26 INFO Executor: Running task 48.0 in stage 13.0 (TID 318) +26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:26 INFO Executor: Finished task 41.0 in stage 13.0 (TID 311). 6563 bytes result sent to driver +26/04/01 08:43:26 INFO TaskSetManager: Starting task 49.0 in stage 13.0 (TID 319) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:26 INFO TaskSetManager: Finished task 41.0 in stage 13.0 (TID 311) in 1596 ms on 10.0.0.133 (executor driver) (42/64) +26/04/01 08:43:26 INFO Executor: Running task 49.0 in stage 13.0 (TID 319) +26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:26 INFO Executor: Finished task 42.0 in stage 13.0 (TID 312). 6563 bytes result sent to driver +26/04/01 08:43:26 INFO TaskSetManager: Starting task 50.0 in stage 13.0 (TID 320) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:26 INFO TaskSetManager: Finished task 42.0 in stage 13.0 (TID 312) in 1592 ms on 10.0.0.133 (executor driver) (43/64) +26/04/01 08:43:26 INFO Executor: Running task 50.0 in stage 13.0 (TID 320) +26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:26 INFO Executor: Finished task 43.0 in stage 13.0 (TID 313). 6563 bytes result sent to driver +26/04/01 08:43:26 INFO TaskSetManager: Starting task 51.0 in stage 13.0 (TID 321) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:26 INFO TaskSetManager: Finished task 43.0 in stage 13.0 (TID 313) in 1579 ms on 10.0.0.133 (executor driver) (44/64) +26/04/01 08:43:26 INFO Executor: Running task 51.0 in stage 13.0 (TID 321) +26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:27 INFO Executor: Finished task 44.0 in stage 13.0 (TID 314). 6563 bytes result sent to driver +26/04/01 08:43:27 INFO TaskSetManager: Starting task 52.0 in stage 13.0 (TID 322) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:27 INFO TaskSetManager: Finished task 44.0 in stage 13.0 (TID 314) in 1611 ms on 10.0.0.133 (executor driver) (45/64) +26/04/01 08:43:27 INFO Executor: Running task 52.0 in stage 13.0 (TID 322) +26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:27 INFO Executor: Finished task 45.0 in stage 13.0 (TID 315). 6563 bytes result sent to driver +26/04/01 08:43:27 INFO TaskSetManager: Starting task 53.0 in stage 13.0 (TID 323) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:27 INFO TaskSetManager: Finished task 45.0 in stage 13.0 (TID 315) in 1602 ms on 10.0.0.133 (executor driver) (46/64) +26/04/01 08:43:27 INFO Executor: Running task 53.0 in stage 13.0 (TID 323) +26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:27 INFO Executor: Finished task 46.0 in stage 13.0 (TID 316). 6563 bytes result sent to driver +26/04/01 08:43:27 INFO TaskSetManager: Starting task 54.0 in stage 13.0 (TID 324) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:27 INFO Executor: Running task 54.0 in stage 13.0 (TID 324) +26/04/01 08:43:27 INFO TaskSetManager: Finished task 46.0 in stage 13.0 (TID 316) in 1615 ms on 10.0.0.133 (executor driver) (47/64) +26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:27 INFO Executor: Finished task 48.0 in stage 13.0 (TID 318). 6563 bytes result sent to driver +26/04/01 08:43:27 INFO TaskSetManager: Starting task 55.0 in stage 13.0 (TID 325) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:27 INFO TaskSetManager: Finished task 48.0 in stage 13.0 (TID 318) in 1615 ms on 10.0.0.133 (executor driver) (48/64) +26/04/01 08:43:27 INFO Executor: Running task 55.0 in stage 13.0 (TID 325) +26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:27 INFO Executor: Finished task 47.0 in stage 13.0 (TID 317). 6563 bytes result sent to driver +26/04/01 08:43:27 INFO TaskSetManager: Starting task 56.0 in stage 13.0 (TID 326) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:27 INFO TaskSetManager: Finished task 47.0 in stage 13.0 (TID 317) in 1626 ms on 10.0.0.133 (executor driver) (49/64) +26/04/01 08:43:27 INFO Executor: Running task 56.0 in stage 13.0 (TID 326) +26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:27 INFO Executor: Finished task 49.0 in stage 13.0 (TID 319). 6563 bytes result sent to driver +26/04/01 08:43:27 INFO TaskSetManager: Starting task 57.0 in stage 13.0 (TID 327) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:27 INFO TaskSetManager: Finished task 49.0 in stage 13.0 (TID 319) in 1616 ms on 10.0.0.133 (executor driver) (50/64) +26/04/01 08:43:27 INFO Executor: Running task 57.0 in stage 13.0 (TID 327) +26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:27 INFO Executor: Finished task 50.0 in stage 13.0 (TID 320). 6563 bytes result sent to driver +26/04/01 08:43:27 INFO TaskSetManager: Starting task 58.0 in stage 13.0 (TID 328) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:27 INFO TaskSetManager: Finished task 50.0 in stage 13.0 (TID 320) in 1607 ms on 10.0.0.133 (executor driver) (51/64) +26/04/01 08:43:27 INFO Executor: Running task 58.0 in stage 13.0 (TID 328) +26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:28 INFO Executor: Finished task 51.0 in stage 13.0 (TID 321). 6563 bytes result sent to driver +26/04/01 08:43:28 INFO TaskSetManager: Starting task 59.0 in stage 13.0 (TID 329) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:28 INFO TaskSetManager: Finished task 51.0 in stage 13.0 (TID 321) in 1570 ms on 10.0.0.133 (executor driver) (52/64) +26/04/01 08:43:28 INFO Executor: Running task 59.0 in stage 13.0 (TID 329) +26/04/01 08:43:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO Executor: Finished task 52.0 in stage 13.0 (TID 322). 6563 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 60.0 in stage 13.0 (TID 330) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 52.0 in stage 13.0 (TID 322) in 1597 ms on 10.0.0.133 (executor driver) (53/64) +26/04/01 08:43:29 INFO Executor: Running task 60.0 in stage 13.0 (TID 330) +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO Executor: Finished task 53.0 in stage 13.0 (TID 323). 6563 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 61.0 in stage 13.0 (TID 331) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 53.0 in stage 13.0 (TID 323) in 1608 ms on 10.0.0.133 (executor driver) (54/64) +26/04/01 08:43:29 INFO Executor: Running task 61.0 in stage 13.0 (TID 331) +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO Executor: Finished task 54.0 in stage 13.0 (TID 324). 6563 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 62.0 in stage 13.0 (TID 332) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 54.0 in stage 13.0 (TID 324) in 1606 ms on 10.0.0.133 (executor driver) (55/64) +26/04/01 08:43:29 INFO Executor: Running task 62.0 in stage 13.0 (TID 332) +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO Executor: Finished task 56.0 in stage 13.0 (TID 326). 6563 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 55.0 in stage 13.0 (TID 325). 6563 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 63.0 in stage 13.0 (TID 333) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:43:29 INFO Executor: Running task 63.0 in stage 13.0 (TID 333) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 334) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO TaskSetManager: Finished task 56.0 in stage 13.0 (TID 326) in 1602 ms on 10.0.0.133 (executor driver) (56/64) +26/04/01 08:43:29 INFO Executor: Running task 0.0 in stage 15.0 (TID 334) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 55.0 in stage 13.0 (TID 325) in 1610 ms on 10.0.0.133 (executor driver) (57/64) +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:43:29 INFO Executor: Finished task 0.0 in stage 15.0 (TID 334). 33895 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 1.0 in stage 15.0 (TID 335) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 1.0 in stage 15.0 (TID 335) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 334) in 26 ms on 10.0.0.133 (executor driver) (1/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 1.0 in stage 15.0 (TID 335). 34027 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 2.0 in stage 15.0 (TID 336) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 2.0 in stage 15.0 (TID 336) +26/04/01 08:43:29 INFO Executor: Finished task 57.0 in stage 13.0 (TID 327). 6563 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 3.0 in stage 15.0 (TID 337) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 1.0 in stage 15.0 (TID 335) in 6 ms on 10.0.0.133 (executor driver) (2/200) +26/04/01 08:43:29 INFO Executor: Running task 3.0 in stage 15.0 (TID 337) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 57.0 in stage 13.0 (TID 327) in 1594 ms on 10.0.0.133 (executor driver) (58/64) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO BlockManagerInfo: Removed broadcast_17_piece0 on 10.0.0.133:59303 in memory (size: 8.0 KiB, free: 8.6 GiB) +26/04/01 08:43:29 INFO Executor: Finished task 2.0 in stage 15.0 (TID 336). 33844 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 3.0 in stage 15.0 (TID 337). 34371 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 4.0 in stage 15.0 (TID 338) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 4.0 in stage 15.0 (TID 338) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 5.0 in stage 15.0 (TID 339) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 2.0 in stage 15.0 (TID 336) in 7 ms on 10.0.0.133 (executor driver) (3/200) +26/04/01 08:43:29 INFO Executor: Running task 5.0 in stage 15.0 (TID 339) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 3.0 in stage 15.0 (TID 337) in 7 ms on 10.0.0.133 (executor driver) (4/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 4.0 in stage 15.0 (TID 338). 34473 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 5.0 in stage 15.0 (TID 339). 34156 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 6.0 in stage 15.0 (TID 340) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 7.0 in stage 15.0 (TID 341) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 6.0 in stage 15.0 (TID 340) +26/04/01 08:43:29 INFO Executor: Running task 7.0 in stage 15.0 (TID 341) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 4.0 in stage 15.0 (TID 338) in 6 ms on 10.0.0.133 (executor driver) (5/200) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 5.0 in stage 15.0 (TID 339) in 6 ms on 10.0.0.133 (executor driver) (6/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 6.0 in stage 15.0 (TID 340). 33938 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 8.0 in stage 15.0 (TID 342) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Finished task 7.0 in stage 15.0 (TID 341). 34062 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Finished task 6.0 in stage 15.0 (TID 340) in 5 ms on 10.0.0.133 (executor driver) (7/200) +26/04/01 08:43:29 INFO Executor: Running task 8.0 in stage 15.0 (TID 342) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 9.0 in stage 15.0 (TID 343) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 9.0 in stage 15.0 (TID 343) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 7.0 in stage 15.0 (TID 341) in 5 ms on 10.0.0.133 (executor driver) (8/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 9.0 in stage 15.0 (TID 343). 34979 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 8.0 in stage 15.0 (TID 342). 33972 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 10.0 in stage 15.0 (TID 344) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 10.0 in stage 15.0 (TID 344) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 11.0 in stage 15.0 (TID 345) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 9.0 in stage 15.0 (TID 343) in 6 ms on 10.0.0.133 (executor driver) (9/200) +26/04/01 08:43:29 INFO Executor: Running task 11.0 in stage 15.0 (TID 345) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 8.0 in stage 15.0 (TID 342) in 6 ms on 10.0.0.133 (executor driver) (10/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 10.0 in stage 15.0 (TID 344). 34352 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 11.0 in stage 15.0 (TID 345). 33620 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 12.0 in stage 15.0 (TID 346) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 12.0 in stage 15.0 (TID 346) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 13.0 in stage 15.0 (TID 347) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 10.0 in stage 15.0 (TID 344) in 5 ms on 10.0.0.133 (executor driver) (11/200) +26/04/01 08:43:29 INFO Executor: Running task 13.0 in stage 15.0 (TID 347) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 11.0 in stage 15.0 (TID 345) in 5 ms on 10.0.0.133 (executor driver) (12/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 12.0 in stage 15.0 (TID 346). 34084 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 14.0 in stage 15.0 (TID 348) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 14.0 in stage 15.0 (TID 348) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 12.0 in stage 15.0 (TID 346) in 5 ms on 10.0.0.133 (executor driver) (13/200) +26/04/01 08:43:29 INFO Executor: Finished task 13.0 in stage 15.0 (TID 347). 34520 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 15.0 in stage 15.0 (TID 349) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 15.0 in stage 15.0 (TID 349) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 13.0 in stage 15.0 (TID 347) in 5 ms on 10.0.0.133 (executor driver) (14/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 14.0 in stage 15.0 (TID 348). 34658 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 16.0 in stage 15.0 (TID 350) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 16.0 in stage 15.0 (TID 350) +26/04/01 08:43:29 INFO Executor: Finished task 15.0 in stage 15.0 (TID 349). 34244 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Finished task 14.0 in stage 15.0 (TID 348) in 5 ms on 10.0.0.133 (executor driver) (15/200) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 17.0 in stage 15.0 (TID 351) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 17.0 in stage 15.0 (TID 351) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 15.0 in stage 15.0 (TID 349) in 4 ms on 10.0.0.133 (executor driver) (16/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 16.0 in stage 15.0 (TID 350). 33628 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 18.0 in stage 15.0 (TID 352) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 18.0 in stage 15.0 (TID 352) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 16.0 in stage 15.0 (TID 350) in 5 ms on 10.0.0.133 (executor driver) (17/200) +26/04/01 08:43:29 INFO Executor: Finished task 17.0 in stage 15.0 (TID 351). 34240 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 19.0 in stage 15.0 (TID 353) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO Executor: Running task 19.0 in stage 15.0 (TID 353) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Finished task 17.0 in stage 15.0 (TID 351) in 6 ms on 10.0.0.133 (executor driver) (18/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 18.0 in stage 15.0 (TID 352). 34217 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 20.0 in stage 15.0 (TID 354) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 20.0 in stage 15.0 (TID 354) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 18.0 in stage 15.0 (TID 352) in 4 ms on 10.0.0.133 (executor driver) (19/200) +26/04/01 08:43:29 INFO Executor: Finished task 19.0 in stage 15.0 (TID 353). 35085 bytes result sent to driver +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Starting task 21.0 in stage 15.0 (TID 355) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 21.0 in stage 15.0 (TID 355) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 19.0 in stage 15.0 (TID 353) in 5 ms on 10.0.0.133 (executor driver) (20/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 20.0 in stage 15.0 (TID 354). 34559 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 22.0 in stage 15.0 (TID 356) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 22.0 in stage 15.0 (TID 356) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 20.0 in stage 15.0 (TID 354) in 5 ms on 10.0.0.133 (executor driver) (21/200) +26/04/01 08:43:29 INFO Executor: Finished task 58.0 in stage 13.0 (TID 328). 6606 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 23.0 in stage 15.0 (TID 357) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 23.0 in stage 15.0 (TID 357) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 58.0 in stage 13.0 (TID 328) in 1597 ms on 10.0.0.133 (executor driver) (59/64) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 21.0 in stage 15.0 (TID 355). 34824 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 24.0 in stage 15.0 (TID 358) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Running task 24.0 in stage 15.0 (TID 358) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 21.0 in stage 15.0 (TID 355) in 5 ms on 10.0.0.133 (executor driver) (22/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 22.0 in stage 15.0 (TID 356). 34572 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 23.0 in stage 15.0 (TID 357). 34082 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 25.0 in stage 15.0 (TID 359) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 25.0 in stage 15.0 (TID 359) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 26.0 in stage 15.0 (TID 360) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 22.0 in stage 15.0 (TID 356) in 5 ms on 10.0.0.133 (executor driver) (23/200) +26/04/01 08:43:29 INFO Executor: Running task 26.0 in stage 15.0 (TID 360) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 23.0 in stage 15.0 (TID 357) in 5 ms on 10.0.0.133 (executor driver) (24/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO Executor: Finished task 24.0 in stage 15.0 (TID 358). 34683 bytes result sent to driver +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Starting task 27.0 in stage 15.0 (TID 361) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 27.0 in stage 15.0 (TID 361) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 24.0 in stage 15.0 (TID 358) in 5 ms on 10.0.0.133 (executor driver) (25/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 25.0 in stage 15.0 (TID 359). 34085 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 28.0 in stage 15.0 (TID 362) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 28.0 in stage 15.0 (TID 362) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 25.0 in stage 15.0 (TID 359) in 5 ms on 10.0.0.133 (executor driver) (26/200) +26/04/01 08:43:29 INFO Executor: Finished task 26.0 in stage 15.0 (TID 360). 33825 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 29.0 in stage 15.0 (TID 363) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 29.0 in stage 15.0 (TID 363) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO TaskSetManager: Finished task 26.0 in stage 15.0 (TID 360) in 4 ms on 10.0.0.133 (executor driver) (27/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 27.0 in stage 15.0 (TID 361). 34141 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 30.0 in stage 15.0 (TID 364) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 30.0 in stage 15.0 (TID 364) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 27.0 in stage 15.0 (TID 361) in 4 ms on 10.0.0.133 (executor driver) (28/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 28.0 in stage 15.0 (TID 362). 34850 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 31.0 in stage 15.0 (TID 365) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 31.0 in stage 15.0 (TID 365) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 28.0 in stage 15.0 (TID 362) in 4 ms on 10.0.0.133 (executor driver) (29/200) +26/04/01 08:43:29 INFO Executor: Finished task 29.0 in stage 15.0 (TID 363). 33924 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 32.0 in stage 15.0 (TID 366) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 32.0 in stage 15.0 (TID 366) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 29.0 in stage 15.0 (TID 363) in 5 ms on 10.0.0.133 (executor driver) (30/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 30.0 in stage 15.0 (TID 364). 33936 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 33.0 in stage 15.0 (TID 367) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 30.0 in stage 15.0 (TID 364) in 5 ms on 10.0.0.133 (executor driver) (31/200) +26/04/01 08:43:29 INFO Executor: Running task 33.0 in stage 15.0 (TID 367) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 31.0 in stage 15.0 (TID 365). 34338 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 34.0 in stage 15.0 (TID 368) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 34.0 in stage 15.0 (TID 368) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 31.0 in stage 15.0 (TID 365) in 5 ms on 10.0.0.133 (executor driver) (32/200) +26/04/01 08:43:29 INFO Executor: Finished task 32.0 in stage 15.0 (TID 366). 34649 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 35.0 in stage 15.0 (TID 369) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 35.0 in stage 15.0 (TID 369) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 32.0 in stage 15.0 (TID 366) in 4 ms on 10.0.0.133 (executor driver) (33/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 33.0 in stage 15.0 (TID 367). 34079 bytes result sent to driver +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Starting task 36.0 in stage 15.0 (TID 370) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 33.0 in stage 15.0 (TID 367) in 4 ms on 10.0.0.133 (executor driver) (34/200) +26/04/01 08:43:29 INFO Executor: Running task 36.0 in stage 15.0 (TID 370) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 34.0 in stage 15.0 (TID 368). 34290 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 37.0 in stage 15.0 (TID 371) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 37.0 in stage 15.0 (TID 371) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 34.0 in stage 15.0 (TID 368) in 5 ms on 10.0.0.133 (executor driver) (35/200) +26/04/01 08:43:29 INFO Executor: Finished task 35.0 in stage 15.0 (TID 369). 34960 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 38.0 in stage 15.0 (TID 372) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 38.0 in stage 15.0 (TID 372) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 35.0 in stage 15.0 (TID 369) in 5 ms on 10.0.0.133 (executor driver) (36/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 36.0 in stage 15.0 (TID 370). 34927 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 39.0 in stage 15.0 (TID 373) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 39.0 in stage 15.0 (TID 373) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 36.0 in stage 15.0 (TID 370) in 5 ms on 10.0.0.133 (executor driver) (37/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 38.0 in stage 15.0 (TID 372). 33680 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 37.0 in stage 15.0 (TID 371). 34242 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 40.0 in stage 15.0 (TID 374) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 40.0 in stage 15.0 (TID 374) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 41.0 in stage 15.0 (TID 375) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 38.0 in stage 15.0 (TID 372) in 5 ms on 10.0.0.133 (executor driver) (38/200) +26/04/01 08:43:29 INFO Executor: Running task 41.0 in stage 15.0 (TID 375) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 37.0 in stage 15.0 (TID 371) in 5 ms on 10.0.0.133 (executor driver) (39/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 39.0 in stage 15.0 (TID 373). 33691 bytes result sent to driver +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Starting task 42.0 in stage 15.0 (TID 376) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 42.0 in stage 15.0 (TID 376) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 39.0 in stage 15.0 (TID 373) in 5 ms on 10.0.0.133 (executor driver) (40/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 40.0 in stage 15.0 (TID 374). 34584 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 41.0 in stage 15.0 (TID 375). 34443 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 43.0 in stage 15.0 (TID 377) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 43.0 in stage 15.0 (TID 377) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 44.0 in stage 15.0 (TID 378) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Finished task 42.0 in stage 15.0 (TID 376). 34213 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Finished task 40.0 in stage 15.0 (TID 374) in 5 ms on 10.0.0.133 (executor driver) (41/200) +26/04/01 08:43:29 INFO Executor: Running task 44.0 in stage 15.0 (TID 378) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 45.0 in stage 15.0 (TID 379) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 41.0 in stage 15.0 (TID 375) in 6 ms on 10.0.0.133 (executor driver) (42/200) +26/04/01 08:43:29 INFO Executor: Running task 45.0 in stage 15.0 (TID 379) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 42.0 in stage 15.0 (TID 376) in 5 ms on 10.0.0.133 (executor driver) (43/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 43.0 in stage 15.0 (TID 377). 33552 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 46.0 in stage 15.0 (TID 380) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 46.0 in stage 15.0 (TID 380) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 43.0 in stage 15.0 (TID 377) in 5 ms on 10.0.0.133 (executor driver) (44/200) +26/04/01 08:43:29 INFO Executor: Finished task 44.0 in stage 15.0 (TID 378). 33901 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 45.0 in stage 15.0 (TID 379). 34122 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 47.0 in stage 15.0 (TID 381) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 47.0 in stage 15.0 (TID 381) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 48.0 in stage 15.0 (TID 382) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO TaskSetManager: Finished task 44.0 in stage 15.0 (TID 378) in 7 ms on 10.0.0.133 (executor driver) (45/200) +26/04/01 08:43:29 INFO Executor: Running task 48.0 in stage 15.0 (TID 382) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 45.0 in stage 15.0 (TID 379) in 7 ms on 10.0.0.133 (executor driver) (46/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 46.0 in stage 15.0 (TID 380). 34158 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 49.0 in stage 15.0 (TID 383) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 49.0 in stage 15.0 (TID 383) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 46.0 in stage 15.0 (TID 380) in 7 ms on 10.0.0.133 (executor driver) (47/200) +26/04/01 08:43:29 INFO Executor: Finished task 47.0 in stage 15.0 (TID 381). 34470 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 50.0 in stage 15.0 (TID 384) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 50.0 in stage 15.0 (TID 384) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 47.0 in stage 15.0 (TID 381) in 5 ms on 10.0.0.133 (executor driver) (48/200) +26/04/01 08:43:29 INFO Executor: Finished task 48.0 in stage 15.0 (TID 382). 33889 bytes result sent to driver +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Starting task 51.0 in stage 15.0 (TID 385) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 51.0 in stage 15.0 (TID 385) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 48.0 in stage 15.0 (TID 382) in 5 ms on 10.0.0.133 (executor driver) (49/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 49.0 in stage 15.0 (TID 383). 33761 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 52.0 in stage 15.0 (TID 386) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 52.0 in stage 15.0 (TID 386) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 49.0 in stage 15.0 (TID 383) in 5 ms on 10.0.0.133 (executor driver) (50/200) +26/04/01 08:43:29 INFO Executor: Finished task 50.0 in stage 15.0 (TID 384). 34418 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 53.0 in stage 15.0 (TID 387) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 53.0 in stage 15.0 (TID 387) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 50.0 in stage 15.0 (TID 384) in 4 ms on 10.0.0.133 (executor driver) (51/200) +26/04/01 08:43:29 INFO Executor: Finished task 51.0 in stage 15.0 (TID 385). 34243 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 54.0 in stage 15.0 (TID 388) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 54.0 in stage 15.0 (TID 388) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 51.0 in stage 15.0 (TID 385) in 4 ms on 10.0.0.133 (executor driver) (52/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 54.0 in stage 15.0 (TID 388). 34702 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 52.0 in stage 15.0 (TID 386). 34119 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 53.0 in stage 15.0 (TID 387). 34449 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 55.0 in stage 15.0 (TID 389) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 55.0 in stage 15.0 (TID 389) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 56.0 in stage 15.0 (TID 390) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 54.0 in stage 15.0 (TID 388) in 5 ms on 10.0.0.133 (executor driver) (53/200) +26/04/01 08:43:29 INFO Executor: Running task 56.0 in stage 15.0 (TID 390) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 57.0 in stage 15.0 (TID 391) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 57.0 in stage 15.0 (TID 391) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 52.0 in stage 15.0 (TID 386) in 5 ms on 10.0.0.133 (executor driver) (54/200) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 53.0 in stage 15.0 (TID 387) in 5 ms on 10.0.0.133 (executor driver) (55/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 56.0 in stage 15.0 (TID 390). 34901 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 55.0 in stage 15.0 (TID 389). 34244 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 58.0 in stage 15.0 (TID 392) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 58.0 in stage 15.0 (TID 392) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 59.0 in stage 15.0 (TID 393) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 56.0 in stage 15.0 (TID 390) in 4 ms on 10.0.0.133 (executor driver) (56/200) +26/04/01 08:43:29 INFO Executor: Running task 59.0 in stage 15.0 (TID 393) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 55.0 in stage 15.0 (TID 389) in 4 ms on 10.0.0.133 (executor driver) (57/200) +26/04/01 08:43:29 INFO Executor: Finished task 57.0 in stage 15.0 (TID 391). 34181 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 60.0 in stage 15.0 (TID 394) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 57.0 in stage 15.0 (TID 391) in 5 ms on 10.0.0.133 (executor driver) (58/200) +26/04/01 08:43:29 INFO Executor: Running task 60.0 in stage 15.0 (TID 394) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 58.0 in stage 15.0 (TID 392). 34726 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 61.0 in stage 15.0 (TID 395) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 61.0 in stage 15.0 (TID 395) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 58.0 in stage 15.0 (TID 392) in 4 ms on 10.0.0.133 (executor driver) (59/200) +26/04/01 08:43:29 INFO Executor: Finished task 59.0 in stage 15.0 (TID 393). 34502 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 62.0 in stage 15.0 (TID 396) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 62.0 in stage 15.0 (TID 396) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 59.0 in stage 15.0 (TID 393) in 5 ms on 10.0.0.133 (executor driver) (60/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO Executor: Finished task 60.0 in stage 15.0 (TID 394). 34580 bytes result sent to driver +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Starting task 63.0 in stage 15.0 (TID 397) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 63.0 in stage 15.0 (TID 397) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 60.0 in stage 15.0 (TID 394) in 5 ms on 10.0.0.133 (executor driver) (61/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 61.0 in stage 15.0 (TID 395). 33977 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 62.0 in stage 15.0 (TID 396). 34073 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 64.0 in stage 15.0 (TID 398) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 64.0 in stage 15.0 (TID 398) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 65.0 in stage 15.0 (TID 399) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 61.0 in stage 15.0 (TID 395) in 4 ms on 10.0.0.133 (executor driver) (62/200) +26/04/01 08:43:29 INFO Executor: Running task 65.0 in stage 15.0 (TID 399) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 62.0 in stage 15.0 (TID 396) in 4 ms on 10.0.0.133 (executor driver) (63/200) +26/04/01 08:43:29 INFO Executor: Finished task 63.0 in stage 15.0 (TID 397). 33522 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 66.0 in stage 15.0 (TID 400) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 66.0 in stage 15.0 (TID 400) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 63.0 in stage 15.0 (TID 397) in 4 ms on 10.0.0.133 (executor driver) (64/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 64.0 in stage 15.0 (TID 398). 33929 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 67.0 in stage 15.0 (TID 401) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Finished task 65.0 in stage 15.0 (TID 399). 34066 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Running task 67.0 in stage 15.0 (TID 401) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 68.0 in stage 15.0 (TID 402) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 64.0 in stage 15.0 (TID 398) in 5 ms on 10.0.0.133 (executor driver) (65/200) +26/04/01 08:43:29 INFO Executor: Running task 68.0 in stage 15.0 (TID 402) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 65.0 in stage 15.0 (TID 399) in 5 ms on 10.0.0.133 (executor driver) (66/200) +26/04/01 08:43:29 INFO Executor: Finished task 66.0 in stage 15.0 (TID 400). 34407 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 69.0 in stage 15.0 (TID 403) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 69.0 in stage 15.0 (TID 403) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 66.0 in stage 15.0 (TID 400) in 4 ms on 10.0.0.133 (executor driver) (67/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 67.0 in stage 15.0 (TID 401). 34324 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 70.0 in stage 15.0 (TID 404) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 70.0 in stage 15.0 (TID 404) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 67.0 in stage 15.0 (TID 401) in 3 ms on 10.0.0.133 (executor driver) (68/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 68.0 in stage 15.0 (TID 402). 33789 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 69.0 in stage 15.0 (TID 403). 33810 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 71.0 in stage 15.0 (TID 405) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 71.0 in stage 15.0 (TID 405) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 72.0 in stage 15.0 (TID 406) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 68.0 in stage 15.0 (TID 402) in 5 ms on 10.0.0.133 (executor driver) (69/200) +26/04/01 08:43:29 INFO Executor: Running task 72.0 in stage 15.0 (TID 406) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 69.0 in stage 15.0 (TID 403) in 5 ms on 10.0.0.133 (executor driver) (70/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 70.0 in stage 15.0 (TID 404). 34912 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 73.0 in stage 15.0 (TID 407) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 73.0 in stage 15.0 (TID 407) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 70.0 in stage 15.0 (TID 404) in 4 ms on 10.0.0.133 (executor driver) (71/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 71.0 in stage 15.0 (TID 405). 34265 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 74.0 in stage 15.0 (TID 408) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 74.0 in stage 15.0 (TID 408) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 71.0 in stage 15.0 (TID 405) in 5 ms on 10.0.0.133 (executor driver) (72/200) +26/04/01 08:43:29 INFO Executor: Finished task 72.0 in stage 15.0 (TID 406). 34272 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 75.0 in stage 15.0 (TID 409) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 75.0 in stage 15.0 (TID 409) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 72.0 in stage 15.0 (TID 406) in 4 ms on 10.0.0.133 (executor driver) (73/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 73.0 in stage 15.0 (TID 407). 34476 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 76.0 in stage 15.0 (TID 410) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 76.0 in stage 15.0 (TID 410) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 73.0 in stage 15.0 (TID 407) in 4 ms on 10.0.0.133 (executor driver) (74/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 74.0 in stage 15.0 (TID 408). 33908 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 75.0 in stage 15.0 (TID 409). 34266 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 77.0 in stage 15.0 (TID 411) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 77.0 in stage 15.0 (TID 411) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 78.0 in stage 15.0 (TID 412) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 75.0 in stage 15.0 (TID 409) in 4 ms on 10.0.0.133 (executor driver) (75/200) +26/04/01 08:43:29 INFO Executor: Running task 78.0 in stage 15.0 (TID 412) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 74.0 in stage 15.0 (TID 408) in 5 ms on 10.0.0.133 (executor driver) (76/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 76.0 in stage 15.0 (TID 410). 34196 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 79.0 in stage 15.0 (TID 413) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 79.0 in stage 15.0 (TID 413) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 76.0 in stage 15.0 (TID 410) in 4 ms on 10.0.0.133 (executor driver) (77/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 78.0 in stage 15.0 (TID 412). 34141 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 77.0 in stage 15.0 (TID 411). 34812 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 80.0 in stage 15.0 (TID 414) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 80.0 in stage 15.0 (TID 414) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 81.0 in stage 15.0 (TID 415) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 77.0 in stage 15.0 (TID 411) in 4 ms on 10.0.0.133 (executor driver) (78/200) +26/04/01 08:43:29 INFO Executor: Running task 81.0 in stage 15.0 (TID 415) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 78.0 in stage 15.0 (TID 412) in 4 ms on 10.0.0.133 (executor driver) (79/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.6 KiB) non-empty blocks including 8 (31.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 79.0 in stage 15.0 (TID 413). 34085 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 82.0 in stage 15.0 (TID 416) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 82.0 in stage 15.0 (TID 416) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 79.0 in stage 15.0 (TID 413) in 4 ms on 10.0.0.133 (executor driver) (80/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 80.0 in stage 15.0 (TID 414). 34593 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 83.0 in stage 15.0 (TID 417) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 83.0 in stage 15.0 (TID 417) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 80.0 in stage 15.0 (TID 414) in 4 ms on 10.0.0.133 (executor driver) (81/200) +26/04/01 08:43:29 INFO Executor: Finished task 81.0 in stage 15.0 (TID 415). 33935 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 84.0 in stage 15.0 (TID 418) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 84.0 in stage 15.0 (TID 418) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 81.0 in stage 15.0 (TID 415) in 4 ms on 10.0.0.133 (executor driver) (82/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 82.0 in stage 15.0 (TID 416). 34466 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 85.0 in stage 15.0 (TID 419) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 85.0 in stage 15.0 (TID 419) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 82.0 in stage 15.0 (TID 416) in 4 ms on 10.0.0.133 (executor driver) (83/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 83.0 in stage 15.0 (TID 417). 34273 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 84.0 in stage 15.0 (TID 418). 33511 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 86.0 in stage 15.0 (TID 420) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 86.0 in stage 15.0 (TID 420) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 87.0 in stage 15.0 (TID 421) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 83.0 in stage 15.0 (TID 417) in 5 ms on 10.0.0.133 (executor driver) (84/200) +26/04/01 08:43:29 INFO Executor: Running task 87.0 in stage 15.0 (TID 421) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 84.0 in stage 15.0 (TID 418) in 4 ms on 10.0.0.133 (executor driver) (85/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 85.0 in stage 15.0 (TID 419). 34506 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 88.0 in stage 15.0 (TID 422) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 88.0 in stage 15.0 (TID 422) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 85.0 in stage 15.0 (TID 419) in 3 ms on 10.0.0.133 (executor driver) (86/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 86.0 in stage 15.0 (TID 420). 34439 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 89.0 in stage 15.0 (TID 423) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 89.0 in stage 15.0 (TID 423) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 86.0 in stage 15.0 (TID 420) in 6 ms on 10.0.0.133 (executor driver) (87/200) +26/04/01 08:43:29 INFO Executor: Finished task 87.0 in stage 15.0 (TID 421). 33817 bytes result sent to driver +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Starting task 90.0 in stage 15.0 (TID 424) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 90.0 in stage 15.0 (TID 424) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 87.0 in stage 15.0 (TID 421) in 7 ms on 10.0.0.133 (executor driver) (88/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 88.0 in stage 15.0 (TID 422). 34831 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 91.0 in stage 15.0 (TID 425) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 91.0 in stage 15.0 (TID 425) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 88.0 in stage 15.0 (TID 422) in 7 ms on 10.0.0.133 (executor driver) (89/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 89.0 in stage 15.0 (TID 423). 33927 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 92.0 in stage 15.0 (TID 426) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 92.0 in stage 15.0 (TID 426) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 89.0 in stage 15.0 (TID 423) in 7 ms on 10.0.0.133 (executor driver) (90/200) +26/04/01 08:43:29 INFO Executor: Finished task 90.0 in stage 15.0 (TID 424). 33756 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 93.0 in stage 15.0 (TID 427) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Running task 93.0 in stage 15.0 (TID 427) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 90.0 in stage 15.0 (TID 424) in 4 ms on 10.0.0.133 (executor driver) (91/200) +26/04/01 08:43:29 INFO Executor: Finished task 91.0 in stage 15.0 (TID 425). 33835 bytes result sent to driver +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO TaskSetManager: Starting task 94.0 in stage 15.0 (TID 428) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Finished task 91.0 in stage 15.0 (TID 425) in 4 ms on 10.0.0.133 (executor driver) (92/200) +26/04/01 08:43:29 INFO Executor: Running task 94.0 in stage 15.0 (TID 428) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 92.0 in stage 15.0 (TID 426). 33953 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 95.0 in stage 15.0 (TID 429) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 95.0 in stage 15.0 (TID 429) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 92.0 in stage 15.0 (TID 426) in 4 ms on 10.0.0.133 (executor driver) (93/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 93.0 in stage 15.0 (TID 427). 33995 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 96.0 in stage 15.0 (TID 430) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 96.0 in stage 15.0 (TID 430) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 93.0 in stage 15.0 (TID 427) in 4 ms on 10.0.0.133 (executor driver) (94/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO Executor: Finished task 94.0 in stage 15.0 (TID 428). 34551 bytes result sent to driver +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Starting task 97.0 in stage 15.0 (TID 431) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 97.0 in stage 15.0 (TID 431) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 94.0 in stage 15.0 (TID 428) in 5 ms on 10.0.0.133 (executor driver) (95/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 95.0 in stage 15.0 (TID 429). 33891 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 98.0 in stage 15.0 (TID 432) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 98.0 in stage 15.0 (TID 432) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 95.0 in stage 15.0 (TID 429) in 4 ms on 10.0.0.133 (executor driver) (96/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 96.0 in stage 15.0 (TID 430). 34205 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 99.0 in stage 15.0 (TID 433) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 99.0 in stage 15.0 (TID 433) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 96.0 in stage 15.0 (TID 430) in 4 ms on 10.0.0.133 (executor driver) (97/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO Executor: Finished task 97.0 in stage 15.0 (TID 431). 33883 bytes result sent to driver +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Starting task 100.0 in stage 15.0 (TID 434) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 100.0 in stage 15.0 (TID 434) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 97.0 in stage 15.0 (TID 431) in 5 ms on 10.0.0.133 (executor driver) (98/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 98.0 in stage 15.0 (TID 432). 34607 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 101.0 in stage 15.0 (TID 435) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 101.0 in stage 15.0 (TID 435) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 98.0 in stage 15.0 (TID 432) in 3 ms on 10.0.0.133 (executor driver) (99/200) +26/04/01 08:43:29 INFO Executor: Finished task 99.0 in stage 15.0 (TID 433). 34229 bytes result sent to driver +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Starting task 102.0 in stage 15.0 (TID 436) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 102.0 in stage 15.0 (TID 436) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 99.0 in stage 15.0 (TID 433) in 3 ms on 10.0.0.133 (executor driver) (100/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 100.0 in stage 15.0 (TID 434). 34078 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 103.0 in stage 15.0 (TID 437) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 103.0 in stage 15.0 (TID 437) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 100.0 in stage 15.0 (TID 434) in 4 ms on 10.0.0.133 (executor driver) (101/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 101.0 in stage 15.0 (TID 435). 33625 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 104.0 in stage 15.0 (TID 438) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 104.0 in stage 15.0 (TID 438) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 101.0 in stage 15.0 (TID 435) in 4 ms on 10.0.0.133 (executor driver) (102/200) +26/04/01 08:43:29 INFO Executor: Finished task 102.0 in stage 15.0 (TID 436). 34103 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 105.0 in stage 15.0 (TID 439) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 105.0 in stage 15.0 (TID 439) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 102.0 in stage 15.0 (TID 436) in 3 ms on 10.0.0.133 (executor driver) (103/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 103.0 in stage 15.0 (TID 437). 34407 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 106.0 in stage 15.0 (TID 440) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 106.0 in stage 15.0 (TID 440) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 103.0 in stage 15.0 (TID 437) in 3 ms on 10.0.0.133 (executor driver) (104/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 104.0 in stage 15.0 (TID 438). 34155 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 107.0 in stage 15.0 (TID 441) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 107.0 in stage 15.0 (TID 441) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 104.0 in stage 15.0 (TID 438) in 4 ms on 10.0.0.133 (executor driver) (105/200) +26/04/01 08:43:29 INFO Executor: Finished task 105.0 in stage 15.0 (TID 439). 33868 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 108.0 in stage 15.0 (TID 442) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO Executor: Running task 108.0 in stage 15.0 (TID 442) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 106.0 in stage 15.0 (TID 440). 33851 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Finished task 105.0 in stage 15.0 (TID 439) in 5 ms on 10.0.0.133 (executor driver) (106/200) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 109.0 in stage 15.0 (TID 443) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 109.0 in stage 15.0 (TID 443) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 106.0 in stage 15.0 (TID 440) in 4 ms on 10.0.0.133 (executor driver) (107/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 107.0 in stage 15.0 (TID 441). 33898 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 110.0 in stage 15.0 (TID 444) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Finished task 108.0 in stage 15.0 (TID 442). 34303 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Running task 110.0 in stage 15.0 (TID 444) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 107.0 in stage 15.0 (TID 441) in 4 ms on 10.0.0.133 (executor driver) (108/200) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 111.0 in stage 15.0 (TID 445) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 111.0 in stage 15.0 (TID 445) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 108.0 in stage 15.0 (TID 442) in 3 ms on 10.0.0.133 (executor driver) (109/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO Executor: Finished task 109.0 in stage 15.0 (TID 443). 34424 bytes result sent to driver +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Starting task 112.0 in stage 15.0 (TID 446) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 112.0 in stage 15.0 (TID 446) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 109.0 in stage 15.0 (TID 443) in 4 ms on 10.0.0.133 (executor driver) (110/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 110.0 in stage 15.0 (TID 444). 34141 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 113.0 in stage 15.0 (TID 447) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 113.0 in stage 15.0 (TID 447) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 110.0 in stage 15.0 (TID 444) in 4 ms on 10.0.0.133 (executor driver) (111/200) +26/04/01 08:43:29 INFO Executor: Finished task 111.0 in stage 15.0 (TID 445). 34432 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 114.0 in stage 15.0 (TID 448) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 114.0 in stage 15.0 (TID 448) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 111.0 in stage 15.0 (TID 445) in 4 ms on 10.0.0.133 (executor driver) (112/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 112.0 in stage 15.0 (TID 446). 34551 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 115.0 in stage 15.0 (TID 449) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 115.0 in stage 15.0 (TID 449) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 112.0 in stage 15.0 (TID 446) in 4 ms on 10.0.0.133 (executor driver) (113/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 113.0 in stage 15.0 (TID 447). 34235 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 116.0 in stage 15.0 (TID 450) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 116.0 in stage 15.0 (TID 450) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 113.0 in stage 15.0 (TID 447) in 4 ms on 10.0.0.133 (executor driver) (114/200) +26/04/01 08:43:29 INFO Executor: Finished task 114.0 in stage 15.0 (TID 448). 34058 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 117.0 in stage 15.0 (TID 451) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Finished task 115.0 in stage 15.0 (TID 449). 33531 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Running task 117.0 in stage 15.0 (TID 451) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 118.0 in stage 15.0 (TID 452) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Finished task 114.0 in stage 15.0 (TID 448) in 5 ms on 10.0.0.133 (executor driver) (115/200) +26/04/01 08:43:29 INFO Executor: Running task 118.0 in stage 15.0 (TID 452) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 115.0 in stage 15.0 (TID 449) in 4 ms on 10.0.0.133 (executor driver) (116/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 116.0 in stage 15.0 (TID 450). 34105 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 119.0 in stage 15.0 (TID 453) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 119.0 in stage 15.0 (TID 453) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 116.0 in stage 15.0 (TID 450) in 4 ms on 10.0.0.133 (executor driver) (117/200) +26/04/01 08:43:29 INFO Executor: Finished task 117.0 in stage 15.0 (TID 451). 34192 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 120.0 in stage 15.0 (TID 454) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 120.0 in stage 15.0 (TID 454) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO TaskSetManager: Finished task 117.0 in stage 15.0 (TID 451) in 5 ms on 10.0.0.133 (executor driver) (118/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 118.0 in stage 15.0 (TID 452). 34647 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 121.0 in stage 15.0 (TID 455) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 121.0 in stage 15.0 (TID 455) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 118.0 in stage 15.0 (TID 452) in 5 ms on 10.0.0.133 (executor driver) (119/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 119.0 in stage 15.0 (TID 453). 33642 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 122.0 in stage 15.0 (TID 456) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 122.0 in stage 15.0 (TID 456) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 119.0 in stage 15.0 (TID 453) in 4 ms on 10.0.0.133 (executor driver) (120/200) +26/04/01 08:43:29 INFO Executor: Finished task 120.0 in stage 15.0 (TID 454). 33991 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 121.0 in stage 15.0 (TID 455). 34107 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 123.0 in stage 15.0 (TID 457) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 123.0 in stage 15.0 (TID 457) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 124.0 in stage 15.0 (TID 458) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 120.0 in stage 15.0 (TID 454) in 5 ms on 10.0.0.133 (executor driver) (121/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO Executor: Running task 124.0 in stage 15.0 (TID 458) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Finished task 121.0 in stage 15.0 (TID 455) in 4 ms on 10.0.0.133 (executor driver) (122/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 122.0 in stage 15.0 (TID 456). 34345 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 125.0 in stage 15.0 (TID 459) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 125.0 in stage 15.0 (TID 459) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 122.0 in stage 15.0 (TID 456) in 4 ms on 10.0.0.133 (executor driver) (123/200) +26/04/01 08:43:29 INFO Executor: Finished task 124.0 in stage 15.0 (TID 458). 34152 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 123.0 in stage 15.0 (TID 457). 34123 bytes result sent to driver +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Starting task 126.0 in stage 15.0 (TID 460) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 126.0 in stage 15.0 (TID 460) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 127.0 in stage 15.0 (TID 461) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 124.0 in stage 15.0 (TID 458) in 4 ms on 10.0.0.133 (executor driver) (124/200) +26/04/01 08:43:29 INFO Executor: Running task 127.0 in stage 15.0 (TID 461) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 123.0 in stage 15.0 (TID 457) in 5 ms on 10.0.0.133 (executor driver) (125/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 125.0 in stage 15.0 (TID 459). 34554 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 128.0 in stage 15.0 (TID 462) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 128.0 in stage 15.0 (TID 462) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 125.0 in stage 15.0 (TID 459) in 4 ms on 10.0.0.133 (executor driver) (126/200) +26/04/01 08:43:29 INFO Executor: Finished task 126.0 in stage 15.0 (TID 460). 33922 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 129.0 in stage 15.0 (TID 463) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 129.0 in stage 15.0 (TID 463) +26/04/01 08:43:29 INFO Executor: Finished task 127.0 in stage 15.0 (TID 461). 34650 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Finished task 126.0 in stage 15.0 (TID 460) in 4 ms on 10.0.0.133 (executor driver) (127/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Starting task 130.0 in stage 15.0 (TID 464) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 130.0 in stage 15.0 (TID 464) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 127.0 in stage 15.0 (TID 461) in 4 ms on 10.0.0.133 (executor driver) (128/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.6 KiB) non-empty blocks including 8 (31.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 128.0 in stage 15.0 (TID 462). 34392 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 131.0 in stage 15.0 (TID 465) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 128.0 in stage 15.0 (TID 462) in 7 ms on 10.0.0.133 (executor driver) (129/200) +26/04/01 08:43:29 INFO Executor: Running task 131.0 in stage 15.0 (TID 465) +26/04/01 08:43:29 INFO Executor: Finished task 130.0 in stage 15.0 (TID 464). 35133 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 132.0 in stage 15.0 (TID 466) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Finished task 129.0 in stage 15.0 (TID 463). 34055 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Finished task 130.0 in stage 15.0 (TID 464) in 6 ms on 10.0.0.133 (executor driver) (130/200) +26/04/01 08:43:29 INFO Executor: Running task 132.0 in stage 15.0 (TID 466) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 133.0 in stage 15.0 (TID 467) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 133.0 in stage 15.0 (TID 467) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 129.0 in stage 15.0 (TID 463) in 6 ms on 10.0.0.133 (executor driver) (131/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 131.0 in stage 15.0 (TID 465). 33973 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 134.0 in stage 15.0 (TID 468) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 134.0 in stage 15.0 (TID 468) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 131.0 in stage 15.0 (TID 465) in 4 ms on 10.0.0.133 (executor driver) (132/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 132.0 in stage 15.0 (TID 466). 34084 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 133.0 in stage 15.0 (TID 467). 33764 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 135.0 in stage 15.0 (TID 469) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 135.0 in stage 15.0 (TID 469) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 136.0 in stage 15.0 (TID 470) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 132.0 in stage 15.0 (TID 466) in 5 ms on 10.0.0.133 (executor driver) (133/200) +26/04/01 08:43:29 INFO Executor: Running task 136.0 in stage 15.0 (TID 470) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 133.0 in stage 15.0 (TID 467) in 5 ms on 10.0.0.133 (executor driver) (134/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 134.0 in stage 15.0 (TID 468). 33712 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 137.0 in stage 15.0 (TID 471) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 137.0 in stage 15.0 (TID 471) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 134.0 in stage 15.0 (TID 468) in 4 ms on 10.0.0.133 (executor driver) (135/200) +26/04/01 08:43:29 INFO Executor: Finished task 135.0 in stage 15.0 (TID 469). 34404 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 138.0 in stage 15.0 (TID 472) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 138.0 in stage 15.0 (TID 472) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 135.0 in stage 15.0 (TID 469) in 3 ms on 10.0.0.133 (executor driver) (136/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 136.0 in stage 15.0 (TID 470). 34430 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 139.0 in stage 15.0 (TID 473) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 139.0 in stage 15.0 (TID 473) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 136.0 in stage 15.0 (TID 470) in 4 ms on 10.0.0.133 (executor driver) (137/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 137.0 in stage 15.0 (TID 471). 34756 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 140.0 in stage 15.0 (TID 474) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 140.0 in stage 15.0 (TID 474) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 137.0 in stage 15.0 (TID 471) in 5 ms on 10.0.0.133 (executor driver) (138/200) +26/04/01 08:43:29 INFO Executor: Finished task 138.0 in stage 15.0 (TID 472). 33693 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 139.0 in stage 15.0 (TID 473). 34378 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 141.0 in stage 15.0 (TID 475) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 141.0 in stage 15.0 (TID 475) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 142.0 in stage 15.0 (TID 476) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 138.0 in stage 15.0 (TID 472) in 4 ms on 10.0.0.133 (executor driver) (139/200) +26/04/01 08:43:29 INFO Executor: Running task 142.0 in stage 15.0 (TID 476) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 139.0 in stage 15.0 (TID 473) in 4 ms on 10.0.0.133 (executor driver) (140/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 140.0 in stage 15.0 (TID 474). 33890 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 143.0 in stage 15.0 (TID 477) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 143.0 in stage 15.0 (TID 477) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 140.0 in stage 15.0 (TID 474) in 5 ms on 10.0.0.133 (executor driver) (141/200) +26/04/01 08:43:29 INFO Executor: Finished task 141.0 in stage 15.0 (TID 475). 34328 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 142.0 in stage 15.0 (TID 476). 34951 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 144.0 in stage 15.0 (TID 478) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO Executor: Running task 144.0 in stage 15.0 (TID 478) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Starting task 145.0 in stage 15.0 (TID 479) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 141.0 in stage 15.0 (TID 475) in 4 ms on 10.0.0.133 (executor driver) (142/200) +26/04/01 08:43:29 INFO Executor: Running task 145.0 in stage 15.0 (TID 479) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 142.0 in stage 15.0 (TID 476) in 4 ms on 10.0.0.133 (executor driver) (143/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 144.0 in stage 15.0 (TID 478). 33655 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 143.0 in stage 15.0 (TID 477). 34375 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 146.0 in stage 15.0 (TID 480) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 146.0 in stage 15.0 (TID 480) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 147.0 in stage 15.0 (TID 481) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 144.0 in stage 15.0 (TID 478) in 4 ms on 10.0.0.133 (executor driver) (144/200) +26/04/01 08:43:29 INFO Executor: Running task 147.0 in stage 15.0 (TID 481) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 143.0 in stage 15.0 (TID 477) in 5 ms on 10.0.0.133 (executor driver) (145/200) +26/04/01 08:43:29 INFO Executor: Finished task 145.0 in stage 15.0 (TID 479). 34189 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 148.0 in stage 15.0 (TID 482) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO Executor: Running task 148.0 in stage 15.0 (TID 482) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 145.0 in stage 15.0 (TID 479) in 4 ms on 10.0.0.133 (executor driver) (146/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 148.0 in stage 15.0 (TID 482). 34244 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 149.0 in stage 15.0 (TID 483) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Finished task 146.0 in stage 15.0 (TID 480). 33726 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 147.0 in stage 15.0 (TID 481). 33597 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Running task 149.0 in stage 15.0 (TID 483) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 148.0 in stage 15.0 (TID 482) in 4 ms on 10.0.0.133 (executor driver) (147/200) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 150.0 in stage 15.0 (TID 484) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 150.0 in stage 15.0 (TID 484) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 151.0 in stage 15.0 (TID 485) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 146.0 in stage 15.0 (TID 480) in 4 ms on 10.0.0.133 (executor driver) (148/200) +26/04/01 08:43:29 INFO Executor: Running task 151.0 in stage 15.0 (TID 485) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 147.0 in stage 15.0 (TID 481) in 4 ms on 10.0.0.133 (executor driver) (149/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 151.0 in stage 15.0 (TID 485). 34392 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 150.0 in stage 15.0 (TID 484). 34597 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 149.0 in stage 15.0 (TID 483). 34799 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 152.0 in stage 15.0 (TID 486) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 152.0 in stage 15.0 (TID 486) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 153.0 in stage 15.0 (TID 487) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 151.0 in stage 15.0 (TID 485) in 4 ms on 10.0.0.133 (executor driver) (150/200) +26/04/01 08:43:29 INFO Executor: Running task 153.0 in stage 15.0 (TID 487) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 154.0 in stage 15.0 (TID 488) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 150.0 in stage 15.0 (TID 484) in 5 ms on 10.0.0.133 (executor driver) (151/200) +26/04/01 08:43:29 INFO Executor: Running task 154.0 in stage 15.0 (TID 488) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 149.0 in stage 15.0 (TID 483) in 5 ms on 10.0.0.133 (executor driver) (152/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 154.0 in stage 15.0 (TID 488). 33726 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 152.0 in stage 15.0 (TID 486). 34441 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 155.0 in stage 15.0 (TID 489) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Finished task 153.0 in stage 15.0 (TID 487). 34127 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Running task 155.0 in stage 15.0 (TID 489) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 156.0 in stage 15.0 (TID 490) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 154.0 in stage 15.0 (TID 488) in 4 ms on 10.0.0.133 (executor driver) (153/200) +26/04/01 08:43:29 INFO Executor: Running task 156.0 in stage 15.0 (TID 490) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 152.0 in stage 15.0 (TID 486) in 5 ms on 10.0.0.133 (executor driver) (154/200) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 157.0 in stage 15.0 (TID 491) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 157.0 in stage 15.0 (TID 491) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 153.0 in stage 15.0 (TID 487) in 5 ms on 10.0.0.133 (executor driver) (155/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 155.0 in stage 15.0 (TID 489). 33738 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 158.0 in stage 15.0 (TID 492) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 158.0 in stage 15.0 (TID 492) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 155.0 in stage 15.0 (TID 489) in 4 ms on 10.0.0.133 (executor driver) (156/200) +26/04/01 08:43:29 INFO Executor: Finished task 157.0 in stage 15.0 (TID 491). 33697 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 156.0 in stage 15.0 (TID 490). 34636 bytes result sent to driver +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO TaskSetManager: Starting task 159.0 in stage 15.0 (TID 493) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 159.0 in stage 15.0 (TID 493) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 160.0 in stage 15.0 (TID 494) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 157.0 in stage 15.0 (TID 491) in 4 ms on 10.0.0.133 (executor driver) (157/200) +26/04/01 08:43:29 INFO Executor: Running task 160.0 in stage 15.0 (TID 494) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 156.0 in stage 15.0 (TID 490) in 4 ms on 10.0.0.133 (executor driver) (158/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 158.0 in stage 15.0 (TID 492). 34759 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 161.0 in stage 15.0 (TID 495) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 161.0 in stage 15.0 (TID 495) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 158.0 in stage 15.0 (TID 492) in 4 ms on 10.0.0.133 (executor driver) (159/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 159.0 in stage 15.0 (TID 493). 34519 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 160.0 in stage 15.0 (TID 494). 34867 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 162.0 in stage 15.0 (TID 496) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 162.0 in stage 15.0 (TID 496) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 163.0 in stage 15.0 (TID 497) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 159.0 in stage 15.0 (TID 493) in 5 ms on 10.0.0.133 (executor driver) (160/200) +26/04/01 08:43:29 INFO Executor: Running task 163.0 in stage 15.0 (TID 497) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 160.0 in stage 15.0 (TID 494) in 5 ms on 10.0.0.133 (executor driver) (161/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 161.0 in stage 15.0 (TID 495). 33835 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 164.0 in stage 15.0 (TID 498) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 164.0 in stage 15.0 (TID 498) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 161.0 in stage 15.0 (TID 495) in 5 ms on 10.0.0.133 (executor driver) (162/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 163.0 in stage 15.0 (TID 497). 34082 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 162.0 in stage 15.0 (TID 496). 34295 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 165.0 in stage 15.0 (TID 499) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 165.0 in stage 15.0 (TID 499) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 166.0 in stage 15.0 (TID 500) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 163.0 in stage 15.0 (TID 497) in 4 ms on 10.0.0.133 (executor driver) (163/200) +26/04/01 08:43:29 INFO Executor: Running task 166.0 in stage 15.0 (TID 500) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 162.0 in stage 15.0 (TID 496) in 4 ms on 10.0.0.133 (executor driver) (164/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 164.0 in stage 15.0 (TID 498). 33851 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 167.0 in stage 15.0 (TID 501) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 167.0 in stage 15.0 (TID 501) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 164.0 in stage 15.0 (TID 498) in 4 ms on 10.0.0.133 (executor driver) (165/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 165.0 in stage 15.0 (TID 499). 34396 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 166.0 in stage 15.0 (TID 500). 34848 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 168.0 in stage 15.0 (TID 502) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 168.0 in stage 15.0 (TID 502) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 169.0 in stage 15.0 (TID 503) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 165.0 in stage 15.0 (TID 499) in 4 ms on 10.0.0.133 (executor driver) (166/200) +26/04/01 08:43:29 INFO Executor: Running task 169.0 in stage 15.0 (TID 503) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 166.0 in stage 15.0 (TID 500) in 4 ms on 10.0.0.133 (executor driver) (167/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 167.0 in stage 15.0 (TID 501). 33887 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 170.0 in stage 15.0 (TID 504) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 170.0 in stage 15.0 (TID 504) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 167.0 in stage 15.0 (TID 501) in 4 ms on 10.0.0.133 (executor driver) (168/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 169.0 in stage 15.0 (TID 503). 34471 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 171.0 in stage 15.0 (TID 505) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 171.0 in stage 15.0 (TID 505) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 169.0 in stage 15.0 (TID 503) in 4 ms on 10.0.0.133 (executor driver) (169/200) +26/04/01 08:43:29 INFO Executor: Finished task 168.0 in stage 15.0 (TID 502). 33588 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 172.0 in stage 15.0 (TID 506) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 172.0 in stage 15.0 (TID 506) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 168.0 in stage 15.0 (TID 502) in 4 ms on 10.0.0.133 (executor driver) (170/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 170.0 in stage 15.0 (TID 504). 34026 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 173.0 in stage 15.0 (TID 507) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 170.0 in stage 15.0 (TID 504) in 4 ms on 10.0.0.133 (executor driver) (171/200) +26/04/01 08:43:29 INFO Executor: Running task 173.0 in stage 15.0 (TID 507) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 171.0 in stage 15.0 (TID 505). 34142 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 174.0 in stage 15.0 (TID 508) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Finished task 172.0 in stage 15.0 (TID 506). 33962 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Running task 174.0 in stage 15.0 (TID 508) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 171.0 in stage 15.0 (TID 505) in 6 ms on 10.0.0.133 (executor driver) (172/200) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 175.0 in stage 15.0 (TID 509) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 172.0 in stage 15.0 (TID 506) in 7 ms on 10.0.0.133 (executor driver) (173/200) +26/04/01 08:43:29 INFO Executor: Running task 175.0 in stage 15.0 (TID 509) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 173.0 in stage 15.0 (TID 507). 34609 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 176.0 in stage 15.0 (TID 510) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 176.0 in stage 15.0 (TID 510) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 173.0 in stage 15.0 (TID 507) in 6 ms on 10.0.0.133 (executor driver) (174/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 174.0 in stage 15.0 (TID 508). 33575 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 177.0 in stage 15.0 (TID 511) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 174.0 in stage 15.0 (TID 508) in 4 ms on 10.0.0.133 (executor driver) (175/200) +26/04/01 08:43:29 INFO Executor: Running task 177.0 in stage 15.0 (TID 511) +26/04/01 08:43:29 INFO Executor: Finished task 175.0 in stage 15.0 (TID 509). 34342 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 178.0 in stage 15.0 (TID 512) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 178.0 in stage 15.0 (TID 512) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 175.0 in stage 15.0 (TID 509) in 3 ms on 10.0.0.133 (executor driver) (176/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 176.0 in stage 15.0 (TID 510). 33675 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 179.0 in stage 15.0 (TID 513) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 179.0 in stage 15.0 (TID 513) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 176.0 in stage 15.0 (TID 510) in 4 ms on 10.0.0.133 (executor driver) (177/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 178.0 in stage 15.0 (TID 512). 34255 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 177.0 in stage 15.0 (TID 511). 34399 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 180.0 in stage 15.0 (TID 514) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 180.0 in stage 15.0 (TID 514) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 181.0 in stage 15.0 (TID 515) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 178.0 in stage 15.0 (TID 512) in 4 ms on 10.0.0.133 (executor driver) (178/200) +26/04/01 08:43:29 INFO Executor: Running task 181.0 in stage 15.0 (TID 515) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 177.0 in stage 15.0 (TID 511) in 4 ms on 10.0.0.133 (executor driver) (179/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 179.0 in stage 15.0 (TID 513). 34030 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 182.0 in stage 15.0 (TID 516) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 182.0 in stage 15.0 (TID 516) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 179.0 in stage 15.0 (TID 513) in 4 ms on 10.0.0.133 (executor driver) (180/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 180.0 in stage 15.0 (TID 514). 34531 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 181.0 in stage 15.0 (TID 515). 34153 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 183.0 in stage 15.0 (TID 517) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 183.0 in stage 15.0 (TID 517) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 184.0 in stage 15.0 (TID 518) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 180.0 in stage 15.0 (TID 514) in 4 ms on 10.0.0.133 (executor driver) (181/200) +26/04/01 08:43:29 INFO Executor: Running task 184.0 in stage 15.0 (TID 518) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 181.0 in stage 15.0 (TID 515) in 4 ms on 10.0.0.133 (executor driver) (182/200) +26/04/01 08:43:29 INFO Executor: Finished task 182.0 in stage 15.0 (TID 516). 34252 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 185.0 in stage 15.0 (TID 519) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 185.0 in stage 15.0 (TID 519) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 182.0 in stage 15.0 (TID 516) in 4 ms on 10.0.0.133 (executor driver) (183/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.6 KiB) non-empty blocks including 8 (31.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 183.0 in stage 15.0 (TID 517). 34150 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 186.0 in stage 15.0 (TID 520) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Finished task 184.0 in stage 15.0 (TID 518). 34956 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Running task 186.0 in stage 15.0 (TID 520) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 187.0 in stage 15.0 (TID 521) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Finished task 185.0 in stage 15.0 (TID 519). 34506 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Finished task 183.0 in stage 15.0 (TID 517) in 4 ms on 10.0.0.133 (executor driver) (184/200) +26/04/01 08:43:29 INFO Executor: Running task 187.0 in stage 15.0 (TID 521) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 184.0 in stage 15.0 (TID 518) in 4 ms on 10.0.0.133 (executor driver) (185/200) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 188.0 in stage 15.0 (TID 522) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 188.0 in stage 15.0 (TID 522) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 185.0 in stage 15.0 (TID 519) in 4 ms on 10.0.0.133 (executor driver) (186/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 186.0 in stage 15.0 (TID 520). 33801 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 189.0 in stage 15.0 (TID 523) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 189.0 in stage 15.0 (TID 523) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 186.0 in stage 15.0 (TID 520) in 4 ms on 10.0.0.133 (executor driver) (187/200) +26/04/01 08:43:29 INFO Executor: Finished task 188.0 in stage 15.0 (TID 522). 34848 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 190.0 in stage 15.0 (TID 524) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 190.0 in stage 15.0 (TID 524) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 188.0 in stage 15.0 (TID 522) in 4 ms on 10.0.0.133 (executor driver) (188/200) +26/04/01 08:43:29 INFO Executor: Finished task 187.0 in stage 15.0 (TID 521). 34650 bytes result sent to driver +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO TaskSetManager: Starting task 191.0 in stage 15.0 (TID 525) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Running task 191.0 in stage 15.0 (TID 525) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 187.0 in stage 15.0 (TID 521) in 5 ms on 10.0.0.133 (executor driver) (189/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 189.0 in stage 15.0 (TID 523). 34333 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 192.0 in stage 15.0 (TID 526) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Finished task 190.0 in stage 15.0 (TID 524). 34397 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 193.0 in stage 15.0 (TID 527) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 192.0 in stage 15.0 (TID 526) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 189.0 in stage 15.0 (TID 523) in 4 ms on 10.0.0.133 (executor driver) (190/200) +26/04/01 08:43:29 INFO Executor: Running task 193.0 in stage 15.0 (TID 527) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 190.0 in stage 15.0 (TID 524) in 4 ms on 10.0.0.133 (executor driver) (191/200) +26/04/01 08:43:29 INFO Executor: Finished task 191.0 in stage 15.0 (TID 525). 34559 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 194.0 in stage 15.0 (TID 528) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 194.0 in stage 15.0 (TID 528) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 191.0 in stage 15.0 (TID 525) in 4 ms on 10.0.0.133 (executor driver) (192/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 194.0 in stage 15.0 (TID 528). 34348 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 193.0 in stage 15.0 (TID 527). 34499 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 192.0 in stage 15.0 (TID 526). 33410 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 195.0 in stage 15.0 (TID 529) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 195.0 in stage 15.0 (TID 529) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 196.0 in stage 15.0 (TID 530) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 194.0 in stage 15.0 (TID 528) in 5 ms on 10.0.0.133 (executor driver) (193/200) +26/04/01 08:43:29 INFO Executor: Running task 196.0 in stage 15.0 (TID 530) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 193.0 in stage 15.0 (TID 527) in 5 ms on 10.0.0.133 (executor driver) (194/200) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 197.0 in stage 15.0 (TID 531) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 197.0 in stage 15.0 (TID 531) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 192.0 in stage 15.0 (TID 526) in 5 ms on 10.0.0.133 (executor driver) (195/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 195.0 in stage 15.0 (TID 529). 34725 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 198.0 in stage 15.0 (TID 532) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 198.0 in stage 15.0 (TID 532) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 195.0 in stage 15.0 (TID 529) in 4 ms on 10.0.0.133 (executor driver) (196/200) +26/04/01 08:43:29 INFO Executor: Finished task 196.0 in stage 15.0 (TID 530). 34709 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 197.0 in stage 15.0 (TID 531). 34348 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 199.0 in stage 15.0 (TID 533) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9215 bytes) +26/04/01 08:43:29 INFO Executor: Running task 199.0 in stage 15.0 (TID 533) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 196.0 in stage 15.0 (TID 530) in 5 ms on 10.0.0.133 (executor driver) (197/200) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 197.0 in stage 15.0 (TID 531) in 4 ms on 10.0.0.133 (executor driver) (198/200) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO Executor: Finished task 198.0 in stage 15.0 (TID 532). 34063 bytes result sent to driver +26/04/01 08:43:29 INFO Executor: Finished task 199.0 in stage 15.0 (TID 533). 34766 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Finished task 198.0 in stage 15.0 (TID 532) in 4 ms on 10.0.0.133 (executor driver) (199/200) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 199.0 in stage 15.0 (TID 533) in 3 ms on 10.0.0.133 (executor driver) (200/200) +26/04/01 08:43:29 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool +26/04/01 08:43:29 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 231.012 s +26/04/01 08:43:29 INFO DAGScheduler: Job 14 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:43:29 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished +26/04/01 08:43:29 INFO DAGScheduler: Job 14 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 231.027845 s +26/04/01 08:43:29 INFO Utils: Coalesced 1600 broadcast batches into 1 (1088125 rows) +26/04/01 08:43:29 INFO MemoryStore: Block broadcast_22 stored as values in memory (estimated size 5.2 MiB, free 8.4 GiB) +26/04/01 08:43:29 INFO MemoryStore: Block broadcast_22_piece0 stored as bytes in memory (estimated size 4.0 MiB, free 8.4 GiB) +26/04/01 08:43:29 INFO BlockManagerInfo: Added broadcast_22_piece0 in memory on 10.0.0.133:59303 (size: 4.0 MiB, free: 8.6 GiB) +26/04/01 08:43:29 INFO MemoryStore: Block broadcast_22_piece1 stored as bytes in memory (estimated size 357.8 KiB, free 8.4 GiB) +26/04/01 08:43:29 INFO BlockManagerInfo: Added broadcast_22_piece1 in memory on 10.0.0.133:59303 (size: 357.8 KiB, free: 8.6 GiB) +26/04/01 08:43:29 INFO SparkContext: Created broadcast 22 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:43:29 INFO ShufflePartitionsUtil: For shuffle(1), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 08:43:29 INFO DAGScheduler: Registering RDD 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 5 +26/04/01 08:43:29 INFO DAGScheduler: Got map stage job 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions +26/04/01 08:43:29 INFO DAGScheduler: Final stage: ShuffleMapStage 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:43:29 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 16) +26/04/01 08:43:29 INFO DAGScheduler: Missing parents: List() +26/04/01 08:43:29 INFO DAGScheduler: Submitting ShuffleMapStage 17 (MapPartitionsRDD[39] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:43:29 INFO MemoryStore: Block broadcast_23 stored as values in memory (estimated size 21.7 KiB, free 8.4 GiB) +26/04/01 08:43:29 INFO MemoryStore: Block broadcast_23_piece0 stored as bytes in memory (estimated size 11.0 KiB, free 8.4 GiB) +26/04/01 08:43:29 INFO BlockManagerInfo: Added broadcast_23_piece0 in memory on 10.0.0.133:59303 (size: 11.0 KiB, free: 8.6 GiB) +26/04/01 08:43:29 INFO SparkContext: Created broadcast 23 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:43:29 INFO DAGScheduler: Submitting 200 missing tasks from ShuffleMapStage 17 (MapPartitionsRDD[39] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:43:29 INFO TaskSchedulerImpl: Adding task set 17.0 with 200 tasks resource profile 0 +26/04/01 08:43:29 INFO TaskSetManager: Starting task 0.0 in stage 17.0 (TID 534) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 1.0 in stage 17.0 (TID 535) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:29 INFO TaskSetManager: Starting task 2.0 in stage 17.0 (TID 536) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:29 INFO Executor: Running task 1.0 in stage 17.0 (TID 535) +26/04/01 08:43:29 INFO Executor: Running task 0.0 in stage 17.0 (TID 534) +26/04/01 08:43:29 INFO Executor: Running task 2.0 in stage 17.0 (TID 536) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO Executor: Finished task 59.0 in stage 13.0 (TID 329). 6606 bytes result sent to driver +26/04/01 08:43:29 INFO TaskSetManager: Starting task 3.0 in stage 17.0 (TID 537) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:29 INFO Executor: Running task 3.0 in stage 17.0 (TID 537) +26/04/01 08:43:29 INFO TaskSetManager: Finished task 59.0 in stage 13.0 (TID 329) in 1573 ms on 10.0.0.133 (executor driver) (60/64) +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:30 INFO Executor: Finished task 60.0 in stage 13.0 (TID 330). 6606 bytes result sent to driver +26/04/01 08:43:30 INFO TaskSetManager: Starting task 4.0 in stage 17.0 (TID 538) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:30 INFO Executor: Running task 4.0 in stage 17.0 (TID 538) +26/04/01 08:43:30 INFO TaskSetManager: Finished task 60.0 in stage 13.0 (TID 330) in 1595 ms on 10.0.0.133 (executor driver) (61/64) +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:30 INFO Executor: Finished task 61.0 in stage 13.0 (TID 331). 6606 bytes result sent to driver +26/04/01 08:43:30 INFO TaskSetManager: Starting task 5.0 in stage 17.0 (TID 539) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:30 INFO Executor: Running task 5.0 in stage 17.0 (TID 539) +26/04/01 08:43:30 INFO TaskSetManager: Finished task 61.0 in stage 13.0 (TID 331) in 1612 ms on 10.0.0.133 (executor driver) (62/64) +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:30 INFO Executor: Finished task 62.0 in stage 13.0 (TID 332). 6606 bytes result sent to driver +26/04/01 08:43:30 INFO TaskSetManager: Starting task 6.0 in stage 17.0 (TID 540) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:30 INFO TaskSetManager: Finished task 62.0 in stage 13.0 (TID 332) in 1592 ms on 10.0.0.133 (executor driver) (63/64) +26/04/01 08:43:30 INFO Executor: Running task 6.0 in stage 17.0 (TID 540) +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:30 INFO Executor: Finished task 63.0 in stage 13.0 (TID 333). 6606 bytes result sent to driver +26/04/01 08:43:30 INFO TaskSetManager: Starting task 7.0 in stage 17.0 (TID 541) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:30 INFO Executor: Running task 7.0 in stage 17.0 (TID 541) +26/04/01 08:43:30 INFO TaskSetManager: Finished task 63.0 in stage 13.0 (TID 333) in 1595 ms on 10.0.0.133 (executor driver) (64/64) +26/04/01 08:43:30 INFO TaskSchedulerImpl: Removed TaskSet 13.0, whose tasks have all completed, from pool +26/04/01 08:43:30 INFO DAGScheduler: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 234.724 s +26/04/01 08:43:30 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:43:30 INFO DAGScheduler: running: Set(ShuffleMapStage 17) +26/04/01 08:43:30 INFO DAGScheduler: waiting: Set() +26/04/01 08:43:30 INFO DAGScheduler: failed: Set() +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:31 INFO Executor: Finished task 0.0 in stage 17.0 (TID 534). 6401 bytes result sent to driver +26/04/01 08:43:31 INFO Executor: Finished task 1.0 in stage 17.0 (TID 535). 6401 bytes result sent to driver +26/04/01 08:43:31 INFO TaskSetManager: Starting task 8.0 in stage 17.0 (TID 542) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:31 INFO Executor: Running task 8.0 in stage 17.0 (TID 542) +26/04/01 08:43:31 INFO TaskSetManager: Starting task 9.0 in stage 17.0 (TID 543) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:31 INFO TaskSetManager: Finished task 0.0 in stage 17.0 (TID 534) in 2028 ms on 10.0.0.133 (executor driver) (1/200) +26/04/01 08:43:31 INFO Executor: Running task 9.0 in stage 17.0 (TID 543) +26/04/01 08:43:31 INFO TaskSetManager: Finished task 1.0 in stage 17.0 (TID 535) in 2027 ms on 10.0.0.133 (executor driver) (2/200) +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:31 INFO Executor: Finished task 2.0 in stage 17.0 (TID 536). 6401 bytes result sent to driver +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:31 INFO TaskSetManager: Starting task 10.0 in stage 17.0 (TID 544) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:31 INFO Executor: Running task 10.0 in stage 17.0 (TID 544) +26/04/01 08:43:31 INFO TaskSetManager: Finished task 2.0 in stage 17.0 (TID 536) in 2028 ms on 10.0.0.133 (executor driver) (3/200) +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:31 INFO Executor: Finished task 3.0 in stage 17.0 (TID 537). 6401 bytes result sent to driver +26/04/01 08:43:31 INFO TaskSetManager: Starting task 11.0 in stage 17.0 (TID 545) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:31 INFO TaskSetManager: Finished task 3.0 in stage 17.0 (TID 537) in 1947 ms on 10.0.0.133 (executor driver) (4/200) +26/04/01 08:43:31 INFO Executor: Running task 11.0 in stage 17.0 (TID 545) +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:32 INFO Executor: Finished task 4.0 in stage 17.0 (TID 538). 6401 bytes result sent to driver +26/04/01 08:43:32 INFO TaskSetManager: Starting task 12.0 in stage 17.0 (TID 546) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:32 INFO TaskSetManager: Finished task 4.0 in stage 17.0 (TID 538) in 1891 ms on 10.0.0.133 (executor driver) (5/200) +26/04/01 08:43:32 INFO Executor: Running task 12.0 in stage 17.0 (TID 546) +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:32 INFO Executor: Finished task 5.0 in stage 17.0 (TID 539). 6401 bytes result sent to driver +26/04/01 08:43:32 INFO TaskSetManager: Starting task 13.0 in stage 17.0 (TID 547) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:32 INFO Executor: Running task 13.0 in stage 17.0 (TID 547) +26/04/01 08:43:32 INFO TaskSetManager: Finished task 5.0 in stage 17.0 (TID 539) in 1879 ms on 10.0.0.133 (executor driver) (6/200) +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:32 INFO Executor: Finished task 6.0 in stage 17.0 (TID 540). 6401 bytes result sent to driver +26/04/01 08:43:32 INFO TaskSetManager: Starting task 14.0 in stage 17.0 (TID 548) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:32 INFO TaskSetManager: Finished task 6.0 in stage 17.0 (TID 540) in 1872 ms on 10.0.0.133 (executor driver) (7/200) +26/04/01 08:43:32 INFO Executor: Running task 14.0 in stage 17.0 (TID 548) +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:32 INFO Executor: Finished task 7.0 in stage 17.0 (TID 541). 6401 bytes result sent to driver +26/04/01 08:43:32 INFO TaskSetManager: Starting task 15.0 in stage 17.0 (TID 549) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:32 INFO Executor: Running task 15.0 in stage 17.0 (TID 549) +26/04/01 08:43:32 INFO TaskSetManager: Finished task 7.0 in stage 17.0 (TID 541) in 1906 ms on 10.0.0.133 (executor driver) (8/200) +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:33 INFO Executor: Finished task 8.0 in stage 17.0 (TID 542). 6401 bytes result sent to driver +26/04/01 08:43:33 INFO TaskSetManager: Starting task 16.0 in stage 17.0 (TID 550) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:33 INFO TaskSetManager: Finished task 8.0 in stage 17.0 (TID 542) in 1950 ms on 10.0.0.133 (executor driver) (9/200) +26/04/01 08:43:33 INFO Executor: Running task 16.0 in stage 17.0 (TID 550) +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:33 INFO Executor: Finished task 11.0 in stage 17.0 (TID 545). 6401 bytes result sent to driver +26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:33 INFO TaskSetManager: Starting task 17.0 in stage 17.0 (TID 551) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:33 INFO Executor: Running task 17.0 in stage 17.0 (TID 551) +26/04/01 08:43:33 INFO TaskSetManager: Finished task 11.0 in stage 17.0 (TID 545) in 1880 ms on 10.0.0.133 (executor driver) (10/200) +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:33 INFO Executor: Finished task 10.0 in stage 17.0 (TID 544). 6401 bytes result sent to driver +26/04/01 08:43:33 INFO TaskSetManager: Starting task 18.0 in stage 17.0 (TID 552) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:33 INFO Executor: Running task 18.0 in stage 17.0 (TID 552) +26/04/01 08:43:33 INFO TaskSetManager: Finished task 10.0 in stage 17.0 (TID 544) in 1962 ms on 10.0.0.133 (executor driver) (11/200) +26/04/01 08:43:33 INFO Executor: Finished task 9.0 in stage 17.0 (TID 543). 6401 bytes result sent to driver +26/04/01 08:43:33 INFO TaskSetManager: Starting task 19.0 in stage 17.0 (TID 553) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:33 INFO TaskSetManager: Finished task 9.0 in stage 17.0 (TID 543) in 1964 ms on 10.0.0.133 (executor driver) (12/200) +26/04/01 08:43:33 INFO Executor: Running task 19.0 in stage 17.0 (TID 553) +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:34 INFO Executor: Finished task 12.0 in stage 17.0 (TID 546). 6401 bytes result sent to driver +26/04/01 08:43:34 INFO TaskSetManager: Starting task 20.0 in stage 17.0 (TID 554) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:34 INFO Executor: Running task 20.0 in stage 17.0 (TID 554) +26/04/01 08:43:34 INFO TaskSetManager: Finished task 12.0 in stage 17.0 (TID 546) in 1862 ms on 10.0.0.133 (executor driver) (13/200) +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:34 INFO Executor: Finished task 13.0 in stage 17.0 (TID 547). 6401 bytes result sent to driver +26/04/01 08:43:34 INFO TaskSetManager: Starting task 21.0 in stage 17.0 (TID 555) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:34 INFO Executor: Running task 21.0 in stage 17.0 (TID 555) +26/04/01 08:43:34 INFO TaskSetManager: Finished task 13.0 in stage 17.0 (TID 547) in 1877 ms on 10.0.0.133 (executor driver) (14/200) +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:34 INFO Executor: Finished task 14.0 in stage 17.0 (TID 548). 6401 bytes result sent to driver +26/04/01 08:43:34 INFO TaskSetManager: Starting task 22.0 in stage 17.0 (TID 556) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:34 INFO TaskSetManager: Finished task 14.0 in stage 17.0 (TID 548) in 1880 ms on 10.0.0.133 (executor driver) (15/200) +26/04/01 08:43:34 INFO Executor: Running task 22.0 in stage 17.0 (TID 556) +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (76.4 MiB) non-empty blocks including 208 (76.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (76.4 MiB) non-empty blocks including 208 (76.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:34 INFO Executor: Finished task 15.0 in stage 17.0 (TID 549). 6401 bytes result sent to driver +26/04/01 08:43:34 INFO TaskSetManager: Starting task 23.0 in stage 17.0 (TID 557) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:34 INFO TaskSetManager: Finished task 15.0 in stage 17.0 (TID 549) in 1883 ms on 10.0.0.133 (executor driver) (16/200) +26/04/01 08:43:34 INFO Executor: Running task 23.0 in stage 17.0 (TID 557) +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:35 INFO Executor: Finished task 16.0 in stage 17.0 (TID 550). 6401 bytes result sent to driver +26/04/01 08:43:35 INFO TaskSetManager: Starting task 24.0 in stage 17.0 (TID 558) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:35 INFO TaskSetManager: Finished task 16.0 in stage 17.0 (TID 550) in 1908 ms on 10.0.0.133 (executor driver) (17/200) +26/04/01 08:43:35 INFO Executor: Running task 24.0 in stage 17.0 (TID 558) +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:35 INFO Executor: Finished task 17.0 in stage 17.0 (TID 551). 6401 bytes result sent to driver +26/04/01 08:43:35 INFO TaskSetManager: Starting task 25.0 in stage 17.0 (TID 559) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:35 INFO TaskSetManager: Finished task 17.0 in stage 17.0 (TID 551) in 1910 ms on 10.0.0.133 (executor driver) (18/200) +26/04/01 08:43:35 INFO Executor: Running task 25.0 in stage 17.0 (TID 559) +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:35 INFO Executor: Finished task 18.0 in stage 17.0 (TID 552). 6401 bytes result sent to driver +26/04/01 08:43:35 INFO TaskSetManager: Starting task 26.0 in stage 17.0 (TID 560) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:35 INFO TaskSetManager: Finished task 18.0 in stage 17.0 (TID 552) in 1913 ms on 10.0.0.133 (executor driver) (19/200) +26/04/01 08:43:35 INFO Executor: Running task 26.0 in stage 17.0 (TID 560) +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:35 INFO Executor: Finished task 19.0 in stage 17.0 (TID 553). 6401 bytes result sent to driver +26/04/01 08:43:35 INFO TaskSetManager: Starting task 27.0 in stage 17.0 (TID 561) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:35 INFO TaskSetManager: Finished task 19.0 in stage 17.0 (TID 553) in 1931 ms on 10.0.0.133 (executor driver) (20/200) +26/04/01 08:43:35 INFO Executor: Running task 27.0 in stage 17.0 (TID 561) +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:36 INFO Executor: Finished task 20.0 in stage 17.0 (TID 554). 6401 bytes result sent to driver +26/04/01 08:43:36 INFO TaskSetManager: Starting task 28.0 in stage 17.0 (TID 562) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:36 INFO Executor: Running task 28.0 in stage 17.0 (TID 562) +26/04/01 08:43:36 INFO TaskSetManager: Finished task 20.0 in stage 17.0 (TID 554) in 1895 ms on 10.0.0.133 (executor driver) (21/200) +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:36 INFO Executor: Finished task 21.0 in stage 17.0 (TID 555). 6401 bytes result sent to driver +26/04/01 08:43:36 INFO TaskSetManager: Starting task 29.0 in stage 17.0 (TID 563) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:36 INFO Executor: Running task 29.0 in stage 17.0 (TID 563) +26/04/01 08:43:36 INFO TaskSetManager: Finished task 21.0 in stage 17.0 (TID 555) in 1891 ms on 10.0.0.133 (executor driver) (22/200) +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:36 INFO Executor: Finished task 22.0 in stage 17.0 (TID 556). 6401 bytes result sent to driver +26/04/01 08:43:36 INFO TaskSetManager: Starting task 30.0 in stage 17.0 (TID 564) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:36 INFO Executor: Running task 30.0 in stage 17.0 (TID 564) +26/04/01 08:43:36 INFO TaskSetManager: Finished task 22.0 in stage 17.0 (TID 556) in 1888 ms on 10.0.0.133 (executor driver) (23/200) +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:36 INFO Executor: Finished task 23.0 in stage 17.0 (TID 557). 6401 bytes result sent to driver +26/04/01 08:43:36 INFO TaskSetManager: Starting task 31.0 in stage 17.0 (TID 565) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:36 INFO TaskSetManager: Finished task 23.0 in stage 17.0 (TID 557) in 1879 ms on 10.0.0.133 (executor driver) (24/200) +26/04/01 08:43:36 INFO Executor: Running task 31.0 in stage 17.0 (TID 565) +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:37 INFO Executor: Finished task 26.0 in stage 17.0 (TID 560). 6401 bytes result sent to driver +26/04/01 08:43:37 INFO TaskSetManager: Starting task 32.0 in stage 17.0 (TID 566) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:37 INFO TaskSetManager: Finished task 26.0 in stage 17.0 (TID 560) in 1869 ms on 10.0.0.133 (executor driver) (25/200) +26/04/01 08:43:37 INFO Executor: Running task 32.0 in stage 17.0 (TID 566) +26/04/01 08:43:37 INFO Executor: Finished task 25.0 in stage 17.0 (TID 559). 6401 bytes result sent to driver +26/04/01 08:43:37 INFO TaskSetManager: Starting task 33.0 in stage 17.0 (TID 567) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:37 INFO TaskSetManager: Finished task 25.0 in stage 17.0 (TID 559) in 1875 ms on 10.0.0.133 (executor driver) (26/200) +26/04/01 08:43:37 INFO Executor: Running task 33.0 in stage 17.0 (TID 567) +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:37 INFO Executor: Finished task 24.0 in stage 17.0 (TID 558). 6401 bytes result sent to driver +26/04/01 08:43:37 INFO TaskSetManager: Starting task 34.0 in stage 17.0 (TID 568) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:37 INFO TaskSetManager: Finished task 24.0 in stage 17.0 (TID 558) in 1890 ms on 10.0.0.133 (executor driver) (27/200) +26/04/01 08:43:37 INFO Executor: Running task 34.0 in stage 17.0 (TID 568) +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:37 INFO Executor: Finished task 27.0 in stage 17.0 (TID 561). 6401 bytes result sent to driver +26/04/01 08:43:37 INFO TaskSetManager: Starting task 35.0 in stage 17.0 (TID 569) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:37 INFO Executor: Running task 35.0 in stage 17.0 (TID 569) +26/04/01 08:43:37 INFO TaskSetManager: Finished task 27.0 in stage 17.0 (TID 561) in 1862 ms on 10.0.0.133 (executor driver) (28/200) +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:37 INFO BlockManagerInfo: Removed broadcast_19_piece0 on 10.0.0.133:59303 in memory (size: 7.9 KiB, free: 8.6 GiB) +26/04/01 08:43:37 INFO BlockManagerInfo: Removed broadcast_21_piece0 on 10.0.0.133:59303 in memory (size: 7.9 KiB, free: 8.6 GiB) +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:38 INFO Executor: Finished task 28.0 in stage 17.0 (TID 562). 6444 bytes result sent to driver +26/04/01 08:43:38 INFO TaskSetManager: Starting task 36.0 in stage 17.0 (TID 570) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:38 INFO Executor: Running task 36.0 in stage 17.0 (TID 570) +26/04/01 08:43:38 INFO TaskSetManager: Finished task 28.0 in stage 17.0 (TID 562) in 1888 ms on 10.0.0.133 (executor driver) (29/200) +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:38 INFO Executor: Finished task 29.0 in stage 17.0 (TID 563). 6444 bytes result sent to driver +26/04/01 08:43:38 INFO TaskSetManager: Starting task 37.0 in stage 17.0 (TID 571) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:38 INFO Executor: Running task 37.0 in stage 17.0 (TID 571) +26/04/01 08:43:38 INFO TaskSetManager: Finished task 29.0 in stage 17.0 (TID 563) in 1878 ms on 10.0.0.133 (executor driver) (30/200) +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:38 INFO Executor: Finished task 30.0 in stage 17.0 (TID 564). 6444 bytes result sent to driver +26/04/01 08:43:38 INFO TaskSetManager: Starting task 38.0 in stage 17.0 (TID 572) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:38 INFO TaskSetManager: Finished task 30.0 in stage 17.0 (TID 564) in 1881 ms on 10.0.0.133 (executor driver) (31/200) +26/04/01 08:43:38 INFO Executor: Running task 38.0 in stage 17.0 (TID 572) +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:38 INFO Executor: Finished task 31.0 in stage 17.0 (TID 565). 6444 bytes result sent to driver +26/04/01 08:43:38 INFO TaskSetManager: Starting task 39.0 in stage 17.0 (TID 573) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:38 INFO TaskSetManager: Finished task 31.0 in stage 17.0 (TID 565) in 1897 ms on 10.0.0.133 (executor driver) (32/200) +26/04/01 08:43:38 INFO Executor: Running task 39.0 in stage 17.0 (TID 573) +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:39 INFO Executor: Finished task 33.0 in stage 17.0 (TID 567). 7175 bytes result sent to driver +26/04/01 08:43:39 INFO Executor: Finished task 32.0 in stage 17.0 (TID 566). 7175 bytes result sent to driver +26/04/01 08:43:39 INFO TaskSetManager: Starting task 40.0 in stage 17.0 (TID 574) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:39 INFO Executor: Running task 40.0 in stage 17.0 (TID 574) +26/04/01 08:43:39 INFO TaskSetManager: Starting task 41.0 in stage 17.0 (TID 575) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:39 INFO TaskSetManager: Finished task 33.0 in stage 17.0 (TID 567) in 1890 ms on 10.0.0.133 (executor driver) (33/200) +26/04/01 08:43:39 INFO Executor: Running task 41.0 in stage 17.0 (TID 575) +26/04/01 08:43:39 INFO TaskSetManager: Finished task 32.0 in stage 17.0 (TID 566) in 1891 ms on 10.0.0.133 (executor driver) (34/200) +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:39 INFO Executor: Finished task 34.0 in stage 17.0 (TID 568). 7175 bytes result sent to driver +26/04/01 08:43:39 INFO TaskSetManager: Starting task 42.0 in stage 17.0 (TID 576) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:39 INFO TaskSetManager: Finished task 34.0 in stage 17.0 (TID 568) in 1893 ms on 10.0.0.133 (executor driver) (35/200) +26/04/01 08:43:39 INFO Executor: Running task 42.0 in stage 17.0 (TID 576) +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:39 INFO Executor: Finished task 35.0 in stage 17.0 (TID 569). 7132 bytes result sent to driver +26/04/01 08:43:39 INFO TaskSetManager: Starting task 43.0 in stage 17.0 (TID 577) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:39 INFO TaskSetManager: Finished task 35.0 in stage 17.0 (TID 569) in 1898 ms on 10.0.0.133 (executor driver) (36/200) +26/04/01 08:43:39 INFO Executor: Running task 43.0 in stage 17.0 (TID 577) +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:40 INFO Executor: Finished task 36.0 in stage 17.0 (TID 570). 7132 bytes result sent to driver +26/04/01 08:43:40 INFO TaskSetManager: Starting task 44.0 in stage 17.0 (TID 578) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:40 INFO TaskSetManager: Finished task 36.0 in stage 17.0 (TID 570) in 1884 ms on 10.0.0.133 (executor driver) (37/200) +26/04/01 08:43:40 INFO Executor: Running task 44.0 in stage 17.0 (TID 578) +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:40 INFO Executor: Finished task 37.0 in stage 17.0 (TID 571). 7132 bytes result sent to driver +26/04/01 08:43:40 INFO TaskSetManager: Starting task 45.0 in stage 17.0 (TID 579) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:40 INFO Executor: Running task 45.0 in stage 17.0 (TID 579) +26/04/01 08:43:40 INFO TaskSetManager: Finished task 37.0 in stage 17.0 (TID 571) in 1866 ms on 10.0.0.133 (executor driver) (38/200) +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:40 INFO Executor: Finished task 38.0 in stage 17.0 (TID 572). 7132 bytes result sent to driver +26/04/01 08:43:40 INFO TaskSetManager: Starting task 46.0 in stage 17.0 (TID 580) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:40 INFO TaskSetManager: Finished task 38.0 in stage 17.0 (TID 572) in 1862 ms on 10.0.0.133 (executor driver) (39/200) +26/04/01 08:43:40 INFO Executor: Running task 46.0 in stage 17.0 (TID 580) +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:40 INFO Executor: Finished task 39.0 in stage 17.0 (TID 573). 7132 bytes result sent to driver +26/04/01 08:43:40 INFO TaskSetManager: Starting task 47.0 in stage 17.0 (TID 581) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:40 INFO TaskSetManager: Finished task 39.0 in stage 17.0 (TID 573) in 1869 ms on 10.0.0.133 (executor driver) (40/200) +26/04/01 08:43:40 INFO Executor: Running task 47.0 in stage 17.0 (TID 581) +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:41 INFO Executor: Finished task 43.0 in stage 17.0 (TID 577). 6401 bytes result sent to driver +26/04/01 08:43:41 INFO TaskSetManager: Starting task 48.0 in stage 17.0 (TID 582) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:41 INFO Executor: Finished task 41.0 in stage 17.0 (TID 575). 6401 bytes result sent to driver +26/04/01 08:43:41 INFO Executor: Running task 48.0 in stage 17.0 (TID 582) +26/04/01 08:43:41 INFO TaskSetManager: Finished task 43.0 in stage 17.0 (TID 577) in 1871 ms on 10.0.0.133 (executor driver) (41/200) +26/04/01 08:43:41 INFO Executor: Finished task 40.0 in stage 17.0 (TID 574). 6401 bytes result sent to driver +26/04/01 08:43:41 INFO TaskSetManager: Starting task 49.0 in stage 17.0 (TID 583) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:41 INFO Executor: Running task 49.0 in stage 17.0 (TID 583) +26/04/01 08:43:41 INFO TaskSetManager: Starting task 50.0 in stage 17.0 (TID 584) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:41 INFO TaskSetManager: Finished task 41.0 in stage 17.0 (TID 575) in 1890 ms on 10.0.0.133 (executor driver) (42/200) +26/04/01 08:43:41 INFO Executor: Running task 50.0 in stage 17.0 (TID 584) +26/04/01 08:43:41 INFO TaskSetManager: Finished task 40.0 in stage 17.0 (TID 574) in 1890 ms on 10.0.0.133 (executor driver) (43/200) +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:41 INFO Executor: Finished task 42.0 in stage 17.0 (TID 576). 6401 bytes result sent to driver +26/04/01 08:43:41 INFO TaskSetManager: Starting task 51.0 in stage 17.0 (TID 585) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:41 INFO TaskSetManager: Finished task 42.0 in stage 17.0 (TID 576) in 1886 ms on 10.0.0.133 (executor driver) (44/200) +26/04/01 08:43:41 INFO Executor: Running task 51.0 in stage 17.0 (TID 585) +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:41 INFO Executor: Finished task 44.0 in stage 17.0 (TID 578). 6401 bytes result sent to driver +26/04/01 08:43:41 INFO TaskSetManager: Starting task 52.0 in stage 17.0 (TID 586) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:41 INFO Executor: Running task 52.0 in stage 17.0 (TID 586) +26/04/01 08:43:41 INFO TaskSetManager: Finished task 44.0 in stage 17.0 (TID 578) in 1868 ms on 10.0.0.133 (executor driver) (45/200) +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:42 INFO Executor: Finished task 45.0 in stage 17.0 (TID 579). 6401 bytes result sent to driver +26/04/01 08:43:42 INFO TaskSetManager: Starting task 53.0 in stage 17.0 (TID 587) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:42 INFO TaskSetManager: Finished task 45.0 in stage 17.0 (TID 579) in 1865 ms on 10.0.0.133 (executor driver) (46/200) +26/04/01 08:43:42 INFO Executor: Running task 53.0 in stage 17.0 (TID 587) +26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:42 INFO Executor: Finished task 46.0 in stage 17.0 (TID 580). 6401 bytes result sent to driver +26/04/01 08:43:42 INFO TaskSetManager: Starting task 54.0 in stage 17.0 (TID 588) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:42 INFO TaskSetManager: Finished task 46.0 in stage 17.0 (TID 580) in 1869 ms on 10.0.0.133 (executor driver) (47/200) +26/04/01 08:43:42 INFO Executor: Running task 54.0 in stage 17.0 (TID 588) +26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:42 INFO Executor: Finished task 47.0 in stage 17.0 (TID 581). 6401 bytes result sent to driver +26/04/01 08:43:42 INFO TaskSetManager: Starting task 55.0 in stage 17.0 (TID 589) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:42 INFO TaskSetManager: Finished task 47.0 in stage 17.0 (TID 581) in 1889 ms on 10.0.0.133 (executor driver) (48/200) +26/04/01 08:43:42 INFO Executor: Running task 55.0 in stage 17.0 (TID 589) +26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:43 INFO Executor: Finished task 48.0 in stage 17.0 (TID 582). 6401 bytes result sent to driver +26/04/01 08:43:43 INFO TaskSetManager: Starting task 56.0 in stage 17.0 (TID 590) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:43 INFO Executor: Finished task 49.0 in stage 17.0 (TID 583). 6401 bytes result sent to driver +26/04/01 08:43:43 INFO Executor: Running task 56.0 in stage 17.0 (TID 590) +26/04/01 08:43:43 INFO TaskSetManager: Starting task 57.0 in stage 17.0 (TID 591) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:43 INFO Executor: Running task 57.0 in stage 17.0 (TID 591) +26/04/01 08:43:43 INFO TaskSetManager: Finished task 48.0 in stage 17.0 (TID 582) in 1890 ms on 10.0.0.133 (executor driver) (49/200) +26/04/01 08:43:43 INFO TaskSetManager: Finished task 49.0 in stage 17.0 (TID 583) in 1891 ms on 10.0.0.133 (executor driver) (50/200) +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:43 INFO Executor: Finished task 50.0 in stage 17.0 (TID 584). 6401 bytes result sent to driver +26/04/01 08:43:43 INFO TaskSetManager: Starting task 58.0 in stage 17.0 (TID 592) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:43 INFO Executor: Running task 58.0 in stage 17.0 (TID 592) +26/04/01 08:43:43 INFO TaskSetManager: Finished task 50.0 in stage 17.0 (TID 584) in 1897 ms on 10.0.0.133 (executor driver) (51/200) +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:43 INFO Executor: Finished task 51.0 in stage 17.0 (TID 585). 6401 bytes result sent to driver +26/04/01 08:43:43 INFO TaskSetManager: Starting task 59.0 in stage 17.0 (TID 593) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:43 INFO TaskSetManager: Finished task 51.0 in stage 17.0 (TID 585) in 1917 ms on 10.0.0.133 (executor driver) (52/200) +26/04/01 08:43:43 INFO Executor: Running task 59.0 in stage 17.0 (TID 593) +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:43 INFO Executor: Finished task 52.0 in stage 17.0 (TID 586). 6401 bytes result sent to driver +26/04/01 08:43:43 INFO TaskSetManager: Starting task 60.0 in stage 17.0 (TID 594) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:43 INFO TaskSetManager: Finished task 52.0 in stage 17.0 (TID 586) in 1908 ms on 10.0.0.133 (executor driver) (53/200) +26/04/01 08:43:43 INFO Executor: Running task 60.0 in stage 17.0 (TID 594) +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:43 INFO Executor: Finished task 53.0 in stage 17.0 (TID 587). 6401 bytes result sent to driver +26/04/01 08:43:43 INFO TaskSetManager: Starting task 61.0 in stage 17.0 (TID 595) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:43 INFO Executor: Running task 61.0 in stage 17.0 (TID 595) +26/04/01 08:43:43 INFO TaskSetManager: Finished task 53.0 in stage 17.0 (TID 587) in 1899 ms on 10.0.0.133 (executor driver) (54/200) +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:43 INFO Executor: Finished task 54.0 in stage 17.0 (TID 588). 6401 bytes result sent to driver +26/04/01 08:43:43 INFO TaskSetManager: Starting task 62.0 in stage 17.0 (TID 596) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:43 INFO TaskSetManager: Finished task 54.0 in stage 17.0 (TID 588) in 1908 ms on 10.0.0.133 (executor driver) (55/200) +26/04/01 08:43:43 INFO Executor: Running task 62.0 in stage 17.0 (TID 596) +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:44 INFO Executor: Finished task 55.0 in stage 17.0 (TID 589). 6401 bytes result sent to driver +26/04/01 08:43:44 INFO TaskSetManager: Starting task 63.0 in stage 17.0 (TID 597) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:44 INFO Executor: Running task 63.0 in stage 17.0 (TID 597) +26/04/01 08:43:44 INFO TaskSetManager: Finished task 55.0 in stage 17.0 (TID 589) in 1861 ms on 10.0.0.133 (executor driver) (56/200) +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:44 INFO Executor: Finished task 57.0 in stage 17.0 (TID 591). 6401 bytes result sent to driver +26/04/01 08:43:44 INFO TaskSetManager: Starting task 64.0 in stage 17.0 (TID 598) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:44 INFO Executor: Running task 64.0 in stage 17.0 (TID 598) +26/04/01 08:43:44 INFO TaskSetManager: Finished task 57.0 in stage 17.0 (TID 591) in 1870 ms on 10.0.0.133 (executor driver) (57/200) +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:44 INFO Executor: Finished task 56.0 in stage 17.0 (TID 590). 6401 bytes result sent to driver +26/04/01 08:43:44 INFO TaskSetManager: Starting task 65.0 in stage 17.0 (TID 599) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:44 INFO Executor: Running task 65.0 in stage 17.0 (TID 599) +26/04/01 08:43:44 INFO TaskSetManager: Finished task 56.0 in stage 17.0 (TID 590) in 1875 ms on 10.0.0.133 (executor driver) (58/200) +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:44 INFO Executor: Finished task 58.0 in stage 17.0 (TID 592). 6401 bytes result sent to driver +26/04/01 08:43:44 INFO TaskSetManager: Starting task 66.0 in stage 17.0 (TID 600) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:44 INFO Executor: Running task 66.0 in stage 17.0 (TID 600) +26/04/01 08:43:44 INFO TaskSetManager: Finished task 58.0 in stage 17.0 (TID 592) in 1874 ms on 10.0.0.133 (executor driver) (59/200) +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:44 INFO Executor: Finished task 59.0 in stage 17.0 (TID 593). 6401 bytes result sent to driver +26/04/01 08:43:44 INFO TaskSetManager: Starting task 67.0 in stage 17.0 (TID 601) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:44 INFO Executor: Running task 67.0 in stage 17.0 (TID 601) +26/04/01 08:43:44 INFO TaskSetManager: Finished task 59.0 in stage 17.0 (TID 593) in 1866 ms on 10.0.0.133 (executor driver) (60/200) +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:45 INFO Executor: Finished task 60.0 in stage 17.0 (TID 594). 6401 bytes result sent to driver +26/04/01 08:43:45 INFO TaskSetManager: Starting task 68.0 in stage 17.0 (TID 602) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:45 INFO Executor: Running task 68.0 in stage 17.0 (TID 602) +26/04/01 08:43:45 INFO TaskSetManager: Finished task 60.0 in stage 17.0 (TID 594) in 1870 ms on 10.0.0.133 (executor driver) (61/200) +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:45 INFO Executor: Finished task 61.0 in stage 17.0 (TID 595). 6401 bytes result sent to driver +26/04/01 08:43:45 INFO TaskSetManager: Starting task 69.0 in stage 17.0 (TID 603) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:45 INFO TaskSetManager: Finished task 61.0 in stage 17.0 (TID 595) in 1867 ms on 10.0.0.133 (executor driver) (62/200) +26/04/01 08:43:45 INFO Executor: Running task 69.0 in stage 17.0 (TID 603) +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:45 INFO Executor: Finished task 62.0 in stage 17.0 (TID 596). 6401 bytes result sent to driver +26/04/01 08:43:45 INFO TaskSetManager: Starting task 70.0 in stage 17.0 (TID 604) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:45 INFO Executor: Running task 70.0 in stage 17.0 (TID 604) +26/04/01 08:43:45 INFO TaskSetManager: Finished task 62.0 in stage 17.0 (TID 596) in 1862 ms on 10.0.0.133 (executor driver) (63/200) +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:45 INFO Executor: Finished task 63.0 in stage 17.0 (TID 597). 6401 bytes result sent to driver +26/04/01 08:43:45 INFO TaskSetManager: Starting task 71.0 in stage 17.0 (TID 605) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:45 INFO Executor: Running task 71.0 in stage 17.0 (TID 605) +26/04/01 08:43:45 INFO TaskSetManager: Finished task 63.0 in stage 17.0 (TID 597) in 1862 ms on 10.0.0.133 (executor driver) (64/200) +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:46 INFO Executor: Finished task 64.0 in stage 17.0 (TID 598). 6401 bytes result sent to driver +26/04/01 08:43:46 INFO TaskSetManager: Starting task 72.0 in stage 17.0 (TID 606) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:46 INFO TaskSetManager: Finished task 64.0 in stage 17.0 (TID 598) in 1841 ms on 10.0.0.133 (executor driver) (65/200) +26/04/01 08:43:46 INFO Executor: Running task 72.0 in stage 17.0 (TID 606) +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:46 INFO Executor: Finished task 65.0 in stage 17.0 (TID 599). 6401 bytes result sent to driver +26/04/01 08:43:46 INFO TaskSetManager: Starting task 73.0 in stage 17.0 (TID 607) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:46 INFO TaskSetManager: Finished task 65.0 in stage 17.0 (TID 599) in 1846 ms on 10.0.0.133 (executor driver) (66/200) +26/04/01 08:43:46 INFO Executor: Running task 73.0 in stage 17.0 (TID 607) +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:46 INFO Executor: Finished task 66.0 in stage 17.0 (TID 600). 6401 bytes result sent to driver +26/04/01 08:43:46 INFO TaskSetManager: Starting task 74.0 in stage 17.0 (TID 608) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:46 INFO TaskSetManager: Finished task 66.0 in stage 17.0 (TID 600) in 1849 ms on 10.0.0.133 (executor driver) (67/200) +26/04/01 08:43:46 INFO Executor: Running task 74.0 in stage 17.0 (TID 608) +26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:46 INFO Executor: Finished task 67.0 in stage 17.0 (TID 601). 6401 bytes result sent to driver +26/04/01 08:43:46 INFO TaskSetManager: Starting task 75.0 in stage 17.0 (TID 609) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:46 INFO TaskSetManager: Finished task 67.0 in stage 17.0 (TID 601) in 1855 ms on 10.0.0.133 (executor driver) (68/200) +26/04/01 08:43:46 INFO Executor: Running task 75.0 in stage 17.0 (TID 609) +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:47 INFO Executor: Finished task 68.0 in stage 17.0 (TID 602). 6401 bytes result sent to driver +26/04/01 08:43:47 INFO TaskSetManager: Starting task 76.0 in stage 17.0 (TID 610) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:47 INFO TaskSetManager: Finished task 68.0 in stage 17.0 (TID 602) in 1884 ms on 10.0.0.133 (executor driver) (69/200) +26/04/01 08:43:47 INFO Executor: Running task 76.0 in stage 17.0 (TID 610) +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:47 INFO Executor: Finished task 69.0 in stage 17.0 (TID 603). 6401 bytes result sent to driver +26/04/01 08:43:47 INFO TaskSetManager: Starting task 77.0 in stage 17.0 (TID 611) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:47 INFO TaskSetManager: Finished task 69.0 in stage 17.0 (TID 603) in 1881 ms on 10.0.0.133 (executor driver) (70/200) +26/04/01 08:43:47 INFO Executor: Running task 77.0 in stage 17.0 (TID 611) +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:47 INFO Executor: Finished task 70.0 in stage 17.0 (TID 604). 6401 bytes result sent to driver +26/04/01 08:43:47 INFO TaskSetManager: Starting task 78.0 in stage 17.0 (TID 612) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:47 INFO Executor: Running task 78.0 in stage 17.0 (TID 612) +26/04/01 08:43:47 INFO TaskSetManager: Finished task 70.0 in stage 17.0 (TID 604) in 1891 ms on 10.0.0.133 (executor driver) (71/200) +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:47 INFO Executor: Finished task 71.0 in stage 17.0 (TID 605). 6401 bytes result sent to driver +26/04/01 08:43:47 INFO TaskSetManager: Starting task 79.0 in stage 17.0 (TID 613) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:47 INFO TaskSetManager: Finished task 71.0 in stage 17.0 (TID 605) in 1889 ms on 10.0.0.133 (executor driver) (72/200) +26/04/01 08:43:47 INFO Executor: Running task 79.0 in stage 17.0 (TID 613) +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:48 INFO Executor: Finished task 72.0 in stage 17.0 (TID 606). 7132 bytes result sent to driver +26/04/01 08:43:48 INFO TaskSetManager: Starting task 80.0 in stage 17.0 (TID 614) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:48 INFO Executor: Running task 80.0 in stage 17.0 (TID 614) +26/04/01 08:43:48 INFO TaskSetManager: Finished task 72.0 in stage 17.0 (TID 606) in 1863 ms on 10.0.0.133 (executor driver) (73/200) +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:48 INFO Executor: Finished task 74.0 in stage 17.0 (TID 608). 7132 bytes result sent to driver +26/04/01 08:43:48 INFO TaskSetManager: Starting task 81.0 in stage 17.0 (TID 615) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:48 INFO TaskSetManager: Finished task 74.0 in stage 17.0 (TID 608) in 1849 ms on 10.0.0.133 (executor driver) (74/200) +26/04/01 08:43:48 INFO Executor: Running task 81.0 in stage 17.0 (TID 615) +26/04/01 08:43:48 INFO Executor: Finished task 73.0 in stage 17.0 (TID 607). 7132 bytes result sent to driver +26/04/01 08:43:48 INFO TaskSetManager: Starting task 82.0 in stage 17.0 (TID 616) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:48 INFO Executor: Running task 82.0 in stage 17.0 (TID 616) +26/04/01 08:43:48 INFO TaskSetManager: Finished task 73.0 in stage 17.0 (TID 607) in 1860 ms on 10.0.0.133 (executor driver) (75/200) +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:48 INFO Executor: Finished task 75.0 in stage 17.0 (TID 609). 7132 bytes result sent to driver +26/04/01 08:43:48 INFO TaskSetManager: Starting task 83.0 in stage 17.0 (TID 617) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:48 INFO Executor: Running task 83.0 in stage 17.0 (TID 617) +26/04/01 08:43:48 INFO TaskSetManager: Finished task 75.0 in stage 17.0 (TID 609) in 1851 ms on 10.0.0.133 (executor driver) (76/200) +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:49 INFO Executor: Finished task 76.0 in stage 17.0 (TID 610). 7132 bytes result sent to driver +26/04/01 08:43:49 INFO TaskSetManager: Starting task 84.0 in stage 17.0 (TID 618) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:49 INFO TaskSetManager: Finished task 76.0 in stage 17.0 (TID 610) in 1849 ms on 10.0.0.133 (executor driver) (77/200) +26/04/01 08:43:49 INFO Executor: Running task 84.0 in stage 17.0 (TID 618) +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:49 INFO Executor: Finished task 78.0 in stage 17.0 (TID 612). 7132 bytes result sent to driver +26/04/01 08:43:49 INFO TaskSetManager: Starting task 85.0 in stage 17.0 (TID 619) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:49 INFO Executor: Running task 85.0 in stage 17.0 (TID 619) +26/04/01 08:43:49 INFO TaskSetManager: Finished task 78.0 in stage 17.0 (TID 612) in 1843 ms on 10.0.0.133 (executor driver) (78/200) +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:49 INFO Executor: Finished task 77.0 in stage 17.0 (TID 611). 7132 bytes result sent to driver +26/04/01 08:43:49 INFO TaskSetManager: Starting task 86.0 in stage 17.0 (TID 620) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:49 INFO TaskSetManager: Finished task 77.0 in stage 17.0 (TID 611) in 1865 ms on 10.0.0.133 (executor driver) (79/200) +26/04/01 08:43:49 INFO Executor: Running task 86.0 in stage 17.0 (TID 620) +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:49 INFO Executor: Finished task 79.0 in stage 17.0 (TID 613). 7132 bytes result sent to driver +26/04/01 08:43:49 INFO TaskSetManager: Starting task 87.0 in stage 17.0 (TID 621) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:49 INFO TaskSetManager: Finished task 79.0 in stage 17.0 (TID 613) in 1847 ms on 10.0.0.133 (executor driver) (80/200) +26/04/01 08:43:49 INFO Executor: Running task 87.0 in stage 17.0 (TID 621) +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:50 INFO Executor: Finished task 81.0 in stage 17.0 (TID 615). 6401 bytes result sent to driver +26/04/01 08:43:50 INFO TaskSetManager: Starting task 88.0 in stage 17.0 (TID 622) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:50 INFO TaskSetManager: Finished task 81.0 in stage 17.0 (TID 615) in 1838 ms on 10.0.0.133 (executor driver) (81/200) +26/04/01 08:43:50 INFO Executor: Running task 88.0 in stage 17.0 (TID 622) +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:50 INFO Executor: Finished task 80.0 in stage 17.0 (TID 614). 6401 bytes result sent to driver +26/04/01 08:43:50 INFO TaskSetManager: Starting task 89.0 in stage 17.0 (TID 623) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:50 INFO TaskSetManager: Finished task 80.0 in stage 17.0 (TID 614) in 1854 ms on 10.0.0.133 (executor driver) (82/200) +26/04/01 08:43:50 INFO Executor: Running task 89.0 in stage 17.0 (TID 623) +26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:50 INFO Executor: Finished task 82.0 in stage 17.0 (TID 616). 6401 bytes result sent to driver +26/04/01 08:43:50 INFO TaskSetManager: Starting task 90.0 in stage 17.0 (TID 624) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:50 INFO TaskSetManager: Finished task 82.0 in stage 17.0 (TID 616) in 1865 ms on 10.0.0.133 (executor driver) (83/200) +26/04/01 08:43:50 INFO Executor: Running task 90.0 in stage 17.0 (TID 624) +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:50 INFO Executor: Finished task 83.0 in stage 17.0 (TID 617). 6401 bytes result sent to driver +26/04/01 08:43:50 INFO TaskSetManager: Starting task 91.0 in stage 17.0 (TID 625) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:50 INFO TaskSetManager: Finished task 83.0 in stage 17.0 (TID 617) in 1856 ms on 10.0.0.133 (executor driver) (84/200) +26/04/01 08:43:50 INFO Executor: Running task 91.0 in stage 17.0 (TID 625) +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:51 INFO Executor: Finished task 84.0 in stage 17.0 (TID 618). 6401 bytes result sent to driver +26/04/01 08:43:51 INFO TaskSetManager: Starting task 92.0 in stage 17.0 (TID 626) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:51 INFO Executor: Running task 92.0 in stage 17.0 (TID 626) +26/04/01 08:43:51 INFO TaskSetManager: Finished task 84.0 in stage 17.0 (TID 618) in 1852 ms on 10.0.0.133 (executor driver) (85/200) +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:51 INFO Executor: Finished task 85.0 in stage 17.0 (TID 619). 6401 bytes result sent to driver +26/04/01 08:43:51 INFO TaskSetManager: Starting task 93.0 in stage 17.0 (TID 627) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:51 INFO Executor: Running task 93.0 in stage 17.0 (TID 627) +26/04/01 08:43:51 INFO TaskSetManager: Finished task 85.0 in stage 17.0 (TID 619) in 1864 ms on 10.0.0.133 (executor driver) (86/200) +26/04/01 08:43:51 INFO Executor: Finished task 86.0 in stage 17.0 (TID 620). 6401 bytes result sent to driver +26/04/01 08:43:51 INFO TaskSetManager: Starting task 94.0 in stage 17.0 (TID 628) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:51 INFO TaskSetManager: Finished task 86.0 in stage 17.0 (TID 620) in 1863 ms on 10.0.0.133 (executor driver) (87/200) +26/04/01 08:43:51 INFO Executor: Running task 94.0 in stage 17.0 (TID 628) +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:51 INFO Executor: Finished task 87.0 in stage 17.0 (TID 621). 6401 bytes result sent to driver +26/04/01 08:43:51 INFO TaskSetManager: Starting task 95.0 in stage 17.0 (TID 629) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:51 INFO Executor: Running task 95.0 in stage 17.0 (TID 629) +26/04/01 08:43:51 INFO TaskSetManager: Finished task 87.0 in stage 17.0 (TID 621) in 1850 ms on 10.0.0.133 (executor driver) (88/200) +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:52 INFO Executor: Finished task 89.0 in stage 17.0 (TID 623). 6401 bytes result sent to driver +26/04/01 08:43:52 INFO TaskSetManager: Starting task 96.0 in stage 17.0 (TID 630) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:52 INFO TaskSetManager: Finished task 89.0 in stage 17.0 (TID 623) in 1842 ms on 10.0.0.133 (executor driver) (89/200) +26/04/01 08:43:52 INFO Executor: Running task 96.0 in stage 17.0 (TID 630) +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:52 INFO Executor: Finished task 88.0 in stage 17.0 (TID 622). 6401 bytes result sent to driver +26/04/01 08:43:52 INFO TaskSetManager: Starting task 97.0 in stage 17.0 (TID 631) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:52 INFO TaskSetManager: Finished task 88.0 in stage 17.0 (TID 622) in 1860 ms on 10.0.0.133 (executor driver) (90/200) +26/04/01 08:43:52 INFO Executor: Running task 97.0 in stage 17.0 (TID 631) +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:52 INFO Executor: Finished task 90.0 in stage 17.0 (TID 624). 6401 bytes result sent to driver +26/04/01 08:43:52 INFO TaskSetManager: Starting task 98.0 in stage 17.0 (TID 632) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:52 INFO Executor: Running task 98.0 in stage 17.0 (TID 632) +26/04/01 08:43:52 INFO TaskSetManager: Finished task 90.0 in stage 17.0 (TID 624) in 1856 ms on 10.0.0.133 (executor driver) (91/200) +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:52 INFO Executor: Finished task 91.0 in stage 17.0 (TID 625). 6401 bytes result sent to driver +26/04/01 08:43:52 INFO TaskSetManager: Starting task 99.0 in stage 17.0 (TID 633) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:52 INFO Executor: Running task 99.0 in stage 17.0 (TID 633) +26/04/01 08:43:52 INFO TaskSetManager: Finished task 91.0 in stage 17.0 (TID 625) in 1850 ms on 10.0.0.133 (executor driver) (92/200) +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:53 INFO Executor: Finished task 92.0 in stage 17.0 (TID 626). 6401 bytes result sent to driver +26/04/01 08:43:53 INFO TaskSetManager: Starting task 100.0 in stage 17.0 (TID 634) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:53 INFO TaskSetManager: Finished task 92.0 in stage 17.0 (TID 626) in 1866 ms on 10.0.0.133 (executor driver) (93/200) +26/04/01 08:43:53 INFO Executor: Running task 100.0 in stage 17.0 (TID 634) +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:53 INFO Executor: Finished task 93.0 in stage 17.0 (TID 627). 6401 bytes result sent to driver +26/04/01 08:43:53 INFO TaskSetManager: Starting task 101.0 in stage 17.0 (TID 635) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:53 INFO TaskSetManager: Finished task 93.0 in stage 17.0 (TID 627) in 1861 ms on 10.0.0.133 (executor driver) (94/200) +26/04/01 08:43:53 INFO Executor: Running task 101.0 in stage 17.0 (TID 635) +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:53 INFO Executor: Finished task 94.0 in stage 17.0 (TID 628). 6401 bytes result sent to driver +26/04/01 08:43:53 INFO TaskSetManager: Starting task 102.0 in stage 17.0 (TID 636) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:53 INFO TaskSetManager: Finished task 94.0 in stage 17.0 (TID 628) in 1875 ms on 10.0.0.133 (executor driver) (95/200) +26/04/01 08:43:53 INFO Executor: Running task 102.0 in stage 17.0 (TID 636) +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:53 INFO Executor: Finished task 95.0 in stage 17.0 (TID 629). 6401 bytes result sent to driver +26/04/01 08:43:53 INFO TaskSetManager: Starting task 103.0 in stage 17.0 (TID 637) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:53 INFO TaskSetManager: Finished task 95.0 in stage 17.0 (TID 629) in 1860 ms on 10.0.0.133 (executor driver) (96/200) +26/04/01 08:43:53 INFO Executor: Running task 103.0 in stage 17.0 (TID 637) +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:54 INFO Executor: Finished task 96.0 in stage 17.0 (TID 630). 6401 bytes result sent to driver +26/04/01 08:43:54 INFO TaskSetManager: Starting task 104.0 in stage 17.0 (TID 638) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:54 INFO TaskSetManager: Finished task 96.0 in stage 17.0 (TID 630) in 1863 ms on 10.0.0.133 (executor driver) (97/200) +26/04/01 08:43:54 INFO Executor: Running task 104.0 in stage 17.0 (TID 638) +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:54 INFO Executor: Finished task 97.0 in stage 17.0 (TID 631). 6401 bytes result sent to driver +26/04/01 08:43:54 INFO TaskSetManager: Starting task 105.0 in stage 17.0 (TID 639) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:54 INFO TaskSetManager: Finished task 97.0 in stage 17.0 (TID 631) in 1861 ms on 10.0.0.133 (executor driver) (98/200) +26/04/01 08:43:54 INFO Executor: Running task 105.0 in stage 17.0 (TID 639) +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:54 INFO Executor: Finished task 99.0 in stage 17.0 (TID 633). 6401 bytes result sent to driver +26/04/01 08:43:54 INFO TaskSetManager: Starting task 106.0 in stage 17.0 (TID 640) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:54 INFO TaskSetManager: Finished task 99.0 in stage 17.0 (TID 633) in 1845 ms on 10.0.0.133 (executor driver) (99/200) +26/04/01 08:43:54 INFO Executor: Running task 106.0 in stage 17.0 (TID 640) +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:54 INFO Executor: Finished task 98.0 in stage 17.0 (TID 632). 6401 bytes result sent to driver +26/04/01 08:43:54 INFO TaskSetManager: Starting task 107.0 in stage 17.0 (TID 641) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:54 INFO TaskSetManager: Finished task 98.0 in stage 17.0 (TID 632) in 1853 ms on 10.0.0.133 (executor driver) (100/200) +26/04/01 08:43:54 INFO Executor: Running task 107.0 in stage 17.0 (TID 641) +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:54 INFO Executor: Finished task 100.0 in stage 17.0 (TID 634). 6401 bytes result sent to driver +26/04/01 08:43:54 INFO TaskSetManager: Starting task 108.0 in stage 17.0 (TID 642) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:54 INFO TaskSetManager: Finished task 100.0 in stage 17.0 (TID 634) in 1866 ms on 10.0.0.133 (executor driver) (101/200) +26/04/01 08:43:54 INFO Executor: Running task 108.0 in stage 17.0 (TID 642) +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:55 INFO Executor: Finished task 101.0 in stage 17.0 (TID 635). 6401 bytes result sent to driver +26/04/01 08:43:55 INFO TaskSetManager: Starting task 109.0 in stage 17.0 (TID 643) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:55 INFO Executor: Running task 109.0 in stage 17.0 (TID 643) +26/04/01 08:43:55 INFO TaskSetManager: Finished task 101.0 in stage 17.0 (TID 635) in 1859 ms on 10.0.0.133 (executor driver) (102/200) +26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:55 INFO Executor: Finished task 102.0 in stage 17.0 (TID 636). 6401 bytes result sent to driver +26/04/01 08:43:55 INFO TaskSetManager: Starting task 110.0 in stage 17.0 (TID 644) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:55 INFO Executor: Running task 110.0 in stage 17.0 (TID 644) +26/04/01 08:43:55 INFO TaskSetManager: Finished task 102.0 in stage 17.0 (TID 636) in 1858 ms on 10.0.0.133 (executor driver) (103/200) +26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:55 INFO Executor: Finished task 103.0 in stage 17.0 (TID 637). 6401 bytes result sent to driver +26/04/01 08:43:55 INFO TaskSetManager: Starting task 111.0 in stage 17.0 (TID 645) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:55 INFO Executor: Running task 111.0 in stage 17.0 (TID 645) +26/04/01 08:43:55 INFO TaskSetManager: Finished task 103.0 in stage 17.0 (TID 637) in 1850 ms on 10.0.0.133 (executor driver) (104/200) +26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:56 INFO Executor: Finished task 105.0 in stage 17.0 (TID 639). 6401 bytes result sent to driver +26/04/01 08:43:56 INFO TaskSetManager: Starting task 112.0 in stage 17.0 (TID 646) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:56 INFO TaskSetManager: Finished task 105.0 in stage 17.0 (TID 639) in 1855 ms on 10.0.0.133 (executor driver) (105/200) +26/04/01 08:43:56 INFO Executor: Running task 112.0 in stage 17.0 (TID 646) +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:56 INFO Executor: Finished task 104.0 in stage 17.0 (TID 638). 6401 bytes result sent to driver +26/04/01 08:43:56 INFO TaskSetManager: Starting task 113.0 in stage 17.0 (TID 647) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:56 INFO Executor: Running task 113.0 in stage 17.0 (TID 647) +26/04/01 08:43:56 INFO TaskSetManager: Finished task 104.0 in stage 17.0 (TID 638) in 1864 ms on 10.0.0.133 (executor driver) (106/200) +26/04/01 08:43:56 INFO Executor: Finished task 106.0 in stage 17.0 (TID 640). 6401 bytes result sent to driver +26/04/01 08:43:56 INFO TaskSetManager: Starting task 114.0 in stage 17.0 (TID 648) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:56 INFO TaskSetManager: Finished task 106.0 in stage 17.0 (TID 640) in 1844 ms on 10.0.0.133 (executor driver) (107/200) +26/04/01 08:43:56 INFO Executor: Running task 114.0 in stage 17.0 (TID 648) +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:56 INFO Executor: Finished task 107.0 in stage 17.0 (TID 641). 6401 bytes result sent to driver +26/04/01 08:43:56 INFO TaskSetManager: Starting task 115.0 in stage 17.0 (TID 649) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:56 INFO Executor: Running task 115.0 in stage 17.0 (TID 649) +26/04/01 08:43:56 INFO TaskSetManager: Finished task 107.0 in stage 17.0 (TID 641) in 1862 ms on 10.0.0.133 (executor driver) (108/200) +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:56 INFO Executor: Finished task 108.0 in stage 17.0 (TID 642). 6401 bytes result sent to driver +26/04/01 08:43:56 INFO TaskSetManager: Starting task 116.0 in stage 17.0 (TID 650) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:56 INFO TaskSetManager: Finished task 108.0 in stage 17.0 (TID 642) in 1857 ms on 10.0.0.133 (executor driver) (109/200) +26/04/01 08:43:56 INFO Executor: Running task 116.0 in stage 17.0 (TID 650) +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:56 INFO Executor: Finished task 109.0 in stage 17.0 (TID 643). 6401 bytes result sent to driver +26/04/01 08:43:56 INFO TaskSetManager: Starting task 117.0 in stage 17.0 (TID 651) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:56 INFO Executor: Running task 117.0 in stage 17.0 (TID 651) +26/04/01 08:43:56 INFO TaskSetManager: Finished task 109.0 in stage 17.0 (TID 643) in 1856 ms on 10.0.0.133 (executor driver) (110/200) +26/04/01 08:43:56 INFO Executor: Finished task 110.0 in stage 17.0 (TID 644). 6401 bytes result sent to driver +26/04/01 08:43:56 INFO TaskSetManager: Starting task 118.0 in stage 17.0 (TID 652) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:56 INFO Executor: Running task 118.0 in stage 17.0 (TID 652) +26/04/01 08:43:56 INFO TaskSetManager: Finished task 110.0 in stage 17.0 (TID 644) in 1843 ms on 10.0.0.133 (executor driver) (111/200) +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (76.0 MiB) non-empty blocks including 208 (76.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (76.0 MiB) non-empty blocks including 208 (76.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:57 INFO Executor: Finished task 111.0 in stage 17.0 (TID 645). 6401 bytes result sent to driver +26/04/01 08:43:57 INFO TaskSetManager: Starting task 119.0 in stage 17.0 (TID 653) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:57 INFO TaskSetManager: Finished task 111.0 in stage 17.0 (TID 645) in 1852 ms on 10.0.0.133 (executor driver) (112/200) +26/04/01 08:43:57 INFO Executor: Running task 119.0 in stage 17.0 (TID 653) +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:57 INFO Executor: Finished task 114.0 in stage 17.0 (TID 648). 6401 bytes result sent to driver +26/04/01 08:43:57 INFO TaskSetManager: Starting task 120.0 in stage 17.0 (TID 654) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:57 INFO TaskSetManager: Finished task 114.0 in stage 17.0 (TID 648) in 1858 ms on 10.0.0.133 (executor driver) (113/200) +26/04/01 08:43:57 INFO Executor: Running task 120.0 in stage 17.0 (TID 654) +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:57 INFO Executor: Finished task 113.0 in stage 17.0 (TID 647). 6401 bytes result sent to driver +26/04/01 08:43:57 INFO TaskSetManager: Starting task 121.0 in stage 17.0 (TID 655) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:57 INFO TaskSetManager: Finished task 113.0 in stage 17.0 (TID 647) in 1862 ms on 10.0.0.133 (executor driver) (114/200) +26/04/01 08:43:57 INFO Executor: Running task 121.0 in stage 17.0 (TID 655) +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:57 INFO Executor: Finished task 112.0 in stage 17.0 (TID 646). 6401 bytes result sent to driver +26/04/01 08:43:57 INFO TaskSetManager: Starting task 122.0 in stage 17.0 (TID 656) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:57 INFO Executor: Running task 122.0 in stage 17.0 (TID 656) +26/04/01 08:43:57 INFO TaskSetManager: Finished task 112.0 in stage 17.0 (TID 646) in 1865 ms on 10.0.0.133 (executor driver) (115/200) +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:57 INFO Executor: Finished task 115.0 in stage 17.0 (TID 649). 6401 bytes result sent to driver +26/04/01 08:43:57 INFO TaskSetManager: Starting task 123.0 in stage 17.0 (TID 657) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:57 INFO TaskSetManager: Finished task 115.0 in stage 17.0 (TID 649) in 1852 ms on 10.0.0.133 (executor driver) (116/200) +26/04/01 08:43:57 INFO Executor: Running task 123.0 in stage 17.0 (TID 657) +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:58 INFO Executor: Finished task 116.0 in stage 17.0 (TID 650). 7132 bytes result sent to driver +26/04/01 08:43:58 INFO TaskSetManager: Starting task 124.0 in stage 17.0 (TID 658) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:58 INFO TaskSetManager: Finished task 116.0 in stage 17.0 (TID 650) in 1859 ms on 10.0.0.133 (executor driver) (117/200) +26/04/01 08:43:58 INFO Executor: Running task 124.0 in stage 17.0 (TID 658) +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:58 INFO Executor: Finished task 117.0 in stage 17.0 (TID 651). 7132 bytes result sent to driver +26/04/01 08:43:58 INFO TaskSetManager: Starting task 125.0 in stage 17.0 (TID 659) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:58 INFO Executor: Running task 125.0 in stage 17.0 (TID 659) +26/04/01 08:43:58 INFO TaskSetManager: Finished task 117.0 in stage 17.0 (TID 651) in 1858 ms on 10.0.0.133 (executor driver) (118/200) +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:58 INFO Executor: Finished task 118.0 in stage 17.0 (TID 652). 7132 bytes result sent to driver +26/04/01 08:43:58 INFO TaskSetManager: Starting task 126.0 in stage 17.0 (TID 660) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:58 INFO Executor: Running task 126.0 in stage 17.0 (TID 660) +26/04/01 08:43:58 INFO TaskSetManager: Finished task 118.0 in stage 17.0 (TID 652) in 1883 ms on 10.0.0.133 (executor driver) (119/200) +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:58 INFO Executor: Finished task 119.0 in stage 17.0 (TID 653). 7132 bytes result sent to driver +26/04/01 08:43:58 INFO TaskSetManager: Starting task 127.0 in stage 17.0 (TID 661) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:58 INFO TaskSetManager: Finished task 119.0 in stage 17.0 (TID 653) in 1851 ms on 10.0.0.133 (executor driver) (120/200) +26/04/01 08:43:58 INFO Executor: Running task 127.0 in stage 17.0 (TID 661) +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:59 INFO Executor: Finished task 120.0 in stage 17.0 (TID 654). 7132 bytes result sent to driver +26/04/01 08:43:59 INFO TaskSetManager: Starting task 128.0 in stage 17.0 (TID 662) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:59 INFO TaskSetManager: Finished task 120.0 in stage 17.0 (TID 654) in 1868 ms on 10.0.0.133 (executor driver) (121/200) +26/04/01 08:43:59 INFO Executor: Running task 128.0 in stage 17.0 (TID 662) +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:59 INFO Executor: Finished task 121.0 in stage 17.0 (TID 655). 7132 bytes result sent to driver +26/04/01 08:43:59 INFO TaskSetManager: Starting task 129.0 in stage 17.0 (TID 663) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:59 INFO Executor: Running task 129.0 in stage 17.0 (TID 663) +26/04/01 08:43:59 INFO TaskSetManager: Finished task 121.0 in stage 17.0 (TID 655) in 1870 ms on 10.0.0.133 (executor driver) (122/200) +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:59 INFO Executor: Finished task 122.0 in stage 17.0 (TID 656). 7132 bytes result sent to driver +26/04/01 08:43:59 INFO TaskSetManager: Starting task 130.0 in stage 17.0 (TID 664) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:59 INFO TaskSetManager: Finished task 122.0 in stage 17.0 (TID 656) in 1875 ms on 10.0.0.133 (executor driver) (123/200) +26/04/01 08:43:59 INFO Executor: Running task 130.0 in stage 17.0 (TID 664) +26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:59 INFO Executor: Finished task 123.0 in stage 17.0 (TID 657). 7132 bytes result sent to driver +26/04/01 08:43:59 INFO TaskSetManager: Starting task 131.0 in stage 17.0 (TID 665) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9624 bytes) +26/04/01 08:43:59 INFO Executor: Running task 131.0 in stage 17.0 (TID 665) +26/04/01 08:43:59 INFO TaskSetManager: Finished task 123.0 in stage 17.0 (TID 657) in 1868 ms on 10.0.0.133 (executor driver) (124/200) +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:00 INFO Executor: Finished task 124.0 in stage 17.0 (TID 658). 6401 bytes result sent to driver +26/04/01 08:44:00 INFO TaskSetManager: Starting task 132.0 in stage 17.0 (TID 666) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:00 INFO Executor: Running task 132.0 in stage 17.0 (TID 666) +26/04/01 08:44:00 INFO TaskSetManager: Finished task 124.0 in stage 17.0 (TID 658) in 1864 ms on 10.0.0.133 (executor driver) (125/200) +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:00 INFO Executor: Finished task 126.0 in stage 17.0 (TID 660). 6401 bytes result sent to driver +26/04/01 08:44:00 INFO TaskSetManager: Starting task 133.0 in stage 17.0 (TID 667) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:00 INFO Executor: Finished task 125.0 in stage 17.0 (TID 659). 6401 bytes result sent to driver +26/04/01 08:44:00 INFO Executor: Running task 133.0 in stage 17.0 (TID 667) +26/04/01 08:44:00 INFO TaskSetManager: Starting task 134.0 in stage 17.0 (TID 668) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:00 INFO Executor: Running task 134.0 in stage 17.0 (TID 668) +26/04/01 08:44:00 INFO TaskSetManager: Finished task 126.0 in stage 17.0 (TID 660) in 1854 ms on 10.0.0.133 (executor driver) (126/200) +26/04/01 08:44:00 INFO TaskSetManager: Finished task 125.0 in stage 17.0 (TID 659) in 1880 ms on 10.0.0.133 (executor driver) (127/200) +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:00 INFO Executor: Finished task 127.0 in stage 17.0 (TID 661). 6401 bytes result sent to driver +26/04/01 08:44:00 INFO TaskSetManager: Starting task 135.0 in stage 17.0 (TID 669) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:00 INFO Executor: Running task 135.0 in stage 17.0 (TID 669) +26/04/01 08:44:00 INFO TaskSetManager: Finished task 127.0 in stage 17.0 (TID 661) in 1852 ms on 10.0.0.133 (executor driver) (128/200) +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:01 INFO Executor: Finished task 128.0 in stage 17.0 (TID 662). 6401 bytes result sent to driver +26/04/01 08:44:01 INFO TaskSetManager: Starting task 136.0 in stage 17.0 (TID 670) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:01 INFO TaskSetManager: Finished task 128.0 in stage 17.0 (TID 662) in 1873 ms on 10.0.0.133 (executor driver) (129/200) +26/04/01 08:44:01 INFO Executor: Running task 136.0 in stage 17.0 (TID 670) +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:01 INFO Executor: Finished task 129.0 in stage 17.0 (TID 663). 6401 bytes result sent to driver +26/04/01 08:44:01 INFO TaskSetManager: Starting task 137.0 in stage 17.0 (TID 671) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:01 INFO TaskSetManager: Finished task 129.0 in stage 17.0 (TID 663) in 1869 ms on 10.0.0.133 (executor driver) (130/200) +26/04/01 08:44:01 INFO Executor: Running task 137.0 in stage 17.0 (TID 671) +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:01 INFO Executor: Finished task 131.0 in stage 17.0 (TID 665). 6401 bytes result sent to driver +26/04/01 08:44:01 INFO TaskSetManager: Starting task 138.0 in stage 17.0 (TID 672) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:01 INFO Executor: Running task 138.0 in stage 17.0 (TID 672) +26/04/01 08:44:01 INFO TaskSetManager: Finished task 131.0 in stage 17.0 (TID 665) in 1873 ms on 10.0.0.133 (executor driver) (131/200) +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:01 INFO Executor: Finished task 130.0 in stage 17.0 (TID 664). 6401 bytes result sent to driver +26/04/01 08:44:01 INFO TaskSetManager: Starting task 139.0 in stage 17.0 (TID 673) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:01 INFO TaskSetManager: Finished task 130.0 in stage 17.0 (TID 664) in 1884 ms on 10.0.0.133 (executor driver) (132/200) +26/04/01 08:44:01 INFO Executor: Running task 139.0 in stage 17.0 (TID 673) +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:02 INFO Executor: Finished task 132.0 in stage 17.0 (TID 666). 6401 bytes result sent to driver +26/04/01 08:44:02 INFO TaskSetManager: Starting task 140.0 in stage 17.0 (TID 674) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:02 INFO TaskSetManager: Finished task 132.0 in stage 17.0 (TID 666) in 1865 ms on 10.0.0.133 (executor driver) (133/200) +26/04/01 08:44:02 INFO Executor: Running task 140.0 in stage 17.0 (TID 674) +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:02 INFO Executor: Finished task 133.0 in stage 17.0 (TID 667). 6401 bytes result sent to driver +26/04/01 08:44:02 INFO TaskSetManager: Starting task 141.0 in stage 17.0 (TID 675) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:02 INFO TaskSetManager: Finished task 133.0 in stage 17.0 (TID 667) in 1870 ms on 10.0.0.133 (executor driver) (134/200) +26/04/01 08:44:02 INFO Executor: Running task 141.0 in stage 17.0 (TID 675) +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:02 INFO Executor: Finished task 134.0 in stage 17.0 (TID 668). 6401 bytes result sent to driver +26/04/01 08:44:02 INFO TaskSetManager: Starting task 142.0 in stage 17.0 (TID 676) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:02 INFO TaskSetManager: Finished task 134.0 in stage 17.0 (TID 668) in 1881 ms on 10.0.0.133 (executor driver) (135/200) +26/04/01 08:44:02 INFO Executor: Running task 142.0 in stage 17.0 (TID 676) +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:02 INFO Executor: Finished task 135.0 in stage 17.0 (TID 669). 6401 bytes result sent to driver +26/04/01 08:44:02 INFO TaskSetManager: Starting task 143.0 in stage 17.0 (TID 677) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:02 INFO TaskSetManager: Finished task 135.0 in stage 17.0 (TID 669) in 1884 ms on 10.0.0.133 (executor driver) (136/200) +26/04/01 08:44:02 INFO Executor: Running task 143.0 in stage 17.0 (TID 677) +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:03 INFO Executor: Finished task 136.0 in stage 17.0 (TID 670). 6401 bytes result sent to driver +26/04/01 08:44:03 INFO TaskSetManager: Starting task 144.0 in stage 17.0 (TID 678) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:03 INFO TaskSetManager: Finished task 136.0 in stage 17.0 (TID 670) in 1888 ms on 10.0.0.133 (executor driver) (137/200) +26/04/01 08:44:03 INFO Executor: Running task 144.0 in stage 17.0 (TID 678) +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:03 INFO Executor: Finished task 138.0 in stage 17.0 (TID 672). 6401 bytes result sent to driver +26/04/01 08:44:03 INFO TaskSetManager: Starting task 145.0 in stage 17.0 (TID 679) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:03 INFO TaskSetManager: Finished task 138.0 in stage 17.0 (TID 672) in 1880 ms on 10.0.0.133 (executor driver) (138/200) +26/04/01 08:44:03 INFO Executor: Running task 145.0 in stage 17.0 (TID 679) +26/04/01 08:44:03 INFO Executor: Finished task 137.0 in stage 17.0 (TID 671). 6401 bytes result sent to driver +26/04/01 08:44:03 INFO TaskSetManager: Starting task 146.0 in stage 17.0 (TID 680) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:03 INFO TaskSetManager: Finished task 137.0 in stage 17.0 (TID 671) in 1892 ms on 10.0.0.133 (executor driver) (139/200) +26/04/01 08:44:03 INFO Executor: Running task 146.0 in stage 17.0 (TID 680) +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:03 INFO Executor: Finished task 139.0 in stage 17.0 (TID 673). 6401 bytes result sent to driver +26/04/01 08:44:03 INFO TaskSetManager: Starting task 147.0 in stage 17.0 (TID 681) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:03 INFO Executor: Running task 147.0 in stage 17.0 (TID 681) +26/04/01 08:44:03 INFO TaskSetManager: Finished task 139.0 in stage 17.0 (TID 673) in 1887 ms on 10.0.0.133 (executor driver) (140/200) +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:04 INFO Executor: Finished task 140.0 in stage 17.0 (TID 674). 6401 bytes result sent to driver +26/04/01 08:44:04 INFO TaskSetManager: Starting task 148.0 in stage 17.0 (TID 682) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:04 INFO Executor: Running task 148.0 in stage 17.0 (TID 682) +26/04/01 08:44:04 INFO TaskSetManager: Finished task 140.0 in stage 17.0 (TID 674) in 1875 ms on 10.0.0.133 (executor driver) (141/200) +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:04 INFO Executor: Finished task 141.0 in stage 17.0 (TID 675). 6401 bytes result sent to driver +26/04/01 08:44:04 INFO TaskSetManager: Starting task 149.0 in stage 17.0 (TID 683) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:04 INFO Executor: Running task 149.0 in stage 17.0 (TID 683) +26/04/01 08:44:04 INFO TaskSetManager: Finished task 141.0 in stage 17.0 (TID 675) in 1874 ms on 10.0.0.133 (executor driver) (142/200) +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:04 INFO Executor: Finished task 143.0 in stage 17.0 (TID 677). 6401 bytes result sent to driver +26/04/01 08:44:04 INFO TaskSetManager: Starting task 150.0 in stage 17.0 (TID 684) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:04 INFO Executor: Running task 150.0 in stage 17.0 (TID 684) +26/04/01 08:44:04 INFO TaskSetManager: Finished task 143.0 in stage 17.0 (TID 677) in 1863 ms on 10.0.0.133 (executor driver) (143/200) +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:04 INFO Executor: Finished task 142.0 in stage 17.0 (TID 676). 6401 bytes result sent to driver +26/04/01 08:44:04 INFO TaskSetManager: Starting task 151.0 in stage 17.0 (TID 685) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:04 INFO Executor: Running task 151.0 in stage 17.0 (TID 685) +26/04/01 08:44:04 INFO TaskSetManager: Finished task 142.0 in stage 17.0 (TID 676) in 1883 ms on 10.0.0.133 (executor driver) (144/200) +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:05 INFO Executor: Finished task 144.0 in stage 17.0 (TID 678). 6401 bytes result sent to driver +26/04/01 08:44:05 INFO TaskSetManager: Starting task 152.0 in stage 17.0 (TID 686) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:05 INFO Executor: Running task 152.0 in stage 17.0 (TID 686) +26/04/01 08:44:05 INFO TaskSetManager: Finished task 144.0 in stage 17.0 (TID 678) in 1851 ms on 10.0.0.133 (executor driver) (145/200) +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:05 INFO Executor: Finished task 145.0 in stage 17.0 (TID 679). 6401 bytes result sent to driver +26/04/01 08:44:05 INFO TaskSetManager: Starting task 153.0 in stage 17.0 (TID 687) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:05 INFO Executor: Running task 153.0 in stage 17.0 (TID 687) +26/04/01 08:44:05 INFO TaskSetManager: Finished task 145.0 in stage 17.0 (TID 679) in 1856 ms on 10.0.0.133 (executor driver) (146/200) +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:05 INFO Executor: Finished task 146.0 in stage 17.0 (TID 680). 6401 bytes result sent to driver +26/04/01 08:44:05 INFO TaskSetManager: Starting task 154.0 in stage 17.0 (TID 688) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:05 INFO Executor: Running task 154.0 in stage 17.0 (TID 688) +26/04/01 08:44:05 INFO TaskSetManager: Finished task 146.0 in stage 17.0 (TID 680) in 1858 ms on 10.0.0.133 (executor driver) (147/200) +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:05 INFO Executor: Finished task 147.0 in stage 17.0 (TID 681). 6401 bytes result sent to driver +26/04/01 08:44:05 INFO TaskSetManager: Starting task 155.0 in stage 17.0 (TID 689) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:05 INFO TaskSetManager: Finished task 147.0 in stage 17.0 (TID 681) in 1861 ms on 10.0.0.133 (executor driver) (148/200) +26/04/01 08:44:05 INFO Executor: Running task 155.0 in stage 17.0 (TID 689) +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:06 INFO Executor: Finished task 148.0 in stage 17.0 (TID 682). 6401 bytes result sent to driver +26/04/01 08:44:06 INFO TaskSetManager: Starting task 156.0 in stage 17.0 (TID 690) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:06 INFO TaskSetManager: Finished task 148.0 in stage 17.0 (TID 682) in 1859 ms on 10.0.0.133 (executor driver) (149/200) +26/04/01 08:44:06 INFO Executor: Running task 156.0 in stage 17.0 (TID 690) +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:06 INFO Executor: Finished task 149.0 in stage 17.0 (TID 683). 6401 bytes result sent to driver +26/04/01 08:44:06 INFO TaskSetManager: Starting task 157.0 in stage 17.0 (TID 691) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:06 INFO TaskSetManager: Finished task 149.0 in stage 17.0 (TID 683) in 1877 ms on 10.0.0.133 (executor driver) (150/200) +26/04/01 08:44:06 INFO Executor: Running task 157.0 in stage 17.0 (TID 691) +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:06 INFO Executor: Finished task 150.0 in stage 17.0 (TID 684). 6401 bytes result sent to driver +26/04/01 08:44:06 INFO TaskSetManager: Starting task 158.0 in stage 17.0 (TID 692) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:06 INFO TaskSetManager: Finished task 150.0 in stage 17.0 (TID 684) in 1862 ms on 10.0.0.133 (executor driver) (151/200) +26/04/01 08:44:06 INFO Executor: Running task 158.0 in stage 17.0 (TID 692) +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:06 INFO Executor: Finished task 151.0 in stage 17.0 (TID 685). 6401 bytes result sent to driver +26/04/01 08:44:06 INFO TaskSetManager: Starting task 159.0 in stage 17.0 (TID 693) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:06 INFO Executor: Running task 159.0 in stage 17.0 (TID 693) +26/04/01 08:44:06 INFO TaskSetManager: Finished task 151.0 in stage 17.0 (TID 685) in 1866 ms on 10.0.0.133 (executor driver) (152/200) +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:07 INFO Executor: Finished task 152.0 in stage 17.0 (TID 686). 6401 bytes result sent to driver +26/04/01 08:44:07 INFO TaskSetManager: Starting task 160.0 in stage 17.0 (TID 694) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:07 INFO Executor: Running task 160.0 in stage 17.0 (TID 694) +26/04/01 08:44:07 INFO TaskSetManager: Finished task 152.0 in stage 17.0 (TID 686) in 1859 ms on 10.0.0.133 (executor driver) (153/200) +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:07 INFO Executor: Finished task 154.0 in stage 17.0 (TID 688). 6401 bytes result sent to driver +26/04/01 08:44:07 INFO TaskSetManager: Starting task 161.0 in stage 17.0 (TID 695) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:07 INFO Executor: Finished task 153.0 in stage 17.0 (TID 687). 6401 bytes result sent to driver +26/04/01 08:44:07 INFO Executor: Running task 161.0 in stage 17.0 (TID 695) +26/04/01 08:44:07 INFO TaskSetManager: Finished task 154.0 in stage 17.0 (TID 688) in 1858 ms on 10.0.0.133 (executor driver) (154/200) +26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:07 INFO TaskSetManager: Starting task 162.0 in stage 17.0 (TID 696) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:07 INFO TaskSetManager: Finished task 153.0 in stage 17.0 (TID 687) in 1861 ms on 10.0.0.133 (executor driver) (155/200) +26/04/01 08:44:07 INFO Executor: Running task 162.0 in stage 17.0 (TID 696) +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:07 INFO Executor: Finished task 155.0 in stage 17.0 (TID 689). 6401 bytes result sent to driver +26/04/01 08:44:07 INFO TaskSetManager: Starting task 163.0 in stage 17.0 (TID 697) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:07 INFO TaskSetManager: Finished task 155.0 in stage 17.0 (TID 689) in 1847 ms on 10.0.0.133 (executor driver) (156/200) +26/04/01 08:44:07 INFO Executor: Running task 163.0 in stage 17.0 (TID 697) +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:08 INFO Executor: Finished task 156.0 in stage 17.0 (TID 690). 6401 bytes result sent to driver +26/04/01 08:44:08 INFO TaskSetManager: Starting task 164.0 in stage 17.0 (TID 698) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:08 INFO TaskSetManager: Finished task 156.0 in stage 17.0 (TID 690) in 1870 ms on 10.0.0.133 (executor driver) (157/200) +26/04/01 08:44:08 INFO Executor: Running task 164.0 in stage 17.0 (TID 698) +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:08 INFO Executor: Finished task 157.0 in stage 17.0 (TID 691). 6401 bytes result sent to driver +26/04/01 08:44:08 INFO TaskSetManager: Starting task 165.0 in stage 17.0 (TID 699) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:08 INFO TaskSetManager: Finished task 157.0 in stage 17.0 (TID 691) in 1856 ms on 10.0.0.133 (executor driver) (158/200) +26/04/01 08:44:08 INFO Executor: Running task 165.0 in stage 17.0 (TID 699) +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:08 INFO Executor: Finished task 158.0 in stage 17.0 (TID 692). 6401 bytes result sent to driver +26/04/01 08:44:08 INFO TaskSetManager: Starting task 166.0 in stage 17.0 (TID 700) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:08 INFO TaskSetManager: Finished task 158.0 in stage 17.0 (TID 692) in 1875 ms on 10.0.0.133 (executor driver) (159/200) +26/04/01 08:44:08 INFO Executor: Running task 166.0 in stage 17.0 (TID 700) +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:08 INFO Executor: Finished task 159.0 in stage 17.0 (TID 693). 6401 bytes result sent to driver +26/04/01 08:44:08 INFO TaskSetManager: Starting task 167.0 in stage 17.0 (TID 701) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:08 INFO TaskSetManager: Finished task 159.0 in stage 17.0 (TID 693) in 1878 ms on 10.0.0.133 (executor driver) (160/200) +26/04/01 08:44:08 INFO Executor: Running task 167.0 in stage 17.0 (TID 701) +26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:09 INFO Executor: Finished task 161.0 in stage 17.0 (TID 695). 7132 bytes result sent to driver +26/04/01 08:44:09 INFO TaskSetManager: Starting task 168.0 in stage 17.0 (TID 702) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:09 INFO Executor: Running task 168.0 in stage 17.0 (TID 702) +26/04/01 08:44:09 INFO TaskSetManager: Finished task 161.0 in stage 17.0 (TID 695) in 1864 ms on 10.0.0.133 (executor driver) (161/200) +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:09 INFO Executor: Finished task 160.0 in stage 17.0 (TID 694). 7132 bytes result sent to driver +26/04/01 08:44:09 INFO TaskSetManager: Starting task 169.0 in stage 17.0 (TID 703) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:09 INFO Executor: Running task 169.0 in stage 17.0 (TID 703) +26/04/01 08:44:09 INFO TaskSetManager: Finished task 160.0 in stage 17.0 (TID 694) in 1888 ms on 10.0.0.133 (executor driver) (162/200) +26/04/01 08:44:09 INFO Executor: Finished task 162.0 in stage 17.0 (TID 696). 7132 bytes result sent to driver +26/04/01 08:44:09 INFO TaskSetManager: Starting task 170.0 in stage 17.0 (TID 704) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:09 INFO Executor: Running task 170.0 in stage 17.0 (TID 704) +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:09 INFO TaskSetManager: Finished task 162.0 in stage 17.0 (TID 696) in 1878 ms on 10.0.0.133 (executor driver) (163/200) +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:09 INFO Executor: Finished task 163.0 in stage 17.0 (TID 697). 7132 bytes result sent to driver +26/04/01 08:44:09 INFO TaskSetManager: Starting task 171.0 in stage 17.0 (TID 705) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:09 INFO Executor: Running task 171.0 in stage 17.0 (TID 705) +26/04/01 08:44:09 INFO TaskSetManager: Finished task 163.0 in stage 17.0 (TID 697) in 1876 ms on 10.0.0.133 (executor driver) (164/200) +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:09 INFO Executor: Finished task 164.0 in stage 17.0 (TID 698). 7132 bytes result sent to driver +26/04/01 08:44:09 INFO TaskSetManager: Starting task 172.0 in stage 17.0 (TID 706) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:09 INFO TaskSetManager: Finished task 164.0 in stage 17.0 (TID 698) in 1878 ms on 10.0.0.133 (executor driver) (165/200) +26/04/01 08:44:09 INFO Executor: Running task 172.0 in stage 17.0 (TID 706) +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:10 INFO Executor: Finished task 165.0 in stage 17.0 (TID 699). 7132 bytes result sent to driver +26/04/01 08:44:10 INFO TaskSetManager: Starting task 173.0 in stage 17.0 (TID 707) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:10 INFO TaskSetManager: Finished task 165.0 in stage 17.0 (TID 699) in 1868 ms on 10.0.0.133 (executor driver) (166/200) +26/04/01 08:44:10 INFO Executor: Running task 173.0 in stage 17.0 (TID 707) +26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:10 INFO Executor: Finished task 167.0 in stage 17.0 (TID 701). 7132 bytes result sent to driver +26/04/01 08:44:10 INFO TaskSetManager: Starting task 174.0 in stage 17.0 (TID 708) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:10 INFO TaskSetManager: Finished task 167.0 in stage 17.0 (TID 701) in 1851 ms on 10.0.0.133 (executor driver) (167/200) +26/04/01 08:44:10 INFO Executor: Running task 174.0 in stage 17.0 (TID 708) +26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:10 INFO Executor: Finished task 166.0 in stage 17.0 (TID 700). 7132 bytes result sent to driver +26/04/01 08:44:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:10 INFO TaskSetManager: Starting task 175.0 in stage 17.0 (TID 709) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:10 INFO TaskSetManager: Finished task 166.0 in stage 17.0 (TID 700) in 1871 ms on 10.0.0.133 (executor driver) (168/200) +26/04/01 08:44:10 INFO Executor: Running task 175.0 in stage 17.0 (TID 709) +26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO Executor: Finished task 168.0 in stage 17.0 (TID 702). 6401 bytes result sent to driver +26/04/01 08:44:11 INFO TaskSetManager: Starting task 176.0 in stage 17.0 (TID 710) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:11 INFO TaskSetManager: Finished task 168.0 in stage 17.0 (TID 702) in 1855 ms on 10.0.0.133 (executor driver) (169/200) +26/04/01 08:44:11 INFO Executor: Running task 176.0 in stage 17.0 (TID 710) +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO Executor: Finished task 170.0 in stage 17.0 (TID 704). 6401 bytes result sent to driver +26/04/01 08:44:11 INFO TaskSetManager: Starting task 177.0 in stage 17.0 (TID 711) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:11 INFO Executor: Running task 177.0 in stage 17.0 (TID 711) +26/04/01 08:44:11 INFO TaskSetManager: Finished task 170.0 in stage 17.0 (TID 704) in 1871 ms on 10.0.0.133 (executor driver) (170/200) +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO Executor: Finished task 169.0 in stage 17.0 (TID 703). 6401 bytes result sent to driver +26/04/01 08:44:11 INFO TaskSetManager: Starting task 178.0 in stage 17.0 (TID 712) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:11 INFO TaskSetManager: Finished task 169.0 in stage 17.0 (TID 703) in 1873 ms on 10.0.0.133 (executor driver) (171/200) +26/04/01 08:44:11 INFO Executor: Running task 178.0 in stage 17.0 (TID 712) +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO Executor: Finished task 171.0 in stage 17.0 (TID 705). 6401 bytes result sent to driver +26/04/01 08:44:11 INFO TaskSetManager: Starting task 179.0 in stage 17.0 (TID 713) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:11 INFO TaskSetManager: Finished task 171.0 in stage 17.0 (TID 705) in 1872 ms on 10.0.0.133 (executor driver) (172/200) +26/04/01 08:44:11 INFO Executor: Running task 179.0 in stage 17.0 (TID 713) +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO Executor: Finished task 172.0 in stage 17.0 (TID 706). 6401 bytes result sent to driver +26/04/01 08:44:11 INFO TaskSetManager: Starting task 180.0 in stage 17.0 (TID 714) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:11 INFO TaskSetManager: Finished task 172.0 in stage 17.0 (TID 706) in 1852 ms on 10.0.0.133 (executor driver) (173/200) +26/04/01 08:44:11 INFO Executor: Running task 180.0 in stage 17.0 (TID 714) +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO Executor: Finished task 173.0 in stage 17.0 (TID 707). 6401 bytes result sent to driver +26/04/01 08:44:11 INFO TaskSetManager: Starting task 181.0 in stage 17.0 (TID 715) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:11 INFO Executor: Running task 181.0 in stage 17.0 (TID 715) +26/04/01 08:44:11 INFO TaskSetManager: Finished task 173.0 in stage 17.0 (TID 707) in 1852 ms on 10.0.0.133 (executor driver) (174/200) +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO Executor: Finished task 174.0 in stage 17.0 (TID 708). 6401 bytes result sent to driver +26/04/01 08:44:11 INFO TaskSetManager: Starting task 182.0 in stage 17.0 (TID 716) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:11 INFO TaskSetManager: Finished task 174.0 in stage 17.0 (TID 708) in 1843 ms on 10.0.0.133 (executor driver) (175/200) +26/04/01 08:44:11 INFO Executor: Running task 182.0 in stage 17.0 (TID 716) +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO Executor: Finished task 175.0 in stage 17.0 (TID 709). 6401 bytes result sent to driver +26/04/01 08:44:11 INFO TaskSetManager: Starting task 183.0 in stage 17.0 (TID 717) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:11 INFO TaskSetManager: Finished task 175.0 in stage 17.0 (TID 709) in 1850 ms on 10.0.0.133 (executor driver) (176/200) +26/04/01 08:44:11 INFO Executor: Running task 183.0 in stage 17.0 (TID 717) +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:12 INFO Executor: Finished task 176.0 in stage 17.0 (TID 710). 6401 bytes result sent to driver +26/04/01 08:44:12 INFO TaskSetManager: Starting task 184.0 in stage 17.0 (TID 718) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:12 INFO TaskSetManager: Finished task 176.0 in stage 17.0 (TID 710) in 1833 ms on 10.0.0.133 (executor driver) (177/200) +26/04/01 08:44:12 INFO Executor: Running task 184.0 in stage 17.0 (TID 718) +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:12 INFO Executor: Finished task 177.0 in stage 17.0 (TID 711). 6401 bytes result sent to driver +26/04/01 08:44:12 INFO TaskSetManager: Starting task 185.0 in stage 17.0 (TID 719) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:12 INFO TaskSetManager: Finished task 177.0 in stage 17.0 (TID 711) in 1858 ms on 10.0.0.133 (executor driver) (178/200) +26/04/01 08:44:12 INFO Executor: Running task 185.0 in stage 17.0 (TID 719) +26/04/01 08:44:12 INFO Executor: Finished task 178.0 in stage 17.0 (TID 712). 6401 bytes result sent to driver +26/04/01 08:44:12 INFO TaskSetManager: Starting task 186.0 in stage 17.0 (TID 720) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:12 INFO Executor: Running task 186.0 in stage 17.0 (TID 720) +26/04/01 08:44:12 INFO TaskSetManager: Finished task 178.0 in stage 17.0 (TID 712) in 1858 ms on 10.0.0.133 (executor driver) (179/200) +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (77.0 MiB) non-empty blocks including 208 (77.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (77.0 MiB) non-empty blocks including 208 (77.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:12 INFO Executor: Finished task 179.0 in stage 17.0 (TID 713). 6401 bytes result sent to driver +26/04/01 08:44:12 INFO TaskSetManager: Starting task 187.0 in stage 17.0 (TID 721) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:12 INFO Executor: Running task 187.0 in stage 17.0 (TID 721) +26/04/01 08:44:12 INFO TaskSetManager: Finished task 179.0 in stage 17.0 (TID 713) in 1867 ms on 10.0.0.133 (executor driver) (180/200) +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:13 INFO Executor: Finished task 180.0 in stage 17.0 (TID 714). 6401 bytes result sent to driver +26/04/01 08:44:13 INFO TaskSetManager: Starting task 188.0 in stage 17.0 (TID 722) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:13 INFO TaskSetManager: Finished task 180.0 in stage 17.0 (TID 714) in 1848 ms on 10.0.0.133 (executor driver) (181/200) +26/04/01 08:44:13 INFO Executor: Running task 188.0 in stage 17.0 (TID 722) +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:13 INFO Executor: Finished task 181.0 in stage 17.0 (TID 715). 6401 bytes result sent to driver +26/04/01 08:44:13 INFO TaskSetManager: Starting task 189.0 in stage 17.0 (TID 723) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:13 INFO Executor: Running task 189.0 in stage 17.0 (TID 723) +26/04/01 08:44:13 INFO TaskSetManager: Finished task 181.0 in stage 17.0 (TID 715) in 1883 ms on 10.0.0.133 (executor driver) (182/200) +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:13 INFO Executor: Finished task 182.0 in stage 17.0 (TID 716). 6401 bytes result sent to driver +26/04/01 08:44:13 INFO TaskSetManager: Starting task 190.0 in stage 17.0 (TID 724) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:13 INFO Executor: Running task 190.0 in stage 17.0 (TID 724) +26/04/01 08:44:13 INFO TaskSetManager: Finished task 182.0 in stage 17.0 (TID 716) in 1881 ms on 10.0.0.133 (executor driver) (183/200) +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:13 INFO Executor: Finished task 183.0 in stage 17.0 (TID 717). 6401 bytes result sent to driver +26/04/01 08:44:13 INFO TaskSetManager: Starting task 191.0 in stage 17.0 (TID 725) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:13 INFO TaskSetManager: Finished task 183.0 in stage 17.0 (TID 717) in 1874 ms on 10.0.0.133 (executor driver) (184/200) +26/04/01 08:44:13 INFO Executor: Running task 191.0 in stage 17.0 (TID 725) +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:14 INFO Executor: Finished task 184.0 in stage 17.0 (TID 718). 6401 bytes result sent to driver +26/04/01 08:44:14 INFO TaskSetManager: Starting task 192.0 in stage 17.0 (TID 726) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:14 INFO Executor: Running task 192.0 in stage 17.0 (TID 726) +26/04/01 08:44:14 INFO TaskSetManager: Finished task 184.0 in stage 17.0 (TID 718) in 1873 ms on 10.0.0.133 (executor driver) (185/200) +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:14 INFO Executor: Finished task 186.0 in stage 17.0 (TID 720). 6401 bytes result sent to driver +26/04/01 08:44:14 INFO TaskSetManager: Starting task 193.0 in stage 17.0 (TID 727) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:14 INFO Executor: Running task 193.0 in stage 17.0 (TID 727) +26/04/01 08:44:14 INFO TaskSetManager: Finished task 186.0 in stage 17.0 (TID 720) in 1881 ms on 10.0.0.133 (executor driver) (186/200) +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:14 INFO Executor: Finished task 185.0 in stage 17.0 (TID 719). 6401 bytes result sent to driver +26/04/01 08:44:14 INFO TaskSetManager: Starting task 194.0 in stage 17.0 (TID 728) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:14 INFO Executor: Running task 194.0 in stage 17.0 (TID 728) +26/04/01 08:44:14 INFO TaskSetManager: Finished task 185.0 in stage 17.0 (TID 719) in 1894 ms on 10.0.0.133 (executor driver) (187/200) +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:14 INFO Executor: Finished task 187.0 in stage 17.0 (TID 721). 6401 bytes result sent to driver +26/04/01 08:44:14 INFO TaskSetManager: Starting task 195.0 in stage 17.0 (TID 729) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:14 INFO TaskSetManager: Finished task 187.0 in stage 17.0 (TID 721) in 1907 ms on 10.0.0.133 (executor driver) (188/200) +26/04/01 08:44:14 INFO Executor: Running task 195.0 in stage 17.0 (TID 729) +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:15 INFO Executor: Finished task 188.0 in stage 17.0 (TID 722). 6401 bytes result sent to driver +26/04/01 08:44:15 INFO TaskSetManager: Starting task 196.0 in stage 17.0 (TID 730) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:15 INFO Executor: Running task 196.0 in stage 17.0 (TID 730) +26/04/01 08:44:15 INFO TaskSetManager: Finished task 188.0 in stage 17.0 (TID 722) in 1916 ms on 10.0.0.133 (executor driver) (189/200) +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:15 INFO Executor: Finished task 189.0 in stage 17.0 (TID 723). 6401 bytes result sent to driver +26/04/01 08:44:15 INFO TaskSetManager: Starting task 197.0 in stage 17.0 (TID 731) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:15 INFO TaskSetManager: Finished task 189.0 in stage 17.0 (TID 723) in 1899 ms on 10.0.0.133 (executor driver) (190/200) +26/04/01 08:44:15 INFO Executor: Running task 197.0 in stage 17.0 (TID 731) +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:15 INFO Executor: Finished task 190.0 in stage 17.0 (TID 724). 6401 bytes result sent to driver +26/04/01 08:44:15 INFO TaskSetManager: Starting task 198.0 in stage 17.0 (TID 732) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:15 INFO Executor: Running task 198.0 in stage 17.0 (TID 732) +26/04/01 08:44:15 INFO TaskSetManager: Finished task 190.0 in stage 17.0 (TID 724) in 1900 ms on 10.0.0.133 (executor driver) (191/200) +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:15 INFO Executor: Finished task 191.0 in stage 17.0 (TID 725). 6401 bytes result sent to driver +26/04/01 08:44:15 INFO TaskSetManager: Starting task 199.0 in stage 17.0 (TID 733) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9624 bytes) +26/04/01 08:44:15 INFO TaskSetManager: Finished task 191.0 in stage 17.0 (TID 725) in 1898 ms on 10.0.0.133 (executor driver) (192/200) +26/04/01 08:44:15 INFO Executor: Running task 199.0 in stage 17.0 (TID 733) +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:16 INFO Executor: Finished task 192.0 in stage 17.0 (TID 726). 6401 bytes result sent to driver +26/04/01 08:44:16 INFO TaskSetManager: Finished task 192.0 in stage 17.0 (TID 726) in 1870 ms on 10.0.0.133 (executor driver) (193/200) +26/04/01 08:44:16 INFO Executor: Finished task 194.0 in stage 17.0 (TID 728). 6401 bytes result sent to driver +26/04/01 08:44:16 INFO TaskSetManager: Finished task 194.0 in stage 17.0 (TID 728) in 1860 ms on 10.0.0.133 (executor driver) (194/200) +26/04/01 08:44:16 INFO Executor: Finished task 193.0 in stage 17.0 (TID 727). 6401 bytes result sent to driver +26/04/01 08:44:16 INFO TaskSetManager: Finished task 193.0 in stage 17.0 (TID 727) in 1874 ms on 10.0.0.133 (executor driver) (195/200) +26/04/01 08:44:16 INFO Executor: Finished task 195.0 in stage 17.0 (TID 729). 6487 bytes result sent to driver +26/04/01 08:44:16 INFO TaskSetManager: Finished task 195.0 in stage 17.0 (TID 729) in 1870 ms on 10.0.0.133 (executor driver) (196/200) +26/04/01 08:44:17 INFO Executor: Finished task 196.0 in stage 17.0 (TID 730). 6444 bytes result sent to driver +26/04/01 08:44:17 INFO TaskSetManager: Finished task 196.0 in stage 17.0 (TID 730) in 1865 ms on 10.0.0.133 (executor driver) (197/200) +26/04/01 08:44:17 INFO Executor: Finished task 197.0 in stage 17.0 (TID 731). 6444 bytes result sent to driver +26/04/01 08:44:17 INFO TaskSetManager: Finished task 197.0 in stage 17.0 (TID 731) in 1867 ms on 10.0.0.133 (executor driver) (198/200) +26/04/01 08:44:17 INFO Executor: Finished task 198.0 in stage 17.0 (TID 732). 6444 bytes result sent to driver +26/04/01 08:44:17 INFO TaskSetManager: Finished task 198.0 in stage 17.0 (TID 732) in 1864 ms on 10.0.0.133 (executor driver) (199/200) +26/04/01 08:44:17 INFO Executor: Finished task 199.0 in stage 17.0 (TID 733). 6444 bytes result sent to driver +26/04/01 08:44:17 INFO TaskSetManager: Finished task 199.0 in stage 17.0 (TID 733) in 1863 ms on 10.0.0.133 (executor driver) (200/200) +26/04/01 08:44:17 INFO TaskSchedulerImpl: Removed TaskSet 17.0, whose tasks have all completed, from pool +26/04/01 08:44:17 INFO DAGScheduler: ShuffleMapStage 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 47.905 s +26/04/01 08:44:17 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:44:17 INFO DAGScheduler: running: Set() +26/04/01 08:44:17 INFO DAGScheduler: waiting: Set() +26/04/01 08:44:17 INFO DAGScheduler: failed: Set() +26/04/01 08:44:17 INFO ShufflePartitionsUtil: For shuffle(5, 2), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 08:44:17 INFO DAGScheduler: Registering RDD 43 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 6 +26/04/01 08:44:17 INFO DAGScheduler: Got map stage job 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 14 output partitions +26/04/01 08:44:17 INFO DAGScheduler: Final stage: ShuffleMapStage 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:44:17 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 19, ShuffleMapStage 20) +26/04/01 08:44:17 INFO DAGScheduler: Missing parents: List() +26/04/01 08:44:17 INFO DAGScheduler: Submitting ShuffleMapStage 21 (MapPartitionsRDD[43] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:44:17 INFO MemoryStore: Block broadcast_24 stored as values in memory (estimated size 24.1 KiB, free 8.6 GiB) +26/04/01 08:44:17 INFO MemoryStore: Block broadcast_24_piece0 stored as bytes in memory (estimated size 9.8 KiB, free 8.6 GiB) +26/04/01 08:44:17 INFO BlockManagerInfo: Added broadcast_24_piece0 in memory on 10.0.0.133:59303 (size: 9.8 KiB, free: 8.6 GiB) +26/04/01 08:44:17 INFO SparkContext: Created broadcast 24 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:44:17 INFO DAGScheduler: Submitting 14 missing tasks from ShuffleMapStage 21 (MapPartitionsRDD[43] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)) +26/04/01 08:44:17 INFO TaskSchedulerImpl: Adding task set 21.0 with 14 tasks resource profile 0 +26/04/01 08:44:17 INFO TaskSetManager: Starting task 0.0 in stage 21.0 (TID 734) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:17 INFO TaskSetManager: Starting task 1.0 in stage 21.0 (TID 735) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:17 INFO TaskSetManager: Starting task 2.0 in stage 21.0 (TID 736) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:17 INFO TaskSetManager: Starting task 3.0 in stage 21.0 (TID 737) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:17 INFO TaskSetManager: Starting task 4.0 in stage 21.0 (TID 738) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:17 INFO TaskSetManager: Starting task 5.0 in stage 21.0 (TID 739) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:17 INFO TaskSetManager: Starting task 6.0 in stage 21.0 (TID 740) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:17 INFO TaskSetManager: Starting task 7.0 in stage 21.0 (TID 741) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:17 INFO Executor: Running task 1.0 in stage 21.0 (TID 735) +26/04/01 08:44:17 INFO Executor: Running task 2.0 in stage 21.0 (TID 736) +26/04/01 08:44:17 INFO Executor: Running task 7.0 in stage 21.0 (TID 741) +26/04/01 08:44:17 INFO Executor: Running task 0.0 in stage 21.0 (TID 734) +26/04/01 08:44:17 INFO Executor: Running task 5.0 in stage 21.0 (TID 739) +26/04/01 08:44:17 INFO Executor: Running task 3.0 in stage 21.0 (TID 737) +26/04/01 08:44:17 INFO Executor: Running task 6.0 in stage 21.0 (TID 740) +26/04/01 08:44:17 INFO Executor: Running task 4.0 in stage 21.0 (TID 738) +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.0 MiB) non-empty blocks including 200 (61.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.3 MiB) non-empty blocks including 200 (61.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.6 MiB) non-empty blocks including 200 (61.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (618.3 KiB) non-empty blocks including 8 (618.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (619.7 KiB) non-empty blocks including 8 (619.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (624.5 KiB) non-empty blocks including 8 (624.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (621.6 KiB) non-empty blocks including 8 (621.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (619.1 KiB) non-empty blocks including 8 (619.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (618.2 KiB) non-empty blocks including 8 (618.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.6 MiB) non-empty blocks including 200 (61.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.0 MiB) non-empty blocks including 200 (61.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.3 MiB) non-empty blocks including 200 (61.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (618.3 KiB) non-empty blocks including 8 (618.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (619.7 KiB) non-empty blocks including 8 (619.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (624.5 KiB) non-empty blocks including 8 (624.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (619.1 KiB) non-empty blocks including 8 (619.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (618.2 KiB) non-empty blocks including 8 (618.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (621.6 KiB) non-empty blocks including 8 (621.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:19 INFO BlockManagerInfo: Removed broadcast_23_piece0 on 10.0.0.133:59303 in memory (size: 11.0 KiB, free: 8.6 GiB) +26/04/01 08:44:27 INFO Executor: Finished task 7.0 in stage 21.0 (TID 741). 9594 bytes result sent to driver +26/04/01 08:44:27 INFO TaskSetManager: Starting task 8.0 in stage 21.0 (TID 742) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:27 INFO Executor: Running task 8.0 in stage 21.0 (TID 742) +26/04/01 08:44:27 INFO TaskSetManager: Finished task 7.0 in stage 21.0 (TID 741) in 9887 ms on 10.0.0.133 (executor driver) (1/14) +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (623.7 KiB) non-empty blocks including 8 (623.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (623.7 KiB) non-empty blocks including 8 (623.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:27 INFO Executor: Finished task 0.0 in stage 21.0 (TID 734). 9594 bytes result sent to driver +26/04/01 08:44:27 INFO TaskSetManager: Starting task 9.0 in stage 21.0 (TID 743) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:27 INFO Executor: Running task 9.0 in stage 21.0 (TID 743) +26/04/01 08:44:27 INFO TaskSetManager: Finished task 0.0 in stage 21.0 (TID 734) in 9947 ms on 10.0.0.133 (executor driver) (2/14) +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (623.6 KiB) non-empty blocks including 8 (623.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (623.6 KiB) non-empty blocks including 8 (623.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:27 INFO Executor: Finished task 5.0 in stage 21.0 (TID 739). 9594 bytes result sent to driver +26/04/01 08:44:27 INFO TaskSetManager: Starting task 10.0 in stage 21.0 (TID 744) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:27 INFO Executor: Running task 10.0 in stage 21.0 (TID 744) +26/04/01 08:44:27 INFO TaskSetManager: Finished task 5.0 in stage 21.0 (TID 739) in 10048 ms on 10.0.0.133 (executor driver) (3/14) +26/04/01 08:44:27 INFO Executor: Finished task 6.0 in stage 21.0 (TID 740). 9594 bytes result sent to driver +26/04/01 08:44:27 INFO TaskSetManager: Starting task 11.0 in stage 21.0 (TID 745) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:27 INFO TaskSetManager: Finished task 6.0 in stage 21.0 (TID 740) in 10049 ms on 10.0.0.133 (executor driver) (4/14) +26/04/01 08:44:27 INFO Executor: Running task 11.0 in stage 21.0 (TID 745) +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.2 MiB) non-empty blocks including 200 (61.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (622.2 KiB) non-empty blocks including 8 (622.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (617.6 KiB) non-empty blocks including 8 (617.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.2 MiB) non-empty blocks including 200 (61.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (622.2 KiB) non-empty blocks including 8 (622.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (617.6 KiB) non-empty blocks including 8 (617.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:27 INFO Executor: Finished task 4.0 in stage 21.0 (TID 738). 9594 bytes result sent to driver +26/04/01 08:44:27 INFO TaskSetManager: Starting task 12.0 in stage 21.0 (TID 746) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:27 INFO TaskSetManager: Finished task 4.0 in stage 21.0 (TID 738) in 10095 ms on 10.0.0.133 (executor driver) (5/14) +26/04/01 08:44:27 INFO Executor: Running task 12.0 in stage 21.0 (TID 746) +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (622.0 KiB) non-empty blocks including 8 (622.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (622.0 KiB) non-empty blocks including 8 (622.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:27 INFO Executor: Finished task 2.0 in stage 21.0 (TID 736). 9594 bytes result sent to driver +26/04/01 08:44:27 INFO TaskSetManager: Starting task 13.0 in stage 21.0 (TID 747) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:27 INFO TaskSetManager: Finished task 2.0 in stage 21.0 (TID 736) in 10111 ms on 10.0.0.133 (executor driver) (6/14) +26/04/01 08:44:27 INFO Executor: Running task 13.0 in stage 21.0 (TID 747) +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (20.4 MiB) non-empty blocks including 200 (20.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (207.3 KiB) non-empty blocks including 8 (207.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (20.4 MiB) non-empty blocks including 200 (20.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (207.3 KiB) non-empty blocks including 8 (207.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:27 INFO Executor: Finished task 3.0 in stage 21.0 (TID 737). 9594 bytes result sent to driver +26/04/01 08:44:27 INFO TaskSetManager: Finished task 3.0 in stage 21.0 (TID 737) in 10162 ms on 10.0.0.133 (executor driver) (7/14) +26/04/01 08:44:27 INFO Executor: Finished task 1.0 in stage 21.0 (TID 735). 9594 bytes result sent to driver +26/04/01 08:44:27 INFO TaskSetManager: Finished task 1.0 in stage 21.0 (TID 735) in 10193 ms on 10.0.0.133 (executor driver) (8/14) +26/04/01 08:44:30 INFO Executor: Finished task 13.0 in stage 21.0 (TID 747). 9594 bytes result sent to driver +26/04/01 08:44:30 INFO TaskSetManager: Finished task 13.0 in stage 21.0 (TID 747) in 2979 ms on 10.0.0.133 (executor driver) (9/14) +26/04/01 08:44:36 INFO Executor: Finished task 10.0 in stage 21.0 (TID 744). 9594 bytes result sent to driver +26/04/01 08:44:36 INFO TaskSetManager: Finished task 10.0 in stage 21.0 (TID 744) in 9283 ms on 10.0.0.133 (executor driver) (10/14) +26/04/01 08:44:36 INFO Executor: Finished task 8.0 in stage 21.0 (TID 742). 9594 bytes result sent to driver +26/04/01 08:44:36 INFO TaskSetManager: Finished task 8.0 in stage 21.0 (TID 742) in 9452 ms on 10.0.0.133 (executor driver) (11/14) +26/04/01 08:44:36 INFO Executor: Finished task 9.0 in stage 21.0 (TID 743). 9594 bytes result sent to driver +26/04/01 08:44:36 INFO TaskSetManager: Finished task 9.0 in stage 21.0 (TID 743) in 9400 ms on 10.0.0.133 (executor driver) (12/14) +26/04/01 08:44:36 INFO Executor: Finished task 12.0 in stage 21.0 (TID 746). 9594 bytes result sent to driver +26/04/01 08:44:36 INFO TaskSetManager: Finished task 12.0 in stage 21.0 (TID 746) in 9261 ms on 10.0.0.133 (executor driver) (13/14) +26/04/01 08:44:36 INFO Executor: Finished task 11.0 in stage 21.0 (TID 745). 9594 bytes result sent to driver +26/04/01 08:44:36 INFO TaskSetManager: Finished task 11.0 in stage 21.0 (TID 745) in 9335 ms on 10.0.0.133 (executor driver) (14/14) +26/04/01 08:44:36 INFO TaskSchedulerImpl: Removed TaskSet 21.0, whose tasks have all completed, from pool +26/04/01 08:44:36 INFO DAGScheduler: ShuffleMapStage 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 19.386 s +26/04/01 08:44:36 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:44:36 INFO DAGScheduler: running: Set() +26/04/01 08:44:36 INFO DAGScheduler: waiting: Set() +26/04/01 08:44:36 INFO DAGScheduler: failed: Set() +26/04/01 08:44:36 INFO ShufflePartitionsUtil: For shuffle(6, 3), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 08:44:36 INFO DAGScheduler: Registering RDD 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 7 +26/04/01 08:44:36 INFO DAGScheduler: Got map stage job 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 29 output partitions +26/04/01 08:44:36 INFO DAGScheduler: Final stage: ShuffleMapStage 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:44:36 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 25, ShuffleMapStage 26) +26/04/01 08:44:36 INFO DAGScheduler: Missing parents: List() +26/04/01 08:44:36 INFO DAGScheduler: Submitting ShuffleMapStage 27 (MapPartitionsRDD[47] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:44:36 INFO MemoryStore: Block broadcast_25 stored as values in memory (estimated size 28.9 KiB, free 8.6 GiB) +26/04/01 08:44:36 INFO MemoryStore: Block broadcast_25_piece0 stored as bytes in memory (estimated size 10.3 KiB, free 8.6 GiB) +26/04/01 08:44:36 INFO BlockManagerInfo: Added broadcast_25_piece0 in memory on 10.0.0.133:59303 (size: 10.3 KiB, free: 8.6 GiB) +26/04/01 08:44:36 INFO SparkContext: Created broadcast 25 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:44:36 INFO DAGScheduler: Submitting 29 missing tasks from ShuffleMapStage 27 (MapPartitionsRDD[47] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:44:36 INFO TaskSchedulerImpl: Adding task set 27.0 with 29 tasks resource profile 0 +26/04/01 08:44:36 INFO TaskSetManager: Starting task 0.0 in stage 27.0 (TID 748) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:36 INFO TaskSetManager: Starting task 1.0 in stage 27.0 (TID 749) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:36 INFO TaskSetManager: Starting task 2.0 in stage 27.0 (TID 750) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:36 INFO TaskSetManager: Starting task 3.0 in stage 27.0 (TID 751) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:36 INFO TaskSetManager: Starting task 4.0 in stage 27.0 (TID 752) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:36 INFO TaskSetManager: Starting task 5.0 in stage 27.0 (TID 753) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:36 INFO TaskSetManager: Starting task 6.0 in stage 27.0 (TID 754) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:36 INFO TaskSetManager: Starting task 7.0 in stage 27.0 (TID 755) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:36 INFO Executor: Running task 0.0 in stage 27.0 (TID 748) +26/04/01 08:44:36 INFO Executor: Running task 2.0 in stage 27.0 (TID 750) +26/04/01 08:44:36 INFO Executor: Running task 1.0 in stage 27.0 (TID 749) +26/04/01 08:44:36 INFO Executor: Running task 5.0 in stage 27.0 (TID 753) +26/04/01 08:44:36 INFO Executor: Running task 3.0 in stage 27.0 (TID 751) +26/04/01 08:44:36 INFO Executor: Running task 6.0 in stage 27.0 (TID 754) +26/04/01 08:44:36 INFO Executor: Running task 7.0 in stage 27.0 (TID 755) +26/04/01 08:44:36 INFO Executor: Running task 4.0 in stage 27.0 (TID 752) +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.6 MiB) non-empty blocks including 14 (23.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.2 MiB) non-empty blocks including 14 (23.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.2 MiB) non-empty blocks including 14 (23.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.6 MiB) non-empty blocks including 14 (23.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:46 INFO Executor: Finished task 5.0 in stage 27.0 (TID 753). 9594 bytes result sent to driver +26/04/01 08:44:46 INFO TaskSetManager: Starting task 8.0 in stage 27.0 (TID 756) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:46 INFO Executor: Running task 8.0 in stage 27.0 (TID 756) +26/04/01 08:44:46 INFO TaskSetManager: Finished task 5.0 in stage 27.0 (TID 753) in 9399 ms on 10.0.0.133 (executor driver) (1/29) +26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:47 INFO Executor: Finished task 7.0 in stage 27.0 (TID 755). 9594 bytes result sent to driver +26/04/01 08:44:47 INFO TaskSetManager: Starting task 9.0 in stage 27.0 (TID 757) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:47 INFO Executor: Running task 9.0 in stage 27.0 (TID 757) +26/04/01 08:44:47 INFO TaskSetManager: Finished task 7.0 in stage 27.0 (TID 755) in 10074 ms on 10.0.0.133 (executor driver) (2/29) +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:47 INFO Executor: Finished task 4.0 in stage 27.0 (TID 752). 9594 bytes result sent to driver +26/04/01 08:44:47 INFO TaskSetManager: Starting task 10.0 in stage 27.0 (TID 758) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:47 INFO Executor: Running task 10.0 in stage 27.0 (TID 758) +26/04/01 08:44:47 INFO TaskSetManager: Finished task 4.0 in stage 27.0 (TID 752) in 10242 ms on 10.0.0.133 (executor driver) (3/29) +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:47 INFO Executor: Finished task 3.0 in stage 27.0 (TID 751). 9594 bytes result sent to driver +26/04/01 08:44:47 INFO TaskSetManager: Starting task 11.0 in stage 27.0 (TID 759) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:47 INFO TaskSetManager: Finished task 3.0 in stage 27.0 (TID 751) in 10315 ms on 10.0.0.133 (executor driver) (4/29) +26/04/01 08:44:47 INFO Executor: Running task 11.0 in stage 27.0 (TID 759) +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:47 INFO Executor: Finished task 6.0 in stage 27.0 (TID 754). 9594 bytes result sent to driver +26/04/01 08:44:47 INFO TaskSetManager: Starting task 12.0 in stage 27.0 (TID 760) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:47 INFO TaskSetManager: Finished task 6.0 in stage 27.0 (TID 754) in 10328 ms on 10.0.0.133 (executor driver) (5/29) +26/04/01 08:44:47 INFO Executor: Running task 12.0 in stage 27.0 (TID 760) +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:47 INFO Executor: Finished task 2.0 in stage 27.0 (TID 750). 9594 bytes result sent to driver +26/04/01 08:44:47 INFO TaskSetManager: Starting task 13.0 in stage 27.0 (TID 761) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:47 INFO TaskSetManager: Finished task 2.0 in stage 27.0 (TID 750) in 10839 ms on 10.0.0.133 (executor driver) (6/29) +26/04/01 08:44:47 INFO Executor: Running task 13.0 in stage 27.0 (TID 761) +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:47 INFO Executor: Finished task 0.0 in stage 27.0 (TID 748). 9594 bytes result sent to driver +26/04/01 08:44:47 INFO TaskSetManager: Starting task 14.0 in stage 27.0 (TID 762) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:47 INFO TaskSetManager: Finished task 0.0 in stage 27.0 (TID 748) in 10894 ms on 10.0.0.133 (executor driver) (7/29) +26/04/01 08:44:47 INFO Executor: Running task 14.0 in stage 27.0 (TID 762) +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:47 INFO Executor: Finished task 1.0 in stage 27.0 (TID 749). 9594 bytes result sent to driver +26/04/01 08:44:47 INFO TaskSetManager: Starting task 15.0 in stage 27.0 (TID 763) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:47 INFO TaskSetManager: Finished task 1.0 in stage 27.0 (TID 749) in 10902 ms on 10.0.0.133 (executor driver) (8/29) +26/04/01 08:44:47 INFO Executor: Running task 15.0 in stage 27.0 (TID 763) +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:55 INFO Executor: Finished task 8.0 in stage 27.0 (TID 756). 9594 bytes result sent to driver +26/04/01 08:44:55 INFO TaskSetManager: Starting task 16.0 in stage 27.0 (TID 764) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:55 INFO Executor: Running task 16.0 in stage 27.0 (TID 764) +26/04/01 08:44:55 INFO TaskSetManager: Finished task 8.0 in stage 27.0 (TID 756) in 9022 ms on 10.0.0.133 (executor driver) (9/29) +26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:56 INFO Executor: Finished task 9.0 in stage 27.0 (TID 757). 9594 bytes result sent to driver +26/04/01 08:44:56 INFO TaskSetManager: Starting task 17.0 in stage 27.0 (TID 765) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:56 INFO Executor: Running task 17.0 in stage 27.0 (TID 765) +26/04/01 08:44:56 INFO TaskSetManager: Finished task 9.0 in stage 27.0 (TID 757) in 9059 ms on 10.0.0.133 (executor driver) (10/29) +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:56 INFO Executor: Finished task 11.0 in stage 27.0 (TID 759). 9594 bytes result sent to driver +26/04/01 08:44:56 INFO TaskSetManager: Starting task 18.0 in stage 27.0 (TID 766) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:56 INFO Executor: Running task 18.0 in stage 27.0 (TID 766) +26/04/01 08:44:56 INFO TaskSetManager: Finished task 11.0 in stage 27.0 (TID 759) in 9606 ms on 10.0.0.133 (executor driver) (11/29) +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:57 INFO Executor: Finished task 10.0 in stage 27.0 (TID 758). 9594 bytes result sent to driver +26/04/01 08:44:57 INFO TaskSetManager: Starting task 19.0 in stage 27.0 (TID 767) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:57 INFO TaskSetManager: Finished task 10.0 in stage 27.0 (TID 758) in 9859 ms on 10.0.0.133 (executor driver) (12/29) +26/04/01 08:44:57 INFO Executor: Running task 19.0 in stage 27.0 (TID 767) +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:57 INFO Executor: Finished task 12.0 in stage 27.0 (TID 760). 9594 bytes result sent to driver +26/04/01 08:44:57 INFO TaskSetManager: Starting task 20.0 in stage 27.0 (TID 768) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:57 INFO Executor: Running task 20.0 in stage 27.0 (TID 768) +26/04/01 08:44:57 INFO TaskSetManager: Finished task 12.0 in stage 27.0 (TID 760) in 9836 ms on 10.0.0.133 (executor driver) (13/29) +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:57 INFO Executor: Finished task 15.0 in stage 27.0 (TID 763). 9594 bytes result sent to driver +26/04/01 08:44:57 INFO Executor: Finished task 14.0 in stage 27.0 (TID 762). 9594 bytes result sent to driver +26/04/01 08:44:57 INFO TaskSetManager: Starting task 21.0 in stage 27.0 (TID 769) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:57 INFO Executor: Running task 21.0 in stage 27.0 (TID 769) +26/04/01 08:44:57 INFO TaskSetManager: Starting task 22.0 in stage 27.0 (TID 770) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:57 INFO TaskSetManager: Finished task 15.0 in stage 27.0 (TID 763) in 9658 ms on 10.0.0.133 (executor driver) (14/29) +26/04/01 08:44:57 INFO Executor: Running task 22.0 in stage 27.0 (TID 770) +26/04/01 08:44:57 INFO TaskSetManager: Finished task 14.0 in stage 27.0 (TID 762) in 9667 ms on 10.0.0.133 (executor driver) (15/29) +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:57 INFO Executor: Finished task 13.0 in stage 27.0 (TID 761). 9594 bytes result sent to driver +26/04/01 08:44:57 INFO TaskSetManager: Starting task 23.0 in stage 27.0 (TID 771) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9428 bytes) +26/04/01 08:44:57 INFO TaskSetManager: Finished task 13.0 in stage 27.0 (TID 761) in 9735 ms on 10.0.0.133 (executor driver) (16/29) +26/04/01 08:44:57 INFO Executor: Running task 23.0 in stage 27.0 (TID 771) +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:04 INFO Executor: Finished task 16.0 in stage 27.0 (TID 764). 9594 bytes result sent to driver +26/04/01 08:45:04 INFO TaskSetManager: Starting task 24.0 in stage 27.0 (TID 772) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9428 bytes) +26/04/01 08:45:04 INFO Executor: Running task 24.0 in stage 27.0 (TID 772) +26/04/01 08:45:04 INFO TaskSetManager: Finished task 16.0 in stage 27.0 (TID 764) in 8861 ms on 10.0.0.133 (executor driver) (17/29) +26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:05 INFO Executor: Finished task 17.0 in stage 27.0 (TID 765). 9594 bytes result sent to driver +26/04/01 08:45:05 INFO TaskSetManager: Starting task 25.0 in stage 27.0 (TID 773) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9428 bytes) +26/04/01 08:45:05 INFO TaskSetManager: Finished task 17.0 in stage 27.0 (TID 765) in 9168 ms on 10.0.0.133 (executor driver) (18/29) +26/04/01 08:45:05 INFO Executor: Running task 25.0 in stage 27.0 (TID 773) +26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:06 INFO Executor: Finished task 18.0 in stage 27.0 (TID 766). 9594 bytes result sent to driver +26/04/01 08:45:06 INFO TaskSetManager: Starting task 26.0 in stage 27.0 (TID 774) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9428 bytes) +26/04/01 08:45:06 INFO Executor: Running task 26.0 in stage 27.0 (TID 774) +26/04/01 08:45:06 INFO TaskSetManager: Finished task 18.0 in stage 27.0 (TID 766) in 9253 ms on 10.0.0.133 (executor driver) (19/29) +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:06 INFO Executor: Finished task 20.0 in stage 27.0 (TID 768). 9594 bytes result sent to driver +26/04/01 08:45:06 INFO TaskSetManager: Starting task 27.0 in stage 27.0 (TID 775) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9428 bytes) +26/04/01 08:45:06 INFO Executor: Running task 27.0 in stage 27.0 (TID 775) +26/04/01 08:45:06 INFO TaskSetManager: Finished task 20.0 in stage 27.0 (TID 768) in 9211 ms on 10.0.0.133 (executor driver) (20/29) +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:06 INFO Executor: Finished task 19.0 in stage 27.0 (TID 767). 9594 bytes result sent to driver +26/04/01 08:45:06 INFO TaskSetManager: Starting task 28.0 in stage 27.0 (TID 776) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9428 bytes) +26/04/01 08:45:06 INFO TaskSetManager: Finished task 19.0 in stage 27.0 (TID 767) in 9525 ms on 10.0.0.133 (executor driver) (21/29) +26/04/01 08:45:06 INFO Executor: Running task 28.0 in stage 27.0 (TID 776) +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 14 (13.4 MiB) non-empty blocks including 14 (13.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 37 (21.8 MiB) non-empty blocks including 37 (21.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 14 (13.4 MiB) non-empty blocks including 14 (13.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 37 (21.8 MiB) non-empty blocks including 37 (21.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:06 INFO Executor: Finished task 21.0 in stage 27.0 (TID 769). 9594 bytes result sent to driver +26/04/01 08:45:06 INFO TaskSetManager: Finished task 21.0 in stage 27.0 (TID 769) in 9189 ms on 10.0.0.133 (executor driver) (22/29) +26/04/01 08:45:06 INFO Executor: Finished task 22.0 in stage 27.0 (TID 770). 9594 bytes result sent to driver +26/04/01 08:45:06 INFO TaskSetManager: Finished task 22.0 in stage 27.0 (TID 770) in 9363 ms on 10.0.0.133 (executor driver) (23/29) +26/04/01 08:45:07 INFO Executor: Finished task 23.0 in stage 27.0 (TID 771). 9594 bytes result sent to driver +26/04/01 08:45:07 INFO TaskSetManager: Finished task 23.0 in stage 27.0 (TID 771) in 9516 ms on 10.0.0.133 (executor driver) (24/29) +26/04/01 08:45:11 INFO Executor: Finished task 28.0 in stage 27.0 (TID 776). 9594 bytes result sent to driver +26/04/01 08:45:11 INFO TaskSetManager: Finished task 28.0 in stage 27.0 (TID 776) in 5264 ms on 10.0.0.133 (executor driver) (25/29) +26/04/01 08:45:13 INFO Executor: Finished task 24.0 in stage 27.0 (TID 772). 9594 bytes result sent to driver +26/04/01 08:45:13 INFO TaskSetManager: Finished task 24.0 in stage 27.0 (TID 772) in 8823 ms on 10.0.0.133 (executor driver) (26/29) +26/04/01 08:45:14 INFO Executor: Finished task 25.0 in stage 27.0 (TID 773). 9594 bytes result sent to driver +26/04/01 08:45:14 INFO TaskSetManager: Finished task 25.0 in stage 27.0 (TID 773) in 8795 ms on 10.0.0.133 (executor driver) (27/29) +26/04/01 08:45:15 INFO Executor: Finished task 26.0 in stage 27.0 (TID 774). 9594 bytes result sent to driver +26/04/01 08:45:15 INFO TaskSetManager: Finished task 26.0 in stage 27.0 (TID 774) in 8852 ms on 10.0.0.133 (executor driver) (28/29) +26/04/01 08:45:15 INFO Executor: Finished task 27.0 in stage 27.0 (TID 775). 9594 bytes result sent to driver +26/04/01 08:45:15 INFO TaskSetManager: Finished task 27.0 in stage 27.0 (TID 775) in 8855 ms on 10.0.0.133 (executor driver) (29/29) +26/04/01 08:45:15 INFO TaskSchedulerImpl: Removed TaskSet 27.0, whose tasks have all completed, from pool +26/04/01 08:45:15 INFO DAGScheduler: ShuffleMapStage 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 38.232 s +26/04/01 08:45:15 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:45:15 INFO DAGScheduler: running: Set() +26/04/01 08:45:15 INFO DAGScheduler: waiting: Set() +26/04/01 08:45:15 INFO DAGScheduler: failed: Set() +26/04/01 08:45:15 INFO ShufflePartitionsUtil: For shuffle(7, 4), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 08:45:15 INFO DAGScheduler: Registering RDD 52 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 8 +26/04/01 08:45:15 INFO DAGScheduler: Got map stage job 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 34 output partitions +26/04/01 08:45:15 INFO DAGScheduler: Final stage: ShuffleMapStage 35 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:45:15 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 33, ShuffleMapStage 34) +26/04/01 08:45:15 INFO DAGScheduler: Missing parents: List() +26/04/01 08:45:15 INFO DAGScheduler: Submitting ShuffleMapStage 35 (MapPartitionsRDD[52] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:45:15 INFO MemoryStore: Block broadcast_26 stored as values in memory (estimated size 44.4 KiB, free 8.6 GiB) +26/04/01 08:45:15 INFO MemoryStore: Block broadcast_26_piece0 stored as bytes in memory (estimated size 12.6 KiB, free 8.6 GiB) +26/04/01 08:45:15 INFO BlockManagerInfo: Added broadcast_26_piece0 in memory on 10.0.0.133:59303 (size: 12.6 KiB, free: 8.6 GiB) +26/04/01 08:45:15 INFO SparkContext: Created broadcast 26 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:45:15 INFO DAGScheduler: Submitting 34 missing tasks from ShuffleMapStage 35 (MapPartitionsRDD[52] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:45:15 INFO TaskSchedulerImpl: Adding task set 35.0 with 34 tasks resource profile 0 +26/04/01 08:45:15 INFO TaskSetManager: Starting task 0.0 in stage 35.0 (TID 777) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:15 INFO TaskSetManager: Starting task 1.0 in stage 35.0 (TID 778) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:15 INFO TaskSetManager: Starting task 2.0 in stage 35.0 (TID 779) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:15 INFO TaskSetManager: Starting task 3.0 in stage 35.0 (TID 780) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:15 INFO TaskSetManager: Starting task 4.0 in stage 35.0 (TID 781) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:15 INFO TaskSetManager: Starting task 5.0 in stage 35.0 (TID 782) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:15 INFO TaskSetManager: Starting task 6.0 in stage 35.0 (TID 783) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:15 INFO TaskSetManager: Starting task 7.0 in stage 35.0 (TID 784) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:15 INFO Executor: Running task 0.0 in stage 35.0 (TID 777) +26/04/01 08:45:15 INFO Executor: Running task 1.0 in stage 35.0 (TID 778) +26/04/01 08:45:15 INFO Executor: Running task 3.0 in stage 35.0 (TID 780) +26/04/01 08:45:15 INFO Executor: Running task 2.0 in stage 35.0 (TID 779) +26/04/01 08:45:15 INFO Executor: Running task 5.0 in stage 35.0 (TID 782) +26/04/01 08:45:15 INFO Executor: Running task 7.0 in stage 35.0 (TID 784) +26/04/01 08:45:15 INFO Executor: Running task 4.0 in stage 35.0 (TID 781) +26/04/01 08:45:15 INFO Executor: Running task 6.0 in stage 35.0 (TID 783) +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO Executor: Finished task 0.0 in stage 35.0 (TID 777). 10455 bytes result sent to driver +26/04/01 08:45:22 INFO TaskSetManager: Starting task 8.0 in stage 35.0 (TID 785) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:22 INFO TaskSetManager: Finished task 0.0 in stage 35.0 (TID 777) in 7148 ms on 10.0.0.133 (executor driver) (1/34) +26/04/01 08:45:22 INFO Executor: Running task 8.0 in stage 35.0 (TID 785) +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO Executor: Finished task 3.0 in stage 35.0 (TID 780). 10455 bytes result sent to driver +26/04/01 08:45:22 INFO TaskSetManager: Starting task 9.0 in stage 35.0 (TID 786) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:22 INFO TaskSetManager: Finished task 3.0 in stage 35.0 (TID 780) in 7424 ms on 10.0.0.133 (executor driver) (2/34) +26/04/01 08:45:22 INFO Executor: Running task 9.0 in stage 35.0 (TID 786) +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO Executor: Finished task 1.0 in stage 35.0 (TID 778). 10455 bytes result sent to driver +26/04/01 08:45:22 INFO TaskSetManager: Starting task 10.0 in stage 35.0 (TID 787) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:22 INFO Executor: Running task 10.0 in stage 35.0 (TID 787) +26/04/01 08:45:22 INFO TaskSetManager: Finished task 1.0 in stage 35.0 (TID 778) in 7433 ms on 10.0.0.133 (executor driver) (3/34) +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO Executor: Finished task 2.0 in stage 35.0 (TID 779). 10455 bytes result sent to driver +26/04/01 08:45:22 INFO TaskSetManager: Starting task 11.0 in stage 35.0 (TID 788) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:22 INFO TaskSetManager: Finished task 2.0 in stage 35.0 (TID 779) in 7501 ms on 10.0.0.133 (executor driver) (4/34) +26/04/01 08:45:22 INFO Executor: Running task 11.0 in stage 35.0 (TID 788) +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO Executor: Finished task 6.0 in stage 35.0 (TID 783). 10455 bytes result sent to driver +26/04/01 08:45:22 INFO TaskSetManager: Starting task 12.0 in stage 35.0 (TID 789) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:22 INFO TaskSetManager: Finished task 6.0 in stage 35.0 (TID 783) in 7539 ms on 10.0.0.133 (executor driver) (5/34) +26/04/01 08:45:22 INFO Executor: Running task 12.0 in stage 35.0 (TID 789) +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO Executor: Finished task 7.0 in stage 35.0 (TID 784). 10455 bytes result sent to driver +26/04/01 08:45:22 INFO TaskSetManager: Starting task 13.0 in stage 35.0 (TID 790) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:22 INFO Executor: Running task 13.0 in stage 35.0 (TID 790) +26/04/01 08:45:22 INFO TaskSetManager: Finished task 7.0 in stage 35.0 (TID 784) in 7559 ms on 10.0.0.133 (executor driver) (6/34) +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO Executor: Finished task 4.0 in stage 35.0 (TID 781). 10455 bytes result sent to driver +26/04/01 08:45:22 INFO TaskSetManager: Starting task 14.0 in stage 35.0 (TID 791) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:22 INFO TaskSetManager: Finished task 4.0 in stage 35.0 (TID 781) in 7573 ms on 10.0.0.133 (executor driver) (7/34) +26/04/01 08:45:22 INFO Executor: Running task 14.0 in stage 35.0 (TID 791) +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO Executor: Finished task 5.0 in stage 35.0 (TID 782). 10455 bytes result sent to driver +26/04/01 08:45:22 INFO TaskSetManager: Starting task 15.0 in stage 35.0 (TID 792) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:22 INFO Executor: Running task 15.0 in stage 35.0 (TID 792) +26/04/01 08:45:22 INFO TaskSetManager: Finished task 5.0 in stage 35.0 (TID 782) in 7576 ms on 10.0.0.133 (executor driver) (8/34) +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO Executor: Finished task 8.0 in stage 35.0 (TID 785). 10455 bytes result sent to driver +26/04/01 08:45:29 INFO TaskSetManager: Starting task 16.0 in stage 35.0 (TID 793) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:29 INFO Executor: Running task 16.0 in stage 35.0 (TID 793) +26/04/01 08:45:29 INFO TaskSetManager: Finished task 8.0 in stage 35.0 (TID 785) in 6922 ms on 10.0.0.133 (executor driver) (9/34) +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO Executor: Finished task 9.0 in stage 35.0 (TID 786). 10455 bytes result sent to driver +26/04/01 08:45:29 INFO TaskSetManager: Starting task 17.0 in stage 35.0 (TID 794) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:29 INFO Executor: Running task 17.0 in stage 35.0 (TID 794) +26/04/01 08:45:29 INFO TaskSetManager: Finished task 9.0 in stage 35.0 (TID 786) in 6863 ms on 10.0.0.133 (executor driver) (10/34) +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO Executor: Finished task 10.0 in stage 35.0 (TID 787). 10455 bytes result sent to driver +26/04/01 08:45:29 INFO TaskSetManager: Starting task 18.0 in stage 35.0 (TID 795) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:29 INFO TaskSetManager: Finished task 10.0 in stage 35.0 (TID 787) in 6919 ms on 10.0.0.133 (executor driver) (11/34) +26/04/01 08:45:29 INFO Executor: Running task 18.0 in stage 35.0 (TID 795) +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO Executor: Finished task 11.0 in stage 35.0 (TID 788). 10455 bytes result sent to driver +26/04/01 08:45:29 INFO TaskSetManager: Starting task 19.0 in stage 35.0 (TID 796) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:29 INFO TaskSetManager: Finished task 11.0 in stage 35.0 (TID 788) in 6907 ms on 10.0.0.133 (executor driver) (12/34) +26/04/01 08:45:29 INFO Executor: Running task 19.0 in stage 35.0 (TID 796) +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO Executor: Finished task 12.0 in stage 35.0 (TID 789). 10455 bytes result sent to driver +26/04/01 08:45:29 INFO TaskSetManager: Starting task 20.0 in stage 35.0 (TID 797) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:29 INFO Executor: Running task 20.0 in stage 35.0 (TID 797) +26/04/01 08:45:29 INFO TaskSetManager: Finished task 12.0 in stage 35.0 (TID 789) in 6937 ms on 10.0.0.133 (executor driver) (13/34) +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO Executor: Finished task 13.0 in stage 35.0 (TID 790). 10455 bytes result sent to driver +26/04/01 08:45:29 INFO TaskSetManager: Starting task 21.0 in stage 35.0 (TID 798) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:29 INFO TaskSetManager: Finished task 13.0 in stage 35.0 (TID 790) in 6989 ms on 10.0.0.133 (executor driver) (14/34) +26/04/01 08:45:29 INFO Executor: Running task 21.0 in stage 35.0 (TID 798) +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO Executor: Finished task 14.0 in stage 35.0 (TID 791). 10455 bytes result sent to driver +26/04/01 08:45:29 INFO TaskSetManager: Starting task 22.0 in stage 35.0 (TID 799) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:29 INFO Executor: Running task 22.0 in stage 35.0 (TID 799) +26/04/01 08:45:29 INFO TaskSetManager: Finished task 14.0 in stage 35.0 (TID 791) in 7033 ms on 10.0.0.133 (executor driver) (15/34) +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO Executor: Finished task 15.0 in stage 35.0 (TID 792). 10455 bytes result sent to driver +26/04/01 08:45:29 INFO TaskSetManager: Starting task 23.0 in stage 35.0 (TID 800) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:29 INFO Executor: Running task 23.0 in stage 35.0 (TID 800) +26/04/01 08:45:29 INFO TaskSetManager: Finished task 15.0 in stage 35.0 (TID 792) in 7050 ms on 10.0.0.133 (executor driver) (16/34) +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO Executor: Finished task 16.0 in stage 35.0 (TID 793). 9724 bytes result sent to driver +26/04/01 08:45:36 INFO TaskSetManager: Starting task 24.0 in stage 35.0 (TID 801) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:36 INFO Executor: Running task 24.0 in stage 35.0 (TID 801) +26/04/01 08:45:36 INFO TaskSetManager: Finished task 16.0 in stage 35.0 (TID 793) in 6896 ms on 10.0.0.133 (executor driver) (17/34) +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO Executor: Finished task 17.0 in stage 35.0 (TID 794). 9724 bytes result sent to driver +26/04/01 08:45:36 INFO TaskSetManager: Starting task 25.0 in stage 35.0 (TID 802) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:36 INFO TaskSetManager: Finished task 17.0 in stage 35.0 (TID 794) in 6801 ms on 10.0.0.133 (executor driver) (18/34) +26/04/01 08:45:36 INFO Executor: Running task 25.0 in stage 35.0 (TID 802) +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO Executor: Finished task 18.0 in stage 35.0 (TID 795). 9724 bytes result sent to driver +26/04/01 08:45:36 INFO TaskSetManager: Starting task 26.0 in stage 35.0 (TID 803) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:36 INFO Executor: Running task 26.0 in stage 35.0 (TID 803) +26/04/01 08:45:36 INFO TaskSetManager: Finished task 18.0 in stage 35.0 (TID 795) in 6844 ms on 10.0.0.133 (executor driver) (19/34) +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO Executor: Finished task 19.0 in stage 35.0 (TID 796). 9724 bytes result sent to driver +26/04/01 08:45:36 INFO TaskSetManager: Starting task 27.0 in stage 35.0 (TID 804) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:36 INFO TaskSetManager: Finished task 19.0 in stage 35.0 (TID 796) in 6862 ms on 10.0.0.133 (executor driver) (20/34) +26/04/01 08:45:36 INFO Executor: Running task 27.0 in stage 35.0 (TID 804) +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO Executor: Finished task 20.0 in stage 35.0 (TID 797). 9724 bytes result sent to driver +26/04/01 08:45:36 INFO TaskSetManager: Starting task 28.0 in stage 35.0 (TID 805) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:36 INFO TaskSetManager: Finished task 20.0 in stage 35.0 (TID 797) in 6876 ms on 10.0.0.133 (executor driver) (21/34) +26/04/01 08:45:36 INFO Executor: Running task 28.0 in stage 35.0 (TID 805) +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO Executor: Finished task 21.0 in stage 35.0 (TID 798). 9724 bytes result sent to driver +26/04/01 08:45:36 INFO TaskSetManager: Starting task 29.0 in stage 35.0 (TID 806) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:36 INFO TaskSetManager: Finished task 21.0 in stage 35.0 (TID 798) in 6812 ms on 10.0.0.133 (executor driver) (22/34) +26/04/01 08:45:36 INFO Executor: Running task 29.0 in stage 35.0 (TID 806) +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO Executor: Finished task 22.0 in stage 35.0 (TID 799). 9724 bytes result sent to driver +26/04/01 08:45:36 INFO TaskSetManager: Starting task 30.0 in stage 35.0 (TID 807) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:36 INFO Executor: Running task 30.0 in stage 35.0 (TID 807) +26/04/01 08:45:36 INFO TaskSetManager: Finished task 22.0 in stage 35.0 (TID 799) in 6887 ms on 10.0.0.133 (executor driver) (23/34) +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO Executor: Finished task 23.0 in stage 35.0 (TID 800). 9724 bytes result sent to driver +26/04/01 08:45:36 INFO TaskSetManager: Starting task 31.0 in stage 35.0 (TID 808) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:36 INFO Executor: Running task 31.0 in stage 35.0 (TID 808) +26/04/01 08:45:36 INFO TaskSetManager: Finished task 23.0 in stage 35.0 (TID 800) in 6910 ms on 10.0.0.133 (executor driver) (24/34) +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:43 INFO Executor: Finished task 24.0 in stage 35.0 (TID 801). 10455 bytes result sent to driver +26/04/01 08:45:43 INFO TaskSetManager: Starting task 32.0 in stage 35.0 (TID 809) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:43 INFO Executor: Running task 32.0 in stage 35.0 (TID 809) +26/04/01 08:45:43 INFO TaskSetManager: Finished task 24.0 in stage 35.0 (TID 801) in 6800 ms on 10.0.0.133 (executor driver) (25/34) +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:43 INFO Executor: Finished task 25.0 in stage 35.0 (TID 802). 10455 bytes result sent to driver +26/04/01 08:45:43 INFO TaskSetManager: Starting task 33.0 in stage 35.0 (TID 810) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9644 bytes) +26/04/01 08:45:43 INFO Executor: Running task 33.0 in stage 35.0 (TID 810) +26/04/01 08:45:43 INFO TaskSetManager: Finished task 25.0 in stage 35.0 (TID 802) in 6900 ms on 10.0.0.133 (executor driver) (26/34) +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 29 (8.1 MiB) non-empty blocks including 29 (8.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 64 (12.0 MiB) non-empty blocks including 64 (12.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 29 (8.1 MiB) non-empty blocks including 29 (8.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 64 (12.0 MiB) non-empty blocks including 64 (12.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:43 INFO Executor: Finished task 26.0 in stage 35.0 (TID 803). 10455 bytes result sent to driver +26/04/01 08:45:43 INFO TaskSetManager: Finished task 26.0 in stage 35.0 (TID 803) in 6838 ms on 10.0.0.133 (executor driver) (27/34) +26/04/01 08:45:43 INFO Executor: Finished task 27.0 in stage 35.0 (TID 804). 10455 bytes result sent to driver +26/04/01 08:45:43 INFO TaskSetManager: Finished task 27.0 in stage 35.0 (TID 804) in 6835 ms on 10.0.0.133 (executor driver) (28/34) +26/04/01 08:45:43 INFO Executor: Finished task 28.0 in stage 35.0 (TID 805). 10455 bytes result sent to driver +26/04/01 08:45:43 INFO TaskSetManager: Finished task 28.0 in stage 35.0 (TID 805) in 6896 ms on 10.0.0.133 (executor driver) (29/34) +26/04/01 08:45:43 INFO Executor: Finished task 29.0 in stage 35.0 (TID 806). 10455 bytes result sent to driver +26/04/01 08:45:43 INFO TaskSetManager: Finished task 29.0 in stage 35.0 (TID 806) in 6889 ms on 10.0.0.133 (executor driver) (30/34) +26/04/01 08:45:43 INFO Executor: Finished task 30.0 in stage 35.0 (TID 807). 10455 bytes result sent to driver +26/04/01 08:45:43 INFO TaskSetManager: Finished task 30.0 in stage 35.0 (TID 807) in 6994 ms on 10.0.0.133 (executor driver) (31/34) +26/04/01 08:45:43 INFO Executor: Finished task 31.0 in stage 35.0 (TID 808). 10455 bytes result sent to driver +26/04/01 08:45:43 INFO TaskSetManager: Finished task 31.0 in stage 35.0 (TID 808) in 6962 ms on 10.0.0.133 (executor driver) (32/34) +26/04/01 08:45:45 INFO Executor: Finished task 33.0 in stage 35.0 (TID 810). 9724 bytes result sent to driver +26/04/01 08:45:45 INFO TaskSetManager: Finished task 33.0 in stage 35.0 (TID 810) in 2160 ms on 10.0.0.133 (executor driver) (33/34) +26/04/01 08:45:49 INFO Executor: Finished task 32.0 in stage 35.0 (TID 809). 10455 bytes result sent to driver +26/04/01 08:45:49 INFO TaskSetManager: Finished task 32.0 in stage 35.0 (TID 809) in 6613 ms on 10.0.0.133 (executor driver) (34/34) +26/04/01 08:45:49 INFO TaskSchedulerImpl: Removed TaskSet 35.0, whose tasks have all completed, from pool +26/04/01 08:45:49 INFO DAGScheduler: ShuffleMapStage 35 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 34.381 s +26/04/01 08:45:49 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:45:49 INFO DAGScheduler: running: Set() +26/04/01 08:45:49 INFO DAGScheduler: waiting: Set() +26/04/01 08:45:49 INFO DAGScheduler: failed: Set() +26/04/01 08:45:49 INFO ShufflePartitionsUtil: For shuffle(8), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 08:45:49 INFO CodeGenerator: Code generated in 65.74125 ms +26/04/01 08:45:49 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:45:49 INFO DAGScheduler: Got job 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 2 output partitions +26/04/01 08:45:49 INFO DAGScheduler: Final stage: ResultStage 44 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:45:49 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 43) +26/04/01 08:45:49 INFO DAGScheduler: Missing parents: List() +26/04/01 08:45:49 INFO DAGScheduler: Submitting ResultStage 44 (MapPartitionsRDD[57] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:45:49 INFO MemoryStore: Block broadcast_27 stored as values in memory (estimated size 20.9 KiB, free 8.6 GiB) +26/04/01 08:45:49 INFO MemoryStore: Block broadcast_27_piece0 stored as bytes in memory (estimated size 8.4 KiB, free 8.6 GiB) +26/04/01 08:45:49 INFO BlockManagerInfo: Added broadcast_27_piece0 in memory on 10.0.0.133:59303 (size: 8.4 KiB, free: 8.6 GiB) +26/04/01 08:45:49 INFO SparkContext: Created broadcast 27 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:45:49 INFO DAGScheduler: Submitting 2 missing tasks from ResultStage 44 (MapPartitionsRDD[57] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1)) +26/04/01 08:45:49 INFO TaskSchedulerImpl: Adding task set 44.0 with 2 tasks resource profile 0 +26/04/01 08:45:49 INFO TaskSetManager: Starting task 0.0 in stage 44.0 (TID 811) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) +26/04/01 08:45:49 INFO TaskSetManager: Starting task 1.0 in stage 44.0 (TID 812) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9415 bytes) +26/04/01 08:45:49 INFO Executor: Running task 1.0 in stage 44.0 (TID 812) +26/04/01 08:45:49 INFO Executor: Running task 0.0 in stage 44.0 (TID 811) +26/04/01 08:45:49 INFO ShuffleBlockFetcherIterator: Getting 34 (1039.8 KiB) non-empty blocks including 34 (1039.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:49 INFO ShuffleBlockFetcherIterator: Getting 34 (1186.0 KiB) non-empty blocks including 34 (1186.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:49 INFO CodeGenerator: Code generated in 7.21075 ms +26/04/01 08:45:50 INFO Executor: Finished task 0.0 in stage 44.0 (TID 811). 8901 bytes result sent to driver +26/04/01 08:45:50 INFO TaskSetManager: Finished task 0.0 in stage 44.0 (TID 811) in 522 ms on 10.0.0.133 (executor driver) (1/2) +26/04/01 08:45:50 INFO Executor: Finished task 1.0 in stage 44.0 (TID 812). 9919 bytes result sent to driver +26/04/01 08:45:50 INFO TaskSetManager: Finished task 1.0 in stage 44.0 (TID 812) in 580 ms on 10.0.0.133 (executor driver) (2/2) +26/04/01 08:45:50 INFO TaskSchedulerImpl: Removed TaskSet 44.0, whose tasks have all completed, from pool +26/04/01 08:45:50 INFO DAGScheduler: ResultStage 44 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.582 s +26/04/01 08:45:50 INFO DAGScheduler: Job 19 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:45:50 INFO TaskSchedulerImpl: Killing all running tasks in stage 44: Stage finished +26/04/01 08:45:50 INFO DAGScheduler: Job 19 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.585016 s +26/04/01 08:45:50 INFO DAGScheduler: Registering RDD 58 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 9 +26/04/01 08:45:50 INFO DAGScheduler: Got map stage job 20 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 2 output partitions +26/04/01 08:45:50 INFO DAGScheduler: Final stage: ShuffleMapStage 53 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:45:50 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 52) +26/04/01 08:45:50 INFO DAGScheduler: Missing parents: List() +26/04/01 08:45:50 INFO DAGScheduler: Submitting ShuffleMapStage 53 (MapPartitionsRDD[58] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:45:50 INFO MemoryStore: Block broadcast_28 stored as values in memory (estimated size 31.4 KiB, free 8.6 GiB) +26/04/01 08:45:50 INFO MemoryStore: Block broadcast_28_piece0 stored as bytes in memory (estimated size 10.6 KiB, free 8.6 GiB) +26/04/01 08:45:50 INFO BlockManagerInfo: Added broadcast_28_piece0 in memory on 10.0.0.133:59303 (size: 10.6 KiB, free: 8.6 GiB) +26/04/01 08:45:50 INFO SparkContext: Created broadcast 28 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:45:50 INFO DAGScheduler: Submitting 2 missing tasks from ShuffleMapStage 53 (MapPartitionsRDD[58] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1)) +26/04/01 08:45:50 INFO TaskSchedulerImpl: Adding task set 53.0 with 2 tasks resource profile 0 +26/04/01 08:45:50 INFO TaskSetManager: Starting task 0.0 in stage 53.0 (TID 813) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9404 bytes) +26/04/01 08:45:50 INFO TaskSetManager: Starting task 1.0 in stage 53.0 (TID 814) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9404 bytes) +26/04/01 08:45:50 INFO Executor: Running task 0.0 in stage 53.0 (TID 813) +26/04/01 08:45:50 INFO Executor: Running task 1.0 in stage 53.0 (TID 814) +26/04/01 08:45:50 INFO CodeGenerator: Code generated in 3.566542 ms +26/04/01 08:45:50 INFO ShuffleBlockFetcherIterator: Getting 34 (1186.0 KiB) non-empty blocks including 34 (1186.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:50 INFO ShuffleBlockFetcherIterator: Getting 34 (1039.8 KiB) non-empty blocks including 34 (1039.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:50 INFO Executor: Finished task 0.0 in stage 53.0 (TID 813). 6566 bytes result sent to driver +26/04/01 08:45:50 INFO TaskSetManager: Finished task 0.0 in stage 53.0 (TID 813) in 491 ms on 10.0.0.133 (executor driver) (1/2) +26/04/01 08:45:50 INFO Executor: Finished task 1.0 in stage 53.0 (TID 814). 6566 bytes result sent to driver +26/04/01 08:45:50 INFO TaskSetManager: Finished task 1.0 in stage 53.0 (TID 814) in 555 ms on 10.0.0.133 (executor driver) (2/2) +26/04/01 08:45:50 INFO TaskSchedulerImpl: Removed TaskSet 53.0, whose tasks have all completed, from pool +26/04/01 08:45:50 INFO DAGScheduler: ShuffleMapStage 53 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.558 s +26/04/01 08:45:50 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:45:50 INFO DAGScheduler: running: Set() +26/04/01 08:45:50 INFO DAGScheduler: waiting: Set() +26/04/01 08:45:50 INFO DAGScheduler: failed: Set() +26/04/01 08:45:50 INFO ShufflePartitionsUtil: For shuffle(9), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 08:45:50 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:45:50 INFO DAGScheduler: Got job 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 08:45:50 INFO DAGScheduler: Final stage: ResultStage 63 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:45:50 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 62) +26/04/01 08:45:50 INFO DAGScheduler: Missing parents: List() +26/04/01 08:45:50 INFO DAGScheduler: Submitting ResultStage 63 (MapPartitionsRDD[62] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:45:50 INFO MemoryStore: Block broadcast_29 stored as values in memory (estimated size 28.6 KiB, free 8.6 GiB) +26/04/01 08:45:50 INFO MemoryStore: Block broadcast_29_piece0 stored as bytes in memory (estimated size 10.4 KiB, free 8.6 GiB) +26/04/01 08:45:50 INFO BlockManagerInfo: Added broadcast_29_piece0 in memory on 10.0.0.133:59303 (size: 10.4 KiB, free: 8.6 GiB) +26/04/01 08:45:50 INFO SparkContext: Created broadcast 29 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:45:50 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 63 (MapPartitionsRDD[62] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:45:50 INFO TaskSchedulerImpl: Adding task set 63.0 with 1 tasks resource profile 0 +26/04/01 08:45:50 INFO TaskSetManager: Starting task 0.0 in stage 63.0 (TID 815) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) +26/04/01 08:45:50 INFO Executor: Running task 0.0 in stage 63.0 (TID 815) +26/04/01 08:45:50 INFO ShuffleBlockFetcherIterator: Getting 2 (76.4 KiB) non-empty blocks including 2 (76.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:45:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:45:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB +26/04/01 08:45:50 INFO Executor: Finished task 0.0 in stage 63.0 (TID 815). 8247 bytes result sent to driver +26/04/01 08:45:50 INFO TaskSetManager: Finished task 0.0 in stage 63.0 (TID 815) in 47 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:45:50 INFO TaskSchedulerImpl: Removed TaskSet 63.0, whose tasks have all completed, from pool +26/04/01 08:45:50 INFO DAGScheduler: ResultStage 63 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.048 s +26/04/01 08:45:50 INFO DAGScheduler: Job 21 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:45:50 INFO TaskSchedulerImpl: Killing all running tasks in stage 63: Stage finished +26/04/01 08:45:50 INFO DAGScheduler: Job 21 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.049918 s +26/04/01 08:45:50 INFO SparkContext: SparkContext is stopping with exitCode 0. +26/04/01 08:45:50 INFO CometDriverPlugin: CometDriverPlugin shutdown +26/04/01 08:45:50 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! +26/04/01 08:45:53 INFO MemoryStore: MemoryStore cleared +26/04/01 08:45:53 INFO BlockManager: BlockManager stopped +26/04/01 08:45:53 INFO BlockManagerMaster: BlockManagerMaster stopped +26/04/01 08:45:53 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! +26/04/01 08:45:53 INFO SparkContext: Successfully stopped SparkContext +26/04/01 08:45:53 INFO ShutdownHookManager: Shutdown hook called +26/04/01 08:45:53 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-cae9fc0e-ad8f-4143-bb25-84be77849f6a +26/04/01 08:45:54 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-3fd37116-6cd4-4238-bd4c-70dc836beec5 +26/04/01 08:45:54 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-cae9fc0e-ad8f-4143-bb25-84be77849f6a/pyspark-f8d215d3-6f69-482a-bccc-77152a4751f3 + 384.09 real 3098.02 user 34.23 sys + 8939782144 maximum resident set size + 0 average shared memory size + 0 average unshared data size + 0 average unshared stack size + 626326 page reclaims + 80 page faults + 0 swaps + 0 block input operations + 0 block output operations + 1278 messages sent + 2365 messages received + 37 signals received + 109370 voluntary context switches + 938768 involuntary context switches + 43620594854966 instructions retired + 11293084515693 cycles elapsed + 3963704144 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.log b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.log new file mode 100644 index 0000000000..1779ac4349 --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.log @@ -0,0 +1,76 @@ +Registering table customer from /opt/tpch/sf100/customer +Registering table lineitem from /opt/tpch/sf100/lineitem +Registering table nation from /opt/tpch/sf100/nation +Registering table orders from /opt/tpch/sf100/orders +Registering table part from /opt/tpch/sf100/part +Registering table partsupp from /opt/tpch/sf100/partsupp +Registering table region from /opt/tpch/sf100/region +Registering table supplier from /opt/tpch/sf100/supplier + +============================================================ +Starting iteration 1 of 1 +============================================================ + +Running query 1 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q1.sql +Executing: -- CometBench-H query 1 derived from TPC-H query 1 under the terms of the TPC Fair Use Policy. +-- TP... +== Physical Plan == +AdaptiveSparkPlan (9) ++- CometSort (8) + +- CometExchange (7) + +- CometHashAggregate (6) + +- CometExchange (5) + +- CometHashAggregate (4) + +- CometProject (3) + +- CometFilter (2) + +- CometNativeScan parquet (1) + + +(1) CometNativeScan parquet +Output [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] +PushedFilters: [IsNotNull(l_shipdate), LessThanOrEqual(l_shipdate,1998-09-24)] +ReadSchema: struct + +(2) CometFilter +Input [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] +Condition : (isnotnull(l_shipdate#26) AND (l_shipdate#26 <= 1998-09-24)) + +(3) CometProject +Input [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] +Arguments: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25], [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25] + +(4) CometHashAggregate +Input [6]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25] +Keys [2]: [l_returnflag#24, l_linestatus#25] +Functions [8]: [partial_sum(l_quantity#20), partial_sum(l_extendedprice#21), partial_sum((l_extendedprice#21 * (1 - l_discount#22))), partial_sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23))), partial_avg(l_quantity#20), partial_avg(l_extendedprice#21), partial_avg(l_discount#22), partial_count(1)] + +(5) CometExchange +Input [17]: [l_returnflag#24, l_linestatus#25, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, count#172L, sum#173, count#174L, sum#175, count#176L, count#177L] +Arguments: hashpartitioning(l_returnflag#24, l_linestatus#25, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=83] + +(6) CometHashAggregate +Input [17]: [l_returnflag#24, l_linestatus#25, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, count#172L, sum#173, count#174L, sum#175, count#176L, count#177L] +Keys [2]: [l_returnflag#24, l_linestatus#25] +Functions [8]: [sum(l_quantity#20), sum(l_extendedprice#21), sum((l_extendedprice#21 * (1 - l_discount#22))), sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23))), avg(l_quantity#20), avg(l_extendedprice#21), avg(l_discount#22), count(1)] + +(7) CometExchange +Input [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] +Arguments: rangepartitioning(l_returnflag#24 ASC NULLS FIRST, l_linestatus#25 ASC NULLS FIRST, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=85] + +(8) CometSort +Input [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] +Arguments: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L], [l_returnflag#24 ASC NULLS FIRST, l_linestatus#25 ASC NULLS FIRST] + +(9) AdaptiveSparkPlan +Output [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] +Arguments: isFinalPlan=false + + +Query 1 returned 4 rows, hash=6839cc802df6220ed2efdaaaa48373a6 +Query 1 took 330.29 seconds + +Iteration 1 took 330.29 seconds + +Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/comet-offheap8g-q1-tpch-1775046669770.json diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.time b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.time new file mode 100644 index 0000000000..29c47eddbd --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.time @@ -0,0 +1,1629 @@ +26/04/01 06:25:34 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) +26/04/01 06:25:34 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address +26/04/01 06:25:34 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +26/04/01 06:25:35 INFO SparkContext: Running Spark version 3.5.8 +26/04/01 06:25:35 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 06:25:35 INFO SparkContext: Java version 17.0.17 +26/04/01 06:25:35 INFO ResourceUtils: ============================================================== +26/04/01 06:25:35 INFO ResourceUtils: No custom resources configured for spark.driver. +26/04/01 06:25:35 INFO ResourceUtils: ============================================================== +26/04/01 06:25:35 INFO SparkContext: Submitted application: comet-offheap8g-q1 benchmark derived from tpch +26/04/01 06:25:35 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 8192, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) +26/04/01 06:25:35 INFO ResourceProfile: Limiting resource is cpu +26/04/01 06:25:35 INFO ResourceProfileManager: Added ResourceProfile id: 0 +26/04/01 06:25:35 INFO SecurityManager: Changing view acls to: andy +26/04/01 06:25:35 INFO SecurityManager: Changing modify acls to: andy +26/04/01 06:25:35 INFO SecurityManager: Changing view acls groups to: +26/04/01 06:25:35 INFO SecurityManager: Changing modify acls groups to: +26/04/01 06:25:35 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY +26/04/01 06:25:35 INFO Utils: Successfully started service 'sparkDriver' on port 58416. +26/04/01 06:25:35 INFO SparkEnv: Registering MapOutputTracker +26/04/01 06:25:35 INFO SparkEnv: Registering BlockManagerMaster +26/04/01 06:25:35 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information +26/04/01 06:25:35 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up +26/04/01 06:25:35 INFO SparkEnv: Registering BlockManagerMasterHeartbeat +26/04/01 06:25:35 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-c231e1ae-1a96-48c7-b30d-283875835f41 +26/04/01 06:25:35 INFO MemoryStore: MemoryStore started with capacity 12.6 GiB +26/04/01 06:25:35 INFO SparkEnv: Registering OutputCommitCoordinator +26/04/01 06:25:35 INFO SparkContext: Added JAR file:///Users/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar at spark://10.0.0.133:58416/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775046335237 +26/04/01 06:25:35 INFO CometDriverPlugin: CometDriverPlugin init +26/04/01 06:25:35 INFO CometDriverPlugin: Setting spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions +26/04/01 06:25:35 INFO CometDriverPlugin: Comet metrics reporting is disabled. Set spark.comet.metrics.enabled=true to enable. +26/04/01 06:25:35 INFO CometDriverPlugin: Comet is running in unified memory mode and sharing off-heap memory with Spark +26/04/01 06:25:35 INFO DriverPluginContainer: Initialized driver component for plugin org.apache.spark.CometPlugin. +26/04/01 06:25:35 INFO Executor: Starting executor ID driver on host 10.0.0.133 +26/04/01 06:25:35 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 06:25:35 INFO Executor: Java version 17.0.17 +26/04/01 06:25:35 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' +26/04/01 06:25:35 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@17670b56 for default. +26/04/01 06:25:35 INFO Executor: Fetching spark://10.0.0.133:58416/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775046335237 +26/04/01 06:25:35 INFO TransportClientFactory: Successfully created connection to /10.0.0.133:58416 after 8 ms (0 ms spent in bootstraps) +26/04/01 06:25:35 INFO Utils: Fetching spark://10.0.0.133:58416/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-9d0b1c01-f2bb-4fb8-875a-59c49514dd0e/userFiles-79baa501-cc0c-488c-9287-530627dc7d52/fetchFileTemp4495643812795777425.tmp +26/04/01 06:25:35 INFO Executor: Adding file:/private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-9d0b1c01-f2bb-4fb8-875a-59c49514dd0e/userFiles-79baa501-cc0c-488c-9287-530627dc7d52/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to class loader default +26/04/01 06:25:35 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 58418. +26/04/01 06:25:35 INFO NettyBlockTransferService: Server created on 10.0.0.133:58418 +26/04/01 06:25:35 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy +26/04/01 06:25:35 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 58418, None) +26/04/01 06:25:35 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:58418 with 12.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 58418, None) +26/04/01 06:25:35 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 58418, None) +26/04/01 06:25:35 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 58418, None) +26/04/01 06:25:35 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. +26/04/01 06:25:35 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. +26/04/01 06:25:35 INFO InMemoryFileIndex: It took 16 ms to list leaf files for 1 paths. +26/04/01 06:25:36 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:25:36 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:25:36 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:25:36 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:25:36 INFO DAGScheduler: Missing parents: List() +26/04/01 06:25:36 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:25:36 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:25:36 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) +26/04/01 06:25:36 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:58418 (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:25:36 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:25:36 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:25:36 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 +26/04/01 06:25:36 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 06:25:36 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) +26/04/01 06:25:36 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 2041 bytes result sent to driver +26/04/01 06:25:36 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 118 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:25:36 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool +26/04/01 06:25:36 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.371 s +26/04/01 06:25:36 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:25:36 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished +26/04/01 06:25:36 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.384166 s +26/04/01 06:25:36 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:58418 in memory (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO NativeBase: Couldn't locate log file from either COMET_CONF_DIR or comet.log.file.path. Using default log configuration with INFO log level which emits to stderr +26/04/01 06:25:39 INFO core/src/lib.rs: Comet native library version 0.15.0 initialized +26/04/01 06:25:39 INFO NativeBase: Setting system property arrow.enable_unsafe_memory_access to true +26/04/01 06:25:39 INFO NativeBase: Setting system property arrow.enable_null_check_for_get to false +26/04/01 06:25:39 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. +26/04/01 06:25:39 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:25:39 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:25:39 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() +26/04/01 06:25:39 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:58418 (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:25:39 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 +26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) +26/04/01 06:25:39 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2430 bytes result sent to driver +26/04/01 06:25:39 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 17 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:25:39 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool +26/04/01 06:25:39 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.026 s +26/04/01 06:25:39 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:25:39 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished +26/04/01 06:25:39 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.027665 s +26/04/01 06:25:39 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 06:25:39 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:25:39 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:25:39 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() +26/04/01 06:25:39 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:58418 (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:25:39 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 +26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) +26/04/01 06:25:39 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver +26/04/01 06:25:39 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:25:39 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool +26/04/01 06:25:39 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s +26/04/01 06:25:39 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:25:39 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished +26/04/01 06:25:39 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.016430 s +26/04/01 06:25:39 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. +26/04/01 06:25:39 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:25:39 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:25:39 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() +26/04/01 06:25:39 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:58418 (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:25:39 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 +26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) +26/04/01 06:25:39 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver +26/04/01 06:25:39 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:25:39 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool +26/04/01 06:25:39 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s +26/04/01 06:25:39 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:25:39 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished +26/04/01 06:25:39 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.015428 s +26/04/01 06:25:39 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. +26/04/01 06:25:39 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:25:39 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:25:39 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() +26/04/01 06:25:39 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:58418 (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:58418 in memory (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:25:39 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 +26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9299 bytes) +26/04/01 06:25:39 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:58418 in memory (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) +26/04/01 06:25:39 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:58418 in memory (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver +26/04/01 06:25:39 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 7 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:25:39 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool +26/04/01 06:25:39 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.016 s +26/04/01 06:25:39 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:25:39 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished +26/04/01 06:25:39 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.017624 s +26/04/01 06:25:39 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 06:25:39 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:25:39 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:25:39 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() +26/04/01 06:25:39 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:58418 (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:58418 in memory (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:25:39 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 +26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) +26/04/01 06:25:39 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver +26/04/01 06:25:39 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:25:39 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool +26/04/01 06:25:39 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s +26/04/01 06:25:39 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:25:39 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished +26/04/01 06:25:39 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.015451 s +26/04/01 06:25:39 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. +26/04/01 06:25:39 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:25:39 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:25:39 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() +26/04/01 06:25:39 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:58418 in memory (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:58418 (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:25:39 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 +26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) +26/04/01 06:25:39 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver +26/04/01 06:25:39 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:25:39 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool +26/04/01 06:25:39 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 06:25:39 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:25:39 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished +26/04/01 06:25:39 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.014321 s +26/04/01 06:25:39 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 06:25:39 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:25:39 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:25:39 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() +26/04/01 06:25:39 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:58418 (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:25:39 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:58418 in memory (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 +26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) +26/04/01 06:25:39 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver +26/04/01 06:25:39 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:25:39 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool +26/04/01 06:25:39 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s +26/04/01 06:25:39 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:25:39 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished +26/04/01 06:25:39 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.015871 s +26/04/01 06:25:39 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_shipdate),LessThanOrEqual(l_shipdate,1998-09-24) +26/04/01 06:25:39 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_shipdate#26),(l_shipdate#26 <= 1998-09-24) +26/04/01 06:25:39 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:58418 in memory (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:25:39 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:58418 (size: 35.4 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:25:39 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:25:39 INFO DAGScheduler: Registering RDD 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 +26/04/01 06:25:39 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions +26/04/01 06:25:39 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() +26/04/01 06:25:39 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 46.8 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 10.1 KiB, free 12.6 GiB) +26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:58418 (size: 10.1 KiB, free: 12.6 GiB) +26/04/01 06:25:39 INFO SparkContext: Created broadcast 9 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:25:39 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 8.0 with 208 tasks resource profile 0 +26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:25:39 INFO TaskSetManager: Starting task 1.0 in stage 8.0 (TID 9) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:25:39 INFO TaskSetManager: Starting task 2.0 in stage 8.0 (TID 10) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:25:39 INFO TaskSetManager: Starting task 3.0 in stage 8.0 (TID 11) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) +26/04/01 06:25:39 INFO Executor: Running task 1.0 in stage 8.0 (TID 9) +26/04/01 06:25:39 INFO Executor: Running task 3.0 in stage 8.0 (TID 11) +26/04/01 06:25:39 INFO Executor: Running task 2.0 in stage 8.0 (TID 10) +26/04/01 06:25:39 INFO BaseAllocator: Debug mode disabled. Enable with the VM option -Darrow.memory.debug.allocator=true. +26/04/01 06:25:39 INFO DefaultAllocationManagerOption: allocation manager type not specified, using netty as the default type +26/04/01 06:25:39 INFO CheckAllocator: Using DefaultAllocationManager at memory/unsafe/DefaultAllocationManagerFactory.class +26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:39 INFO core/src/execution/jni_api.rs: Comet tokio runtime: using spark.executor.cores=4 worker threads +26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:46 INFO Executor: Finished task 1.0 in stage 8.0 (TID 9). 6830 bytes result sent to driver +26/04/01 06:25:46 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 6830 bytes result sent to driver +26/04/01 06:25:46 INFO Executor: Finished task 3.0 in stage 8.0 (TID 11). 6830 bytes result sent to driver +26/04/01 06:25:46 INFO Executor: Finished task 2.0 in stage 8.0 (TID 10). 6830 bytes result sent to driver +26/04/01 06:25:46 INFO TaskSetManager: Starting task 4.0 in stage 8.0 (TID 12) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:25:46 INFO Executor: Running task 4.0 in stage 8.0 (TID 12) +26/04/01 06:25:46 INFO TaskSetManager: Starting task 5.0 in stage 8.0 (TID 13) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:25:46 INFO Executor: Running task 5.0 in stage 8.0 (TID 13) +26/04/01 06:25:46 INFO TaskSetManager: Starting task 6.0 in stage 8.0 (TID 14) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:25:46 INFO TaskSetManager: Finished task 3.0 in stage 8.0 (TID 11) in 6895 ms on 10.0.0.133 (executor driver) (1/208) +26/04/01 06:25:46 INFO Executor: Running task 6.0 in stage 8.0 (TID 14) +26/04/01 06:25:46 INFO TaskSetManager: Starting task 7.0 in stage 8.0 (TID 15) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:25:46 INFO Executor: Running task 7.0 in stage 8.0 (TID 15) +26/04/01 06:25:46 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 6898 ms on 10.0.0.133 (executor driver) (2/208) +26/04/01 06:25:46 INFO TaskSetManager: Finished task 1.0 in stage 8.0 (TID 9) in 6898 ms on 10.0.0.133 (executor driver) (3/208) +26/04/01 06:25:46 INFO TaskSetManager: Finished task 2.0 in stage 8.0 (TID 10) in 6898 ms on 10.0.0.133 (executor driver) (4/208) +26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:53 INFO Executor: Finished task 7.0 in stage 8.0 (TID 15). 6830 bytes result sent to driver +26/04/01 06:25:53 INFO TaskSetManager: Starting task 8.0 in stage 8.0 (TID 16) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:25:53 INFO Executor: Finished task 5.0 in stage 8.0 (TID 13). 6830 bytes result sent to driver +26/04/01 06:25:53 INFO Executor: Running task 8.0 in stage 8.0 (TID 16) +26/04/01 06:25:53 INFO TaskSetManager: Starting task 9.0 in stage 8.0 (TID 17) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:25:53 INFO TaskSetManager: Finished task 7.0 in stage 8.0 (TID 15) in 6548 ms on 10.0.0.133 (executor driver) (5/208) +26/04/01 06:25:53 INFO Executor: Running task 9.0 in stage 8.0 (TID 17) +26/04/01 06:25:53 INFO TaskSetManager: Finished task 5.0 in stage 8.0 (TID 13) in 6549 ms on 10.0.0.133 (executor driver) (6/208) +26/04/01 06:25:53 INFO Executor: Finished task 4.0 in stage 8.0 (TID 12). 6830 bytes result sent to driver +26/04/01 06:25:53 INFO TaskSetManager: Starting task 10.0 in stage 8.0 (TID 18) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:25:53 INFO Executor: Running task 10.0 in stage 8.0 (TID 18) +26/04/01 06:25:53 INFO Executor: Finished task 6.0 in stage 8.0 (TID 14). 6830 bytes result sent to driver +26/04/01 06:25:53 INFO TaskSetManager: Finished task 4.0 in stage 8.0 (TID 12) in 6552 ms on 10.0.0.133 (executor driver) (7/208) +26/04/01 06:25:53 INFO TaskSetManager: Starting task 11.0 in stage 8.0 (TID 19) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:25:53 INFO Executor: Running task 11.0 in stage 8.0 (TID 19) +26/04/01 06:25:53 INFO TaskSetManager: Finished task 6.0 in stage 8.0 (TID 14) in 6552 ms on 10.0.0.133 (executor driver) (8/208) +26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:59 INFO Executor: Finished task 9.0 in stage 8.0 (TID 17). 6830 bytes result sent to driver +26/04/01 06:25:59 INFO TaskSetManager: Starting task 12.0 in stage 8.0 (TID 20) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:25:59 INFO Executor: Finished task 11.0 in stage 8.0 (TID 19). 6830 bytes result sent to driver +26/04/01 06:25:59 INFO Executor: Running task 12.0 in stage 8.0 (TID 20) +26/04/01 06:25:59 INFO TaskSetManager: Starting task 13.0 in stage 8.0 (TID 21) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:25:59 INFO Executor: Running task 13.0 in stage 8.0 (TID 21) +26/04/01 06:25:59 INFO Executor: Finished task 8.0 in stage 8.0 (TID 16). 6830 bytes result sent to driver +26/04/01 06:25:59 INFO TaskSetManager: Finished task 9.0 in stage 8.0 (TID 17) in 6542 ms on 10.0.0.133 (executor driver) (9/208) +26/04/01 06:25:59 INFO TaskSetManager: Finished task 11.0 in stage 8.0 (TID 19) in 6540 ms on 10.0.0.133 (executor driver) (10/208) +26/04/01 06:25:59 INFO TaskSetManager: Starting task 14.0 in stage 8.0 (TID 22) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:25:59 INFO Executor: Running task 14.0 in stage 8.0 (TID 22) +26/04/01 06:25:59 INFO TaskSetManager: Finished task 8.0 in stage 8.0 (TID 16) in 6543 ms on 10.0.0.133 (executor driver) (11/208) +26/04/01 06:25:59 INFO Executor: Finished task 10.0 in stage 8.0 (TID 18). 6830 bytes result sent to driver +26/04/01 06:25:59 INFO TaskSetManager: Starting task 15.0 in stage 8.0 (TID 23) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:25:59 INFO Executor: Running task 15.0 in stage 8.0 (TID 23) +26/04/01 06:25:59 INFO TaskSetManager: Finished task 10.0 in stage 8.0 (TID 18) in 6542 ms on 10.0.0.133 (executor driver) (12/208) +26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:06 INFO Executor: Finished task 14.0 in stage 8.0 (TID 22). 6787 bytes result sent to driver +26/04/01 06:26:06 INFO TaskSetManager: Starting task 16.0 in stage 8.0 (TID 24) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:06 INFO Executor: Running task 16.0 in stage 8.0 (TID 24) +26/04/01 06:26:06 INFO TaskSetManager: Finished task 14.0 in stage 8.0 (TID 22) in 6550 ms on 10.0.0.133 (executor driver) (13/208) +26/04/01 06:26:06 INFO Executor: Finished task 15.0 in stage 8.0 (TID 23). 6787 bytes result sent to driver +26/04/01 06:26:06 INFO TaskSetManager: Starting task 17.0 in stage 8.0 (TID 25) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:06 INFO Executor: Running task 17.0 in stage 8.0 (TID 25) +26/04/01 06:26:06 INFO TaskSetManager: Finished task 15.0 in stage 8.0 (TID 23) in 6551 ms on 10.0.0.133 (executor driver) (14/208) +26/04/01 06:26:06 INFO Executor: Finished task 13.0 in stage 8.0 (TID 21). 6830 bytes result sent to driver +26/04/01 06:26:06 INFO TaskSetManager: Starting task 18.0 in stage 8.0 (TID 26) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:06 INFO TaskSetManager: Finished task 13.0 in stage 8.0 (TID 21) in 6554 ms on 10.0.0.133 (executor driver) (15/208) +26/04/01 06:26:06 INFO Executor: Running task 18.0 in stage 8.0 (TID 26) +26/04/01 06:26:06 INFO Executor: Finished task 12.0 in stage 8.0 (TID 20). 6830 bytes result sent to driver +26/04/01 06:26:06 INFO TaskSetManager: Starting task 19.0 in stage 8.0 (TID 27) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:26:06 INFO Executor: Running task 19.0 in stage 8.0 (TID 27) +26/04/01 06:26:06 INFO TaskSetManager: Finished task 12.0 in stage 8.0 (TID 20) in 6554 ms on 10.0.0.133 (executor driver) (16/208) +26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:12 INFO Executor: Finished task 18.0 in stage 8.0 (TID 26). 6787 bytes result sent to driver +26/04/01 06:26:12 INFO Executor: Finished task 19.0 in stage 8.0 (TID 27). 6787 bytes result sent to driver +26/04/01 06:26:12 INFO Executor: Finished task 17.0 in stage 8.0 (TID 25). 6830 bytes result sent to driver +26/04/01 06:26:12 INFO TaskSetManager: Starting task 20.0 in stage 8.0 (TID 28) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:26:12 INFO Executor: Finished task 16.0 in stage 8.0 (TID 24). 6787 bytes result sent to driver +26/04/01 06:26:12 INFO Executor: Running task 20.0 in stage 8.0 (TID 28) +26/04/01 06:26:12 INFO TaskSetManager: Starting task 21.0 in stage 8.0 (TID 29) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:26:12 INFO TaskSetManager: Finished task 18.0 in stage 8.0 (TID 26) in 6547 ms on 10.0.0.133 (executor driver) (17/208) +26/04/01 06:26:12 INFO Executor: Running task 21.0 in stage 8.0 (TID 29) +26/04/01 06:26:12 INFO TaskSetManager: Finished task 19.0 in stage 8.0 (TID 27) in 6546 ms on 10.0.0.133 (executor driver) (18/208) +26/04/01 06:26:12 INFO TaskSetManager: Starting task 22.0 in stage 8.0 (TID 30) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:26:12 INFO TaskSetManager: Starting task 23.0 in stage 8.0 (TID 31) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:26:12 INFO Executor: Running task 22.0 in stage 8.0 (TID 30) +26/04/01 06:26:12 INFO TaskSetManager: Finished task 17.0 in stage 8.0 (TID 25) in 6547 ms on 10.0.0.133 (executor driver) (19/208) +26/04/01 06:26:12 INFO Executor: Running task 23.0 in stage 8.0 (TID 31) +26/04/01 06:26:12 INFO TaskSetManager: Finished task 16.0 in stage 8.0 (TID 24) in 6550 ms on 10.0.0.133 (executor driver) (20/208) +26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:19 INFO Executor: Finished task 23.0 in stage 8.0 (TID 31). 6830 bytes result sent to driver +26/04/01 06:26:19 INFO Executor: Finished task 22.0 in stage 8.0 (TID 30). 6830 bytes result sent to driver +26/04/01 06:26:19 INFO TaskSetManager: Starting task 24.0 in stage 8.0 (TID 32) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:26:19 INFO Executor: Finished task 21.0 in stage 8.0 (TID 29). 6830 bytes result sent to driver +26/04/01 06:26:19 INFO TaskSetManager: Starting task 25.0 in stage 8.0 (TID 33) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:26:19 INFO Executor: Running task 24.0 in stage 8.0 (TID 32) +26/04/01 06:26:19 INFO TaskSetManager: Finished task 23.0 in stage 8.0 (TID 31) in 6530 ms on 10.0.0.133 (executor driver) (21/208) +26/04/01 06:26:19 INFO Executor: Running task 25.0 in stage 8.0 (TID 33) +26/04/01 06:26:19 INFO TaskSetManager: Finished task 22.0 in stage 8.0 (TID 30) in 6531 ms on 10.0.0.133 (executor driver) (22/208) +26/04/01 06:26:19 INFO TaskSetManager: Starting task 26.0 in stage 8.0 (TID 34) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:19 INFO TaskSetManager: Finished task 21.0 in stage 8.0 (TID 29) in 6532 ms on 10.0.0.133 (executor driver) (23/208) +26/04/01 06:26:19 INFO Executor: Running task 26.0 in stage 8.0 (TID 34) +26/04/01 06:26:19 INFO Executor: Finished task 20.0 in stage 8.0 (TID 28). 6830 bytes result sent to driver +26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:19 INFO TaskSetManager: Starting task 27.0 in stage 8.0 (TID 35) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:19 INFO Executor: Running task 27.0 in stage 8.0 (TID 35) +26/04/01 06:26:19 INFO TaskSetManager: Finished task 20.0 in stage 8.0 (TID 28) in 6534 ms on 10.0.0.133 (executor driver) (24/208) +26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:25 INFO Executor: Finished task 25.0 in stage 8.0 (TID 33). 6873 bytes result sent to driver +26/04/01 06:26:25 INFO Executor: Finished task 27.0 in stage 8.0 (TID 35). 6873 bytes result sent to driver +26/04/01 06:26:25 INFO TaskSetManager: Starting task 28.0 in stage 8.0 (TID 36) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:25 INFO TaskSetManager: Starting task 29.0 in stage 8.0 (TID 37) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:25 INFO Executor: Running task 28.0 in stage 8.0 (TID 36) +26/04/01 06:26:25 INFO Executor: Running task 29.0 in stage 8.0 (TID 37) +26/04/01 06:26:25 INFO TaskSetManager: Finished task 25.0 in stage 8.0 (TID 33) in 6463 ms on 10.0.0.133 (executor driver) (25/208) +26/04/01 06:26:25 INFO TaskSetManager: Finished task 27.0 in stage 8.0 (TID 35) in 6461 ms on 10.0.0.133 (executor driver) (26/208) +26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:25 INFO Executor: Finished task 24.0 in stage 8.0 (TID 32). 6830 bytes result sent to driver +26/04/01 06:26:25 INFO Executor: Finished task 26.0 in stage 8.0 (TID 34). 6830 bytes result sent to driver +26/04/01 06:26:25 INFO TaskSetManager: Starting task 30.0 in stage 8.0 (TID 38) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:26:25 INFO Executor: Running task 30.0 in stage 8.0 (TID 38) +26/04/01 06:26:25 INFO TaskSetManager: Starting task 31.0 in stage 8.0 (TID 39) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:26:25 INFO TaskSetManager: Finished task 24.0 in stage 8.0 (TID 32) in 6469 ms on 10.0.0.133 (executor driver) (27/208) +26/04/01 06:26:25 INFO Executor: Running task 31.0 in stage 8.0 (TID 39) +26/04/01 06:26:25 INFO TaskSetManager: Finished task 26.0 in stage 8.0 (TID 34) in 6468 ms on 10.0.0.133 (executor driver) (28/208) +26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:32 INFO Executor: Finished task 28.0 in stage 8.0 (TID 36). 6787 bytes result sent to driver +26/04/01 06:26:32 INFO Executor: Finished task 30.0 in stage 8.0 (TID 38). 6787 bytes result sent to driver +26/04/01 06:26:32 INFO TaskSetManager: Starting task 32.0 in stage 8.0 (TID 40) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:32 INFO TaskSetManager: Starting task 33.0 in stage 8.0 (TID 41) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:32 INFO Executor: Running task 32.0 in stage 8.0 (TID 40) +26/04/01 06:26:32 INFO TaskSetManager: Finished task 28.0 in stage 8.0 (TID 36) in 6475 ms on 10.0.0.133 (executor driver) (29/208) +26/04/01 06:26:32 INFO Executor: Running task 33.0 in stage 8.0 (TID 41) +26/04/01 06:26:32 INFO TaskSetManager: Finished task 30.0 in stage 8.0 (TID 38) in 6470 ms on 10.0.0.133 (executor driver) (30/208) +26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:32 INFO Executor: Finished task 29.0 in stage 8.0 (TID 37). 6787 bytes result sent to driver +26/04/01 06:26:32 INFO TaskSetManager: Starting task 34.0 in stage 8.0 (TID 42) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:32 INFO TaskSetManager: Finished task 29.0 in stage 8.0 (TID 37) in 6485 ms on 10.0.0.133 (executor driver) (31/208) +26/04/01 06:26:32 INFO Executor: Running task 34.0 in stage 8.0 (TID 42) +26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:32 INFO Executor: Finished task 31.0 in stage 8.0 (TID 39). 6787 bytes result sent to driver +26/04/01 06:26:32 INFO TaskSetManager: Starting task 35.0 in stage 8.0 (TID 43) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:32 INFO TaskSetManager: Finished task 31.0 in stage 8.0 (TID 39) in 6484 ms on 10.0.0.133 (executor driver) (32/208) +26/04/01 06:26:32 INFO Executor: Running task 35.0 in stage 8.0 (TID 43) +26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:38 INFO Executor: Finished task 32.0 in stage 8.0 (TID 40). 6787 bytes result sent to driver +26/04/01 06:26:38 INFO Executor: Finished task 33.0 in stage 8.0 (TID 41). 6787 bytes result sent to driver +26/04/01 06:26:38 INFO TaskSetManager: Starting task 36.0 in stage 8.0 (TID 44) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:26:38 INFO Executor: Running task 36.0 in stage 8.0 (TID 44) +26/04/01 06:26:38 INFO TaskSetManager: Starting task 37.0 in stage 8.0 (TID 45) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:26:38 INFO TaskSetManager: Finished task 32.0 in stage 8.0 (TID 40) in 6461 ms on 10.0.0.133 (executor driver) (33/208) +26/04/01 06:26:38 INFO Executor: Running task 37.0 in stage 8.0 (TID 45) +26/04/01 06:26:38 INFO TaskSetManager: Finished task 33.0 in stage 8.0 (TID 41) in 6461 ms on 10.0.0.133 (executor driver) (34/208) +26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:38 INFO Executor: Finished task 34.0 in stage 8.0 (TID 42). 6787 bytes result sent to driver +26/04/01 06:26:38 INFO TaskSetManager: Starting task 38.0 in stage 8.0 (TID 46) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:38 INFO Executor: Running task 38.0 in stage 8.0 (TID 46) +26/04/01 06:26:38 INFO TaskSetManager: Finished task 34.0 in stage 8.0 (TID 42) in 6457 ms on 10.0.0.133 (executor driver) (35/208) +26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:38 INFO Executor: Finished task 35.0 in stage 8.0 (TID 43). 6787 bytes result sent to driver +26/04/01 06:26:38 INFO TaskSetManager: Starting task 39.0 in stage 8.0 (TID 47) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:38 INFO Executor: Running task 39.0 in stage 8.0 (TID 47) +26/04/01 06:26:38 INFO TaskSetManager: Finished task 35.0 in stage 8.0 (TID 43) in 6472 ms on 10.0.0.133 (executor driver) (36/208) +26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:45 INFO Executor: Finished task 37.0 in stage 8.0 (TID 45). 6787 bytes result sent to driver +26/04/01 06:26:45 INFO TaskSetManager: Starting task 40.0 in stage 8.0 (TID 48) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:45 INFO Executor: Finished task 36.0 in stage 8.0 (TID 44). 6787 bytes result sent to driver +26/04/01 06:26:45 INFO TaskSetManager: Finished task 37.0 in stage 8.0 (TID 45) in 6538 ms on 10.0.0.133 (executor driver) (37/208) +26/04/01 06:26:45 INFO Executor: Running task 40.0 in stage 8.0 (TID 48) +26/04/01 06:26:45 INFO TaskSetManager: Starting task 41.0 in stage 8.0 (TID 49) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:45 INFO Executor: Running task 41.0 in stage 8.0 (TID 49) +26/04/01 06:26:45 INFO TaskSetManager: Finished task 36.0 in stage 8.0 (TID 44) in 6539 ms on 10.0.0.133 (executor driver) (38/208) +26/04/01 06:26:45 INFO Executor: Finished task 38.0 in stage 8.0 (TID 46). 6787 bytes result sent to driver +26/04/01 06:26:45 INFO TaskSetManager: Starting task 42.0 in stage 8.0 (TID 50) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:26:45 INFO TaskSetManager: Finished task 38.0 in stage 8.0 (TID 46) in 6534 ms on 10.0.0.133 (executor driver) (39/208) +26/04/01 06:26:45 INFO Executor: Running task 42.0 in stage 8.0 (TID 50) +26/04/01 06:26:45 INFO Executor: Finished task 39.0 in stage 8.0 (TID 47). 6787 bytes result sent to driver +26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:45 INFO TaskSetManager: Starting task 43.0 in stage 8.0 (TID 51) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:26:45 INFO Executor: Running task 43.0 in stage 8.0 (TID 51) +26/04/01 06:26:45 INFO TaskSetManager: Finished task 39.0 in stage 8.0 (TID 47) in 6515 ms on 10.0.0.133 (executor driver) (40/208) +26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:51 INFO Executor: Finished task 42.0 in stage 8.0 (TID 50). 6787 bytes result sent to driver +26/04/01 06:26:51 INFO Executor: Finished task 43.0 in stage 8.0 (TID 51). 6787 bytes result sent to driver +26/04/01 06:26:51 INFO TaskSetManager: Starting task 44.0 in stage 8.0 (TID 52) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:26:51 INFO TaskSetManager: Starting task 45.0 in stage 8.0 (TID 53) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:26:51 INFO Executor: Running task 44.0 in stage 8.0 (TID 52) +26/04/01 06:26:51 INFO TaskSetManager: Finished task 42.0 in stage 8.0 (TID 50) in 6454 ms on 10.0.0.133 (executor driver) (41/208) +26/04/01 06:26:51 INFO TaskSetManager: Finished task 43.0 in stage 8.0 (TID 51) in 6453 ms on 10.0.0.133 (executor driver) (42/208) +26/04/01 06:26:51 INFO Executor: Running task 45.0 in stage 8.0 (TID 53) +26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:51 INFO Executor: Finished task 41.0 in stage 8.0 (TID 49). 6787 bytes result sent to driver +26/04/01 06:26:51 INFO TaskSetManager: Starting task 46.0 in stage 8.0 (TID 54) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:26:51 INFO TaskSetManager: Finished task 41.0 in stage 8.0 (TID 49) in 6491 ms on 10.0.0.133 (executor driver) (43/208) +26/04/01 06:26:51 INFO Executor: Running task 46.0 in stage 8.0 (TID 54) +26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:51 INFO Executor: Finished task 40.0 in stage 8.0 (TID 48). 6787 bytes result sent to driver +26/04/01 06:26:51 INFO TaskSetManager: Starting task 47.0 in stage 8.0 (TID 55) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:26:51 INFO Executor: Running task 47.0 in stage 8.0 (TID 55) +26/04/01 06:26:51 INFO TaskSetManager: Finished task 40.0 in stage 8.0 (TID 48) in 6498 ms on 10.0.0.133 (executor driver) (44/208) +26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:58 INFO Executor: Finished task 45.0 in stage 8.0 (TID 53). 6787 bytes result sent to driver +26/04/01 06:26:58 INFO TaskSetManager: Starting task 48.0 in stage 8.0 (TID 56) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:26:58 INFO Executor: Running task 48.0 in stage 8.0 (TID 56) +26/04/01 06:26:58 INFO TaskSetManager: Finished task 45.0 in stage 8.0 (TID 53) in 6221 ms on 10.0.0.133 (executor driver) (45/208) +26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:58 INFO Executor: Finished task 47.0 in stage 8.0 (TID 55). 6787 bytes result sent to driver +26/04/01 06:26:58 INFO TaskSetManager: Starting task 49.0 in stage 8.0 (TID 57) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:26:58 INFO TaskSetManager: Finished task 47.0 in stage 8.0 (TID 55) in 6455 ms on 10.0.0.133 (executor driver) (46/208) +26/04/01 06:26:58 INFO Executor: Running task 49.0 in stage 8.0 (TID 57) +26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:58 INFO Executor: Finished task 46.0 in stage 8.0 (TID 54). 6787 bytes result sent to driver +26/04/01 06:26:58 INFO TaskSetManager: Starting task 50.0 in stage 8.0 (TID 58) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:58 INFO TaskSetManager: Finished task 46.0 in stage 8.0 (TID 54) in 6468 ms on 10.0.0.133 (executor driver) (47/208) +26/04/01 06:26:58 INFO Executor: Running task 50.0 in stage 8.0 (TID 58) +26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:58 INFO Executor: Finished task 44.0 in stage 8.0 (TID 52). 6787 bytes result sent to driver +26/04/01 06:26:58 INFO TaskSetManager: Starting task 51.0 in stage 8.0 (TID 59) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:26:58 INFO TaskSetManager: Finished task 44.0 in stage 8.0 (TID 52) in 6711 ms on 10.0.0.133 (executor driver) (48/208) +26/04/01 06:26:58 INFO Executor: Running task 51.0 in stage 8.0 (TID 59) +26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:04 INFO Executor: Finished task 48.0 in stage 8.0 (TID 56). 6787 bytes result sent to driver +26/04/01 06:27:04 INFO TaskSetManager: Starting task 52.0 in stage 8.0 (TID 60) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:04 INFO Executor: Running task 52.0 in stage 8.0 (TID 60) +26/04/01 06:27:04 INFO TaskSetManager: Finished task 48.0 in stage 8.0 (TID 56) in 6527 ms on 10.0.0.133 (executor driver) (49/208) +26/04/01 06:27:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:04 INFO Executor: Finished task 49.0 in stage 8.0 (TID 57). 6787 bytes result sent to driver +26/04/01 06:27:04 INFO TaskSetManager: Starting task 53.0 in stage 8.0 (TID 61) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:04 INFO Executor: Running task 53.0 in stage 8.0 (TID 61) +26/04/01 06:27:04 INFO TaskSetManager: Finished task 49.0 in stage 8.0 (TID 57) in 6498 ms on 10.0.0.133 (executor driver) (50/208) +26/04/01 06:27:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:04 INFO Executor: Finished task 50.0 in stage 8.0 (TID 58). 6787 bytes result sent to driver +26/04/01 06:27:04 INFO TaskSetManager: Starting task 54.0 in stage 8.0 (TID 62) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:27:04 INFO TaskSetManager: Finished task 50.0 in stage 8.0 (TID 58) in 6499 ms on 10.0.0.133 (executor driver) (51/208) +26/04/01 06:27:04 INFO Executor: Running task 54.0 in stage 8.0 (TID 62) +26/04/01 06:27:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:05 INFO Executor: Finished task 51.0 in stage 8.0 (TID 59). 6787 bytes result sent to driver +26/04/01 06:27:05 INFO TaskSetManager: Starting task 55.0 in stage 8.0 (TID 63) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:27:05 INFO Executor: Running task 55.0 in stage 8.0 (TID 63) +26/04/01 06:27:05 INFO TaskSetManager: Finished task 51.0 in stage 8.0 (TID 59) in 6536 ms on 10.0.0.133 (executor driver) (52/208) +26/04/01 06:27:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:11 INFO Executor: Finished task 52.0 in stage 8.0 (TID 60). 6787 bytes result sent to driver +26/04/01 06:27:11 INFO TaskSetManager: Starting task 56.0 in stage 8.0 (TID 64) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:11 INFO Executor: Running task 56.0 in stage 8.0 (TID 64) +26/04/01 06:27:11 INFO TaskSetManager: Finished task 52.0 in stage 8.0 (TID 60) in 6497 ms on 10.0.0.133 (executor driver) (53/208) +26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:11 INFO Executor: Finished task 53.0 in stage 8.0 (TID 61). 6787 bytes result sent to driver +26/04/01 06:27:11 INFO TaskSetManager: Starting task 57.0 in stage 8.0 (TID 65) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:11 INFO TaskSetManager: Finished task 53.0 in stage 8.0 (TID 61) in 6498 ms on 10.0.0.133 (executor driver) (54/208) +26/04/01 06:27:11 INFO Executor: Running task 57.0 in stage 8.0 (TID 65) +26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:11 INFO Executor: Finished task 54.0 in stage 8.0 (TID 62). 6787 bytes result sent to driver +26/04/01 06:27:11 INFO TaskSetManager: Starting task 58.0 in stage 8.0 (TID 66) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:11 INFO Executor: Running task 58.0 in stage 8.0 (TID 66) +26/04/01 06:27:11 INFO TaskSetManager: Finished task 54.0 in stage 8.0 (TID 62) in 6501 ms on 10.0.0.133 (executor driver) (55/208) +26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:11 INFO Executor: Finished task 55.0 in stage 8.0 (TID 63). 6787 bytes result sent to driver +26/04/01 06:27:11 INFO TaskSetManager: Starting task 59.0 in stage 8.0 (TID 67) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:11 INFO TaskSetManager: Finished task 55.0 in stage 8.0 (TID 63) in 6496 ms on 10.0.0.133 (executor driver) (56/208) +26/04/01 06:27:11 INFO Executor: Running task 59.0 in stage 8.0 (TID 67) +26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:17 INFO Executor: Finished task 56.0 in stage 8.0 (TID 64). 6787 bytes result sent to driver +26/04/01 06:27:17 INFO TaskSetManager: Starting task 60.0 in stage 8.0 (TID 68) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:27:17 INFO Executor: Running task 60.0 in stage 8.0 (TID 68) +26/04/01 06:27:17 INFO TaskSetManager: Finished task 56.0 in stage 8.0 (TID 64) in 6536 ms on 10.0.0.133 (executor driver) (57/208) +26/04/01 06:27:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:17 INFO Executor: Finished task 58.0 in stage 8.0 (TID 66). 6787 bytes result sent to driver +26/04/01 06:27:17 INFO TaskSetManager: Starting task 61.0 in stage 8.0 (TID 69) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:27:17 INFO TaskSetManager: Finished task 58.0 in stage 8.0 (TID 66) in 6525 ms on 10.0.0.133 (executor driver) (58/208) +26/04/01 06:27:17 INFO Executor: Running task 61.0 in stage 8.0 (TID 69) +26/04/01 06:27:17 INFO Executor: Finished task 57.0 in stage 8.0 (TID 65). 6787 bytes result sent to driver +26/04/01 06:27:17 INFO TaskSetManager: Starting task 62.0 in stage 8.0 (TID 70) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:17 INFO TaskSetManager: Finished task 57.0 in stage 8.0 (TID 65) in 6535 ms on 10.0.0.133 (executor driver) (59/208) +26/04/01 06:27:17 INFO Executor: Running task 62.0 in stage 8.0 (TID 70) +26/04/01 06:27:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:18 INFO Executor: Finished task 59.0 in stage 8.0 (TID 67). 6787 bytes result sent to driver +26/04/01 06:27:18 INFO TaskSetManager: Starting task 63.0 in stage 8.0 (TID 71) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:18 INFO TaskSetManager: Finished task 59.0 in stage 8.0 (TID 67) in 6537 ms on 10.0.0.133 (executor driver) (60/208) +26/04/01 06:27:18 INFO Executor: Running task 63.0 in stage 8.0 (TID 71) +26/04/01 06:27:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:24 INFO Executor: Finished task 60.0 in stage 8.0 (TID 68). 6787 bytes result sent to driver +26/04/01 06:27:24 INFO TaskSetManager: Starting task 64.0 in stage 8.0 (TID 72) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:24 INFO Executor: Running task 64.0 in stage 8.0 (TID 72) +26/04/01 06:27:24 INFO TaskSetManager: Finished task 60.0 in stage 8.0 (TID 68) in 6539 ms on 10.0.0.133 (executor driver) (61/208) +26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:24 INFO Executor: Finished task 61.0 in stage 8.0 (TID 69). 6787 bytes result sent to driver +26/04/01 06:27:24 INFO Executor: Finished task 62.0 in stage 8.0 (TID 70). 6787 bytes result sent to driver +26/04/01 06:27:24 INFO TaskSetManager: Starting task 65.0 in stage 8.0 (TID 73) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:24 INFO Executor: Running task 65.0 in stage 8.0 (TID 73) +26/04/01 06:27:24 INFO TaskSetManager: Starting task 66.0 in stage 8.0 (TID 74) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:27:24 INFO TaskSetManager: Finished task 61.0 in stage 8.0 (TID 69) in 6542 ms on 10.0.0.133 (executor driver) (62/208) +26/04/01 06:27:24 INFO Executor: Running task 66.0 in stage 8.0 (TID 74) +26/04/01 06:27:24 INFO TaskSetManager: Finished task 62.0 in stage 8.0 (TID 70) in 6542 ms on 10.0.0.133 (executor driver) (63/208) +26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:24 INFO Executor: Finished task 63.0 in stage 8.0 (TID 71). 6787 bytes result sent to driver +26/04/01 06:27:24 INFO TaskSetManager: Starting task 67.0 in stage 8.0 (TID 75) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:27:24 INFO Executor: Running task 67.0 in stage 8.0 (TID 75) +26/04/01 06:27:24 INFO TaskSetManager: Finished task 63.0 in stage 8.0 (TID 71) in 6539 ms on 10.0.0.133 (executor driver) (64/208) +26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:30 INFO Executor: Finished task 64.0 in stage 8.0 (TID 72). 6787 bytes result sent to driver +26/04/01 06:27:30 INFO TaskSetManager: Starting task 68.0 in stage 8.0 (TID 76) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:27:30 INFO TaskSetManager: Finished task 64.0 in stage 8.0 (TID 72) in 6480 ms on 10.0.0.133 (executor driver) (65/208) +26/04/01 06:27:30 INFO Executor: Running task 68.0 in stage 8.0 (TID 76) +26/04/01 06:27:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:30 INFO Executor: Finished task 66.0 in stage 8.0 (TID 74). 6787 bytes result sent to driver +26/04/01 06:27:30 INFO TaskSetManager: Starting task 69.0 in stage 8.0 (TID 77) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:27:30 INFO TaskSetManager: Finished task 66.0 in stage 8.0 (TID 74) in 6444 ms on 10.0.0.133 (executor driver) (66/208) +26/04/01 06:27:30 INFO Executor: Running task 69.0 in stage 8.0 (TID 77) +26/04/01 06:27:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:30 INFO Executor: Finished task 65.0 in stage 8.0 (TID 73). 6787 bytes result sent to driver +26/04/01 06:27:30 INFO TaskSetManager: Starting task 70.0 in stage 8.0 (TID 78) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:27:30 INFO TaskSetManager: Finished task 65.0 in stage 8.0 (TID 73) in 6486 ms on 10.0.0.133 (executor driver) (67/208) +26/04/01 06:27:30 INFO Executor: Running task 70.0 in stage 8.0 (TID 78) +26/04/01 06:27:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:31 INFO Executor: Finished task 67.0 in stage 8.0 (TID 75). 6787 bytes result sent to driver +26/04/01 06:27:31 INFO TaskSetManager: Starting task 71.0 in stage 8.0 (TID 79) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:27:31 INFO TaskSetManager: Finished task 67.0 in stage 8.0 (TID 75) in 6440 ms on 10.0.0.133 (executor driver) (68/208) +26/04/01 06:27:31 INFO Executor: Running task 71.0 in stage 8.0 (TID 79) +26/04/01 06:27:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:37 INFO Executor: Finished task 68.0 in stage 8.0 (TID 76). 6787 bytes result sent to driver +26/04/01 06:27:37 INFO TaskSetManager: Starting task 72.0 in stage 8.0 (TID 80) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:27:37 INFO TaskSetManager: Finished task 68.0 in stage 8.0 (TID 76) in 6446 ms on 10.0.0.133 (executor driver) (69/208) +26/04/01 06:27:37 INFO Executor: Running task 72.0 in stage 8.0 (TID 80) +26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:37 INFO Executor: Finished task 69.0 in stage 8.0 (TID 77). 6830 bytes result sent to driver +26/04/01 06:27:37 INFO TaskSetManager: Starting task 73.0 in stage 8.0 (TID 81) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:27:37 INFO TaskSetManager: Finished task 69.0 in stage 8.0 (TID 77) in 6444 ms on 10.0.0.133 (executor driver) (70/208) +26/04/01 06:27:37 INFO Executor: Running task 73.0 in stage 8.0 (TID 81) +26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:37 INFO Executor: Finished task 70.0 in stage 8.0 (TID 78). 6830 bytes result sent to driver +26/04/01 06:27:37 INFO TaskSetManager: Starting task 74.0 in stage 8.0 (TID 82) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:27:37 INFO Executor: Running task 74.0 in stage 8.0 (TID 82) +26/04/01 06:27:37 INFO TaskSetManager: Finished task 70.0 in stage 8.0 (TID 78) in 6455 ms on 10.0.0.133 (executor driver) (71/208) +26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:37 INFO Executor: Finished task 71.0 in stage 8.0 (TID 79). 6830 bytes result sent to driver +26/04/01 06:27:37 INFO TaskSetManager: Starting task 75.0 in stage 8.0 (TID 83) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:27:37 INFO Executor: Running task 75.0 in stage 8.0 (TID 83) +26/04/01 06:27:37 INFO TaskSetManager: Finished task 71.0 in stage 8.0 (TID 79) in 6448 ms on 10.0.0.133 (executor driver) (72/208) +26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:43 INFO Executor: Finished task 72.0 in stage 8.0 (TID 80). 6830 bytes result sent to driver +26/04/01 06:27:43 INFO TaskSetManager: Starting task 76.0 in stage 8.0 (TID 84) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:27:43 INFO TaskSetManager: Finished task 72.0 in stage 8.0 (TID 80) in 6499 ms on 10.0.0.133 (executor driver) (73/208) +26/04/01 06:27:43 INFO Executor: Running task 76.0 in stage 8.0 (TID 84) +26/04/01 06:27:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:43 INFO Executor: Finished task 73.0 in stage 8.0 (TID 81). 6787 bytes result sent to driver +26/04/01 06:27:43 INFO TaskSetManager: Starting task 77.0 in stage 8.0 (TID 85) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:27:43 INFO TaskSetManager: Finished task 73.0 in stage 8.0 (TID 81) in 6515 ms on 10.0.0.133 (executor driver) (74/208) +26/04/01 06:27:43 INFO Executor: Running task 77.0 in stage 8.0 (TID 85) +26/04/01 06:27:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:43 INFO Executor: Finished task 74.0 in stage 8.0 (TID 82). 6787 bytes result sent to driver +26/04/01 06:27:43 INFO TaskSetManager: Starting task 78.0 in stage 8.0 (TID 86) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:27:43 INFO TaskSetManager: Finished task 74.0 in stage 8.0 (TID 82) in 6498 ms on 10.0.0.133 (executor driver) (75/208) +26/04/01 06:27:43 INFO Executor: Running task 78.0 in stage 8.0 (TID 86) +26/04/01 06:27:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:44 INFO Executor: Finished task 75.0 in stage 8.0 (TID 83). 6787 bytes result sent to driver +26/04/01 06:27:44 INFO TaskSetManager: Starting task 79.0 in stage 8.0 (TID 87) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:27:44 INFO TaskSetManager: Finished task 75.0 in stage 8.0 (TID 83) in 6520 ms on 10.0.0.133 (executor driver) (76/208) +26/04/01 06:27:44 INFO Executor: Running task 79.0 in stage 8.0 (TID 87) +26/04/01 06:27:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:50 INFO Executor: Finished task 76.0 in stage 8.0 (TID 84). 6787 bytes result sent to driver +26/04/01 06:27:50 INFO TaskSetManager: Starting task 80.0 in stage 8.0 (TID 88) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:50 INFO TaskSetManager: Finished task 76.0 in stage 8.0 (TID 84) in 6507 ms on 10.0.0.133 (executor driver) (77/208) +26/04/01 06:27:50 INFO Executor: Running task 80.0 in stage 8.0 (TID 88) +26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:50 INFO Executor: Finished task 77.0 in stage 8.0 (TID 85). 6787 bytes result sent to driver +26/04/01 06:27:50 INFO TaskSetManager: Starting task 81.0 in stage 8.0 (TID 89) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:50 INFO Executor: Running task 81.0 in stage 8.0 (TID 89) +26/04/01 06:27:50 INFO TaskSetManager: Finished task 77.0 in stage 8.0 (TID 85) in 6525 ms on 10.0.0.133 (executor driver) (78/208) +26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:50 INFO Executor: Finished task 78.0 in stage 8.0 (TID 86). 6787 bytes result sent to driver +26/04/01 06:27:50 INFO TaskSetManager: Starting task 82.0 in stage 8.0 (TID 90) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:50 INFO TaskSetManager: Finished task 78.0 in stage 8.0 (TID 86) in 6507 ms on 10.0.0.133 (executor driver) (79/208) +26/04/01 06:27:50 INFO Executor: Running task 82.0 in stage 8.0 (TID 90) +26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:50 INFO Executor: Finished task 79.0 in stage 8.0 (TID 87). 6787 bytes result sent to driver +26/04/01 06:27:50 INFO TaskSetManager: Starting task 83.0 in stage 8.0 (TID 91) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:50 INFO TaskSetManager: Finished task 79.0 in stage 8.0 (TID 87) in 6525 ms on 10.0.0.133 (executor driver) (80/208) +26/04/01 06:27:50 INFO Executor: Running task 83.0 in stage 8.0 (TID 91) +26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:56 INFO Executor: Finished task 80.0 in stage 8.0 (TID 88). 6787 bytes result sent to driver +26/04/01 06:27:56 INFO TaskSetManager: Starting task 84.0 in stage 8.0 (TID 92) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:27:56 INFO Executor: Running task 84.0 in stage 8.0 (TID 92) +26/04/01 06:27:56 INFO TaskSetManager: Finished task 80.0 in stage 8.0 (TID 88) in 6534 ms on 10.0.0.133 (executor driver) (81/208) +26/04/01 06:27:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:56 INFO Executor: Finished task 81.0 in stage 8.0 (TID 89). 6787 bytes result sent to driver +26/04/01 06:27:56 INFO TaskSetManager: Starting task 85.0 in stage 8.0 (TID 93) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:27:56 INFO Executor: Running task 85.0 in stage 8.0 (TID 93) +26/04/01 06:27:56 INFO TaskSetManager: Finished task 81.0 in stage 8.0 (TID 89) in 6532 ms on 10.0.0.133 (executor driver) (82/208) +26/04/01 06:27:56 INFO Executor: Finished task 82.0 in stage 8.0 (TID 90). 6787 bytes result sent to driver +26/04/01 06:27:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:56 INFO TaskSetManager: Starting task 86.0 in stage 8.0 (TID 94) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:56 INFO TaskSetManager: Finished task 82.0 in stage 8.0 (TID 90) in 6516 ms on 10.0.0.133 (executor driver) (83/208) +26/04/01 06:27:56 INFO Executor: Running task 86.0 in stage 8.0 (TID 94) +26/04/01 06:27:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:57 INFO Executor: Finished task 83.0 in stage 8.0 (TID 91). 6787 bytes result sent to driver +26/04/01 06:27:57 INFO TaskSetManager: Starting task 87.0 in stage 8.0 (TID 95) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:27:57 INFO TaskSetManager: Finished task 83.0 in stage 8.0 (TID 91) in 6540 ms on 10.0.0.133 (executor driver) (84/208) +26/04/01 06:27:57 INFO Executor: Running task 87.0 in stage 8.0 (TID 95) +26/04/01 06:27:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:27:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:03 INFO Executor: Finished task 84.0 in stage 8.0 (TID 92). 6787 bytes result sent to driver +26/04/01 06:28:03 INFO TaskSetManager: Starting task 88.0 in stage 8.0 (TID 96) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:03 INFO TaskSetManager: Finished task 84.0 in stage 8.0 (TID 92) in 6529 ms on 10.0.0.133 (executor driver) (85/208) +26/04/01 06:28:03 INFO Executor: Running task 88.0 in stage 8.0 (TID 96) +26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:03 INFO Executor: Finished task 86.0 in stage 8.0 (TID 94). 6787 bytes result sent to driver +26/04/01 06:28:03 INFO Executor: Finished task 85.0 in stage 8.0 (TID 93). 6787 bytes result sent to driver +26/04/01 06:28:03 INFO TaskSetManager: Starting task 89.0 in stage 8.0 (TID 97) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:03 INFO Executor: Running task 89.0 in stage 8.0 (TID 97) +26/04/01 06:28:03 INFO TaskSetManager: Starting task 90.0 in stage 8.0 (TID 98) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:28:03 INFO Executor: Running task 90.0 in stage 8.0 (TID 98) +26/04/01 06:28:03 INFO TaskSetManager: Finished task 86.0 in stage 8.0 (TID 94) in 6531 ms on 10.0.0.133 (executor driver) (86/208) +26/04/01 06:28:03 INFO TaskSetManager: Finished task 85.0 in stage 8.0 (TID 93) in 6533 ms on 10.0.0.133 (executor driver) (87/208) +26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:03 INFO Executor: Finished task 87.0 in stage 8.0 (TID 95). 6787 bytes result sent to driver +26/04/01 06:28:03 INFO TaskSetManager: Starting task 91.0 in stage 8.0 (TID 99) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:28:03 INFO TaskSetManager: Finished task 87.0 in stage 8.0 (TID 95) in 6525 ms on 10.0.0.133 (executor driver) (88/208) +26/04/01 06:28:03 INFO Executor: Running task 91.0 in stage 8.0 (TID 99) +26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:09 INFO Executor: Finished task 88.0 in stage 8.0 (TID 96). 6787 bytes result sent to driver +26/04/01 06:28:09 INFO TaskSetManager: Starting task 92.0 in stage 8.0 (TID 100) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:09 INFO Executor: Running task 92.0 in stage 8.0 (TID 100) +26/04/01 06:28:09 INFO TaskSetManager: Finished task 88.0 in stage 8.0 (TID 96) in 6537 ms on 10.0.0.133 (executor driver) (89/208) +26/04/01 06:28:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:09 INFO Executor: Finished task 89.0 in stage 8.0 (TID 97). 6787 bytes result sent to driver +26/04/01 06:28:09 INFO TaskSetManager: Starting task 93.0 in stage 8.0 (TID 101) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:09 INFO TaskSetManager: Finished task 89.0 in stage 8.0 (TID 97) in 6532 ms on 10.0.0.133 (executor driver) (90/208) +26/04/01 06:28:09 INFO Executor: Running task 93.0 in stage 8.0 (TID 101) +26/04/01 06:28:09 INFO Executor: Finished task 91.0 in stage 8.0 (TID 99). 6787 bytes result sent to driver +26/04/01 06:28:09 INFO TaskSetManager: Starting task 94.0 in stage 8.0 (TID 102) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:09 INFO TaskSetManager: Finished task 91.0 in stage 8.0 (TID 99) in 6283 ms on 10.0.0.133 (executor driver) (91/208) +26/04/01 06:28:09 INFO Executor: Running task 94.0 in stage 8.0 (TID 102) +26/04/01 06:28:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:10 INFO Executor: Finished task 90.0 in stage 8.0 (TID 98). 6787 bytes result sent to driver +26/04/01 06:28:10 INFO TaskSetManager: Starting task 95.0 in stage 8.0 (TID 103) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:10 INFO Executor: Running task 95.0 in stage 8.0 (TID 103) +26/04/01 06:28:10 INFO TaskSetManager: Finished task 90.0 in stage 8.0 (TID 98) in 6782 ms on 10.0.0.133 (executor driver) (92/208) +26/04/01 06:28:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:16 INFO Executor: Finished task 92.0 in stage 8.0 (TID 100). 6787 bytes result sent to driver +26/04/01 06:28:16 INFO TaskSetManager: Starting task 96.0 in stage 8.0 (TID 104) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:28:16 INFO Executor: Running task 96.0 in stage 8.0 (TID 104) +26/04/01 06:28:16 INFO TaskSetManager: Finished task 92.0 in stage 8.0 (TID 100) in 6540 ms on 10.0.0.133 (executor driver) (93/208) +26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:16 INFO Executor: Finished task 93.0 in stage 8.0 (TID 101). 6787 bytes result sent to driver +26/04/01 06:28:16 INFO TaskSetManager: Starting task 97.0 in stage 8.0 (TID 105) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:28:16 INFO TaskSetManager: Finished task 93.0 in stage 8.0 (TID 101) in 6542 ms on 10.0.0.133 (executor driver) (94/208) +26/04/01 06:28:16 INFO Executor: Running task 97.0 in stage 8.0 (TID 105) +26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:16 INFO Executor: Finished task 94.0 in stage 8.0 (TID 102). 6787 bytes result sent to driver +26/04/01 06:28:16 INFO TaskSetManager: Starting task 98.0 in stage 8.0 (TID 106) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:28:16 INFO TaskSetManager: Finished task 94.0 in stage 8.0 (TID 102) in 6544 ms on 10.0.0.133 (executor driver) (95/208) +26/04/01 06:28:16 INFO Executor: Running task 98.0 in stage 8.0 (TID 106) +26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:16 INFO Executor: Finished task 95.0 in stage 8.0 (TID 103). 6787 bytes result sent to driver +26/04/01 06:28:16 INFO TaskSetManager: Starting task 99.0 in stage 8.0 (TID 107) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:28:16 INFO TaskSetManager: Finished task 95.0 in stage 8.0 (TID 103) in 6542 ms on 10.0.0.133 (executor driver) (96/208) +26/04/01 06:28:16 INFO Executor: Running task 99.0 in stage 8.0 (TID 107) +26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:22 INFO Executor: Finished task 96.0 in stage 8.0 (TID 104). 6787 bytes result sent to driver +26/04/01 06:28:22 INFO TaskSetManager: Starting task 100.0 in stage 8.0 (TID 108) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:28:22 INFO TaskSetManager: Finished task 96.0 in stage 8.0 (TID 104) in 6542 ms on 10.0.0.133 (executor driver) (97/208) +26/04/01 06:28:22 INFO Executor: Running task 100.0 in stage 8.0 (TID 108) +26/04/01 06:28:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:23 INFO Executor: Finished task 98.0 in stage 8.0 (TID 106). 6787 bytes result sent to driver +26/04/01 06:28:23 INFO Executor: Finished task 97.0 in stage 8.0 (TID 105). 6787 bytes result sent to driver +26/04/01 06:28:23 INFO TaskSetManager: Starting task 101.0 in stage 8.0 (TID 109) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:28:23 INFO Executor: Running task 101.0 in stage 8.0 (TID 109) +26/04/01 06:28:23 INFO TaskSetManager: Starting task 102.0 in stage 8.0 (TID 110) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:28:23 INFO TaskSetManager: Finished task 98.0 in stage 8.0 (TID 106) in 6531 ms on 10.0.0.133 (executor driver) (98/208) +26/04/01 06:28:23 INFO Executor: Running task 102.0 in stage 8.0 (TID 110) +26/04/01 06:28:23 INFO TaskSetManager: Finished task 97.0 in stage 8.0 (TID 105) in 6534 ms on 10.0.0.133 (executor driver) (99/208) +26/04/01 06:28:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:23 INFO Executor: Finished task 99.0 in stage 8.0 (TID 107). 6787 bytes result sent to driver +26/04/01 06:28:23 INFO TaskSetManager: Starting task 103.0 in stage 8.0 (TID 111) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:28:23 INFO TaskSetManager: Finished task 99.0 in stage 8.0 (TID 107) in 6534 ms on 10.0.0.133 (executor driver) (100/208) +26/04/01 06:28:23 INFO Executor: Running task 103.0 in stage 8.0 (TID 111) +26/04/01 06:28:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:29 INFO Executor: Finished task 100.0 in stage 8.0 (TID 108). 6787 bytes result sent to driver +26/04/01 06:28:29 INFO TaskSetManager: Starting task 104.0 in stage 8.0 (TID 112) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:29 INFO Executor: Running task 104.0 in stage 8.0 (TID 112) +26/04/01 06:28:29 INFO TaskSetManager: Finished task 100.0 in stage 8.0 (TID 108) in 6518 ms on 10.0.0.133 (executor driver) (101/208) +26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:29 INFO Executor: Finished task 101.0 in stage 8.0 (TID 109). 6787 bytes result sent to driver +26/04/01 06:28:29 INFO Executor: Finished task 102.0 in stage 8.0 (TID 110). 6787 bytes result sent to driver +26/04/01 06:28:29 INFO TaskSetManager: Starting task 105.0 in stage 8.0 (TID 113) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:29 INFO TaskSetManager: Starting task 106.0 in stage 8.0 (TID 114) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:29 INFO Executor: Running task 105.0 in stage 8.0 (TID 113) +26/04/01 06:28:29 INFO TaskSetManager: Finished task 101.0 in stage 8.0 (TID 109) in 6523 ms on 10.0.0.133 (executor driver) (102/208) +26/04/01 06:28:29 INFO Executor: Running task 106.0 in stage 8.0 (TID 114) +26/04/01 06:28:29 INFO TaskSetManager: Finished task 102.0 in stage 8.0 (TID 110) in 6523 ms on 10.0.0.133 (executor driver) (103/208) +26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:29 INFO Executor: Finished task 103.0 in stage 8.0 (TID 111). 6787 bytes result sent to driver +26/04/01 06:28:29 INFO TaskSetManager: Starting task 107.0 in stage 8.0 (TID 115) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:29 INFO TaskSetManager: Finished task 103.0 in stage 8.0 (TID 111) in 6525 ms on 10.0.0.133 (executor driver) (104/208) +26/04/01 06:28:29 INFO Executor: Running task 107.0 in stage 8.0 (TID 115) +26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:35 INFO Executor: Finished task 104.0 in stage 8.0 (TID 112). 6787 bytes result sent to driver +26/04/01 06:28:35 INFO TaskSetManager: Starting task 108.0 in stage 8.0 (TID 116) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:28:35 INFO Executor: Running task 108.0 in stage 8.0 (TID 116) +26/04/01 06:28:35 INFO TaskSetManager: Finished task 104.0 in stage 8.0 (TID 112) in 6501 ms on 10.0.0.133 (executor driver) (105/208) +26/04/01 06:28:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:36 INFO Executor: Finished task 105.0 in stage 8.0 (TID 113). 6787 bytes result sent to driver +26/04/01 06:28:36 INFO TaskSetManager: Starting task 109.0 in stage 8.0 (TID 117) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:28:36 INFO Executor: Running task 109.0 in stage 8.0 (TID 117) +26/04/01 06:28:36 INFO TaskSetManager: Finished task 105.0 in stage 8.0 (TID 113) in 6503 ms on 10.0.0.133 (executor driver) (106/208) +26/04/01 06:28:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:36 INFO Executor: Finished task 106.0 in stage 8.0 (TID 114). 6787 bytes result sent to driver +26/04/01 06:28:36 INFO TaskSetManager: Starting task 110.0 in stage 8.0 (TID 118) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:28:36 INFO TaskSetManager: Finished task 106.0 in stage 8.0 (TID 114) in 6507 ms on 10.0.0.133 (executor driver) (107/208) +26/04/01 06:28:36 INFO Executor: Running task 110.0 in stage 8.0 (TID 118) +26/04/01 06:28:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:36 INFO Executor: Finished task 107.0 in stage 8.0 (TID 115). 6787 bytes result sent to driver +26/04/01 06:28:36 INFO TaskSetManager: Starting task 111.0 in stage 8.0 (TID 119) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:28:36 INFO Executor: Running task 111.0 in stage 8.0 (TID 119) +26/04/01 06:28:36 INFO TaskSetManager: Finished task 107.0 in stage 8.0 (TID 115) in 6502 ms on 10.0.0.133 (executor driver) (108/208) +26/04/01 06:28:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:42 INFO Executor: Finished task 108.0 in stage 8.0 (TID 116). 6787 bytes result sent to driver +26/04/01 06:28:42 INFO TaskSetManager: Starting task 112.0 in stage 8.0 (TID 120) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:28:42 INFO Executor: Running task 112.0 in stage 8.0 (TID 120) +26/04/01 06:28:42 INFO TaskSetManager: Finished task 108.0 in stage 8.0 (TID 116) in 6527 ms on 10.0.0.133 (executor driver) (109/208) +26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:42 INFO Executor: Finished task 109.0 in stage 8.0 (TID 117). 6787 bytes result sent to driver +26/04/01 06:28:42 INFO TaskSetManager: Starting task 113.0 in stage 8.0 (TID 121) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:28:42 INFO Executor: Running task 113.0 in stage 8.0 (TID 121) +26/04/01 06:28:42 INFO TaskSetManager: Finished task 109.0 in stage 8.0 (TID 117) in 6524 ms on 10.0.0.133 (executor driver) (110/208) +26/04/01 06:28:42 INFO Executor: Finished task 110.0 in stage 8.0 (TID 118). 6787 bytes result sent to driver +26/04/01 06:28:42 INFO TaskSetManager: Starting task 114.0 in stage 8.0 (TID 122) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:28:42 INFO TaskSetManager: Finished task 110.0 in stage 8.0 (TID 118) in 6521 ms on 10.0.0.133 (executor driver) (111/208) +26/04/01 06:28:42 INFO Executor: Running task 114.0 in stage 8.0 (TID 122) +26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:42 INFO Executor: Finished task 111.0 in stage 8.0 (TID 119). 6787 bytes result sent to driver +26/04/01 06:28:42 INFO TaskSetManager: Starting task 115.0 in stage 8.0 (TID 123) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:28:42 INFO TaskSetManager: Finished task 111.0 in stage 8.0 (TID 119) in 6527 ms on 10.0.0.133 (executor driver) (112/208) +26/04/01 06:28:42 INFO Executor: Running task 115.0 in stage 8.0 (TID 123) +26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:48 INFO Executor: Finished task 112.0 in stage 8.0 (TID 120). 6787 bytes result sent to driver +26/04/01 06:28:48 INFO TaskSetManager: Starting task 116.0 in stage 8.0 (TID 124) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:48 INFO Executor: Running task 116.0 in stage 8.0 (TID 124) +26/04/01 06:28:48 INFO TaskSetManager: Finished task 112.0 in stage 8.0 (TID 120) in 6522 ms on 10.0.0.133 (executor driver) (113/208) +26/04/01 06:28:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:49 INFO Executor: Finished task 114.0 in stage 8.0 (TID 122). 6787 bytes result sent to driver +26/04/01 06:28:49 INFO Executor: Finished task 113.0 in stage 8.0 (TID 121). 6787 bytes result sent to driver +26/04/01 06:28:49 INFO TaskSetManager: Starting task 117.0 in stage 8.0 (TID 125) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:49 INFO TaskSetManager: Finished task 114.0 in stage 8.0 (TID 122) in 6522 ms on 10.0.0.133 (executor driver) (114/208) +26/04/01 06:28:49 INFO Executor: Running task 117.0 in stage 8.0 (TID 125) +26/04/01 06:28:49 INFO TaskSetManager: Starting task 118.0 in stage 8.0 (TID 126) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:49 INFO TaskSetManager: Finished task 113.0 in stage 8.0 (TID 121) in 6523 ms on 10.0.0.133 (executor driver) (115/208) +26/04/01 06:28:49 INFO Executor: Running task 118.0 in stage 8.0 (TID 126) +26/04/01 06:28:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:49 INFO Executor: Finished task 115.0 in stage 8.0 (TID 123). 6787 bytes result sent to driver +26/04/01 06:28:49 INFO TaskSetManager: Starting task 119.0 in stage 8.0 (TID 127) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:49 INFO TaskSetManager: Finished task 115.0 in stage 8.0 (TID 123) in 6517 ms on 10.0.0.133 (executor driver) (116/208) +26/04/01 06:28:49 INFO Executor: Running task 119.0 in stage 8.0 (TID 127) +26/04/01 06:28:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:55 INFO Executor: Finished task 116.0 in stage 8.0 (TID 124). 6787 bytes result sent to driver +26/04/01 06:28:55 INFO TaskSetManager: Starting task 120.0 in stage 8.0 (TID 128) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:28:55 INFO TaskSetManager: Finished task 116.0 in stage 8.0 (TID 124) in 6540 ms on 10.0.0.133 (executor driver) (117/208) +26/04/01 06:28:55 INFO Executor: Running task 120.0 in stage 8.0 (TID 128) +26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:55 INFO Executor: Finished task 117.0 in stage 8.0 (TID 125). 6830 bytes result sent to driver +26/04/01 06:28:55 INFO TaskSetManager: Starting task 121.0 in stage 8.0 (TID 129) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:28:55 INFO Executor: Running task 121.0 in stage 8.0 (TID 129) +26/04/01 06:28:55 INFO TaskSetManager: Finished task 117.0 in stage 8.0 (TID 125) in 6535 ms on 10.0.0.133 (executor driver) (118/208) +26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:55 INFO Executor: Finished task 118.0 in stage 8.0 (TID 126). 6830 bytes result sent to driver +26/04/01 06:28:55 INFO TaskSetManager: Starting task 122.0 in stage 8.0 (TID 130) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:55 INFO Executor: Running task 122.0 in stage 8.0 (TID 130) +26/04/01 06:28:55 INFO TaskSetManager: Finished task 118.0 in stage 8.0 (TID 126) in 6537 ms on 10.0.0.133 (executor driver) (119/208) +26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:55 INFO Executor: Finished task 119.0 in stage 8.0 (TID 127). 6830 bytes result sent to driver +26/04/01 06:28:55 INFO TaskSetManager: Starting task 123.0 in stage 8.0 (TID 131) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:28:55 INFO Executor: Running task 123.0 in stage 8.0 (TID 131) +26/04/01 06:28:55 INFO TaskSetManager: Finished task 119.0 in stage 8.0 (TID 127) in 6546 ms on 10.0.0.133 (executor driver) (120/208) +26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:01 INFO Executor: Finished task 120.0 in stage 8.0 (TID 128). 6787 bytes result sent to driver +26/04/01 06:29:01 INFO TaskSetManager: Starting task 124.0 in stage 8.0 (TID 132) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:01 INFO TaskSetManager: Finished task 120.0 in stage 8.0 (TID 128) in 6516 ms on 10.0.0.133 (executor driver) (121/208) +26/04/01 06:29:01 INFO Executor: Running task 124.0 in stage 8.0 (TID 132) +26/04/01 06:29:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:02 INFO Executor: Finished task 121.0 in stage 8.0 (TID 129). 6787 bytes result sent to driver +26/04/01 06:29:02 INFO TaskSetManager: Starting task 125.0 in stage 8.0 (TID 133) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:02 INFO TaskSetManager: Finished task 121.0 in stage 8.0 (TID 129) in 6515 ms on 10.0.0.133 (executor driver) (122/208) +26/04/01 06:29:02 INFO Executor: Running task 125.0 in stage 8.0 (TID 133) +26/04/01 06:29:02 INFO Executor: Finished task 122.0 in stage 8.0 (TID 130). 6787 bytes result sent to driver +26/04/01 06:29:02 INFO TaskSetManager: Starting task 126.0 in stage 8.0 (TID 134) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:29:02 INFO TaskSetManager: Finished task 122.0 in stage 8.0 (TID 130) in 6514 ms on 10.0.0.133 (executor driver) (123/208) +26/04/01 06:29:02 INFO Executor: Running task 126.0 in stage 8.0 (TID 134) +26/04/01 06:29:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:02 INFO Executor: Finished task 123.0 in stage 8.0 (TID 131). 6787 bytes result sent to driver +26/04/01 06:29:02 INFO TaskSetManager: Starting task 127.0 in stage 8.0 (TID 135) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:29:02 INFO TaskSetManager: Finished task 123.0 in stage 8.0 (TID 131) in 6508 ms on 10.0.0.133 (executor driver) (124/208) +26/04/01 06:29:02 INFO Executor: Running task 127.0 in stage 8.0 (TID 135) +26/04/01 06:29:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:08 INFO Executor: Finished task 124.0 in stage 8.0 (TID 132). 6787 bytes result sent to driver +26/04/01 06:29:08 INFO TaskSetManager: Starting task 128.0 in stage 8.0 (TID 136) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:08 INFO TaskSetManager: Finished task 124.0 in stage 8.0 (TID 132) in 6520 ms on 10.0.0.133 (executor driver) (125/208) +26/04/01 06:29:08 INFO Executor: Running task 128.0 in stage 8.0 (TID 136) +26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:08 INFO Executor: Finished task 126.0 in stage 8.0 (TID 134). 6787 bytes result sent to driver +26/04/01 06:29:08 INFO TaskSetManager: Starting task 129.0 in stage 8.0 (TID 137) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:08 INFO TaskSetManager: Finished task 126.0 in stage 8.0 (TID 134) in 6523 ms on 10.0.0.133 (executor driver) (126/208) +26/04/01 06:29:08 INFO Executor: Running task 129.0 in stage 8.0 (TID 137) +26/04/01 06:29:08 INFO Executor: Finished task 125.0 in stage 8.0 (TID 133). 6787 bytes result sent to driver +26/04/01 06:29:08 INFO TaskSetManager: Starting task 130.0 in stage 8.0 (TID 138) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:08 INFO TaskSetManager: Finished task 125.0 in stage 8.0 (TID 133) in 6526 ms on 10.0.0.133 (executor driver) (127/208) +26/04/01 06:29:08 INFO Executor: Running task 130.0 in stage 8.0 (TID 138) +26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:08 INFO Executor: Finished task 127.0 in stage 8.0 (TID 135). 6787 bytes result sent to driver +26/04/01 06:29:08 INFO TaskSetManager: Starting task 131.0 in stage 8.0 (TID 139) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:08 INFO TaskSetManager: Finished task 127.0 in stage 8.0 (TID 135) in 6529 ms on 10.0.0.133 (executor driver) (128/208) +26/04/01 06:29:08 INFO Executor: Running task 131.0 in stage 8.0 (TID 139) +26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:14 INFO Executor: Finished task 128.0 in stage 8.0 (TID 136). 6787 bytes result sent to driver +26/04/01 06:29:14 INFO TaskSetManager: Starting task 132.0 in stage 8.0 (TID 140) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:29:14 INFO TaskSetManager: Finished task 128.0 in stage 8.0 (TID 136) in 6530 ms on 10.0.0.133 (executor driver) (129/208) +26/04/01 06:29:14 INFO Executor: Running task 132.0 in stage 8.0 (TID 140) +26/04/01 06:29:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:15 INFO Executor: Finished task 129.0 in stage 8.0 (TID 137). 6787 bytes result sent to driver +26/04/01 06:29:15 INFO Executor: Finished task 130.0 in stage 8.0 (TID 138). 6787 bytes result sent to driver +26/04/01 06:29:15 INFO TaskSetManager: Starting task 133.0 in stage 8.0 (TID 141) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:29:15 INFO Executor: Running task 133.0 in stage 8.0 (TID 141) +26/04/01 06:29:15 INFO TaskSetManager: Starting task 134.0 in stage 8.0 (TID 142) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:15 INFO TaskSetManager: Finished task 129.0 in stage 8.0 (TID 137) in 6528 ms on 10.0.0.133 (executor driver) (130/208) +26/04/01 06:29:15 INFO Executor: Running task 134.0 in stage 8.0 (TID 142) +26/04/01 06:29:15 INFO TaskSetManager: Finished task 130.0 in stage 8.0 (TID 138) in 6528 ms on 10.0.0.133 (executor driver) (131/208) +26/04/01 06:29:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:15 INFO Executor: Finished task 131.0 in stage 8.0 (TID 139). 6787 bytes result sent to driver +26/04/01 06:29:15 INFO TaskSetManager: Starting task 135.0 in stage 8.0 (TID 143) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:15 INFO TaskSetManager: Finished task 131.0 in stage 8.0 (TID 139) in 6524 ms on 10.0.0.133 (executor driver) (132/208) +26/04/01 06:29:15 INFO Executor: Running task 135.0 in stage 8.0 (TID 143) +26/04/01 06:29:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:21 INFO Executor: Finished task 132.0 in stage 8.0 (TID 140). 6787 bytes result sent to driver +26/04/01 06:29:21 INFO TaskSetManager: Starting task 136.0 in stage 8.0 (TID 144) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:21 INFO TaskSetManager: Finished task 132.0 in stage 8.0 (TID 140) in 6477 ms on 10.0.0.133 (executor driver) (133/208) +26/04/01 06:29:21 INFO Executor: Running task 136.0 in stage 8.0 (TID 144) +26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:21 INFO Executor: Finished task 133.0 in stage 8.0 (TID 141). 6787 bytes result sent to driver +26/04/01 06:29:21 INFO TaskSetManager: Starting task 137.0 in stage 8.0 (TID 145) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:21 INFO TaskSetManager: Finished task 133.0 in stage 8.0 (TID 141) in 6478 ms on 10.0.0.133 (executor driver) (134/208) +26/04/01 06:29:21 INFO Executor: Running task 137.0 in stage 8.0 (TID 145) +26/04/01 06:29:21 INFO Executor: Finished task 134.0 in stage 8.0 (TID 142). 6787 bytes result sent to driver +26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:21 INFO TaskSetManager: Starting task 138.0 in stage 8.0 (TID 146) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:29:21 INFO TaskSetManager: Finished task 134.0 in stage 8.0 (TID 142) in 6479 ms on 10.0.0.133 (executor driver) (135/208) +26/04/01 06:29:21 INFO Executor: Running task 138.0 in stage 8.0 (TID 146) +26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:21 INFO Executor: Finished task 135.0 in stage 8.0 (TID 143). 6787 bytes result sent to driver +26/04/01 06:29:21 INFO TaskSetManager: Starting task 139.0 in stage 8.0 (TID 147) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:29:21 INFO Executor: Running task 139.0 in stage 8.0 (TID 147) +26/04/01 06:29:21 INFO TaskSetManager: Finished task 135.0 in stage 8.0 (TID 143) in 6480 ms on 10.0.0.133 (executor driver) (136/208) +26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:27 INFO Executor: Finished task 136.0 in stage 8.0 (TID 144). 6787 bytes result sent to driver +26/04/01 06:29:27 INFO TaskSetManager: Starting task 140.0 in stage 8.0 (TID 148) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:27 INFO TaskSetManager: Finished task 136.0 in stage 8.0 (TID 144) in 6474 ms on 10.0.0.133 (executor driver) (137/208) +26/04/01 06:29:27 INFO Executor: Running task 140.0 in stage 8.0 (TID 148) +26/04/01 06:29:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:28 INFO Executor: Finished task 137.0 in stage 8.0 (TID 145). 6787 bytes result sent to driver +26/04/01 06:29:28 INFO TaskSetManager: Starting task 141.0 in stage 8.0 (TID 149) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:28 INFO TaskSetManager: Finished task 137.0 in stage 8.0 (TID 145) in 6473 ms on 10.0.0.133 (executor driver) (138/208) +26/04/01 06:29:28 INFO Executor: Running task 141.0 in stage 8.0 (TID 149) +26/04/01 06:29:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:28 INFO Executor: Finished task 138.0 in stage 8.0 (TID 146). 6787 bytes result sent to driver +26/04/01 06:29:28 INFO TaskSetManager: Starting task 142.0 in stage 8.0 (TID 150) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:28 INFO TaskSetManager: Finished task 138.0 in stage 8.0 (TID 146) in 6477 ms on 10.0.0.133 (executor driver) (139/208) +26/04/01 06:29:28 INFO Executor: Running task 142.0 in stage 8.0 (TID 150) +26/04/01 06:29:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:28 INFO Executor: Finished task 139.0 in stage 8.0 (TID 147). 6787 bytes result sent to driver +26/04/01 06:29:28 INFO TaskSetManager: Starting task 143.0 in stage 8.0 (TID 151) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:28 INFO TaskSetManager: Finished task 139.0 in stage 8.0 (TID 147) in 6463 ms on 10.0.0.133 (executor driver) (140/208) +26/04/01 06:29:28 INFO Executor: Running task 143.0 in stage 8.0 (TID 151) +26/04/01 06:29:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:34 INFO Executor: Finished task 140.0 in stage 8.0 (TID 148). 6787 bytes result sent to driver +26/04/01 06:29:34 INFO TaskSetManager: Starting task 144.0 in stage 8.0 (TID 152) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:29:34 INFO TaskSetManager: Finished task 140.0 in stage 8.0 (TID 148) in 6443 ms on 10.0.0.133 (executor driver) (141/208) +26/04/01 06:29:34 INFO Executor: Running task 144.0 in stage 8.0 (TID 152) +26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:34 INFO Executor: Finished task 141.0 in stage 8.0 (TID 149). 6787 bytes result sent to driver +26/04/01 06:29:34 INFO TaskSetManager: Starting task 145.0 in stage 8.0 (TID 153) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:29:34 INFO Executor: Running task 145.0 in stage 8.0 (TID 153) +26/04/01 06:29:34 INFO TaskSetManager: Finished task 141.0 in stage 8.0 (TID 149) in 6433 ms on 10.0.0.133 (executor driver) (142/208) +26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:34 INFO Executor: Finished task 142.0 in stage 8.0 (TID 150). 6787 bytes result sent to driver +26/04/01 06:29:34 INFO TaskSetManager: Starting task 146.0 in stage 8.0 (TID 154) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:29:34 INFO Executor: Running task 146.0 in stage 8.0 (TID 154) +26/04/01 06:29:34 INFO TaskSetManager: Finished task 142.0 in stage 8.0 (TID 150) in 6439 ms on 10.0.0.133 (executor driver) (143/208) +26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:34 INFO Executor: Finished task 143.0 in stage 8.0 (TID 151). 6787 bytes result sent to driver +26/04/01 06:29:34 INFO TaskSetManager: Starting task 147.0 in stage 8.0 (TID 155) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:29:34 INFO TaskSetManager: Finished task 143.0 in stage 8.0 (TID 151) in 6428 ms on 10.0.0.133 (executor driver) (144/208) +26/04/01 06:29:34 INFO Executor: Running task 147.0 in stage 8.0 (TID 155) +26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:40 INFO Executor: Finished task 144.0 in stage 8.0 (TID 152). 6787 bytes result sent to driver +26/04/01 06:29:40 INFO TaskSetManager: Starting task 148.0 in stage 8.0 (TID 156) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:29:40 INFO TaskSetManager: Finished task 144.0 in stage 8.0 (TID 152) in 6500 ms on 10.0.0.133 (executor driver) (145/208) +26/04/01 06:29:40 INFO Executor: Running task 148.0 in stage 8.0 (TID 156) +26/04/01 06:29:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:41 INFO Executor: Finished task 146.0 in stage 8.0 (TID 154). 6787 bytes result sent to driver +26/04/01 06:29:41 INFO TaskSetManager: Starting task 149.0 in stage 8.0 (TID 157) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:29:41 INFO Executor: Finished task 145.0 in stage 8.0 (TID 153). 6787 bytes result sent to driver +26/04/01 06:29:41 INFO TaskSetManager: Finished task 146.0 in stage 8.0 (TID 154) in 6502 ms on 10.0.0.133 (executor driver) (146/208) +26/04/01 06:29:41 INFO Executor: Running task 149.0 in stage 8.0 (TID 157) +26/04/01 06:29:41 INFO TaskSetManager: Starting task 150.0 in stage 8.0 (TID 158) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:29:41 INFO TaskSetManager: Finished task 145.0 in stage 8.0 (TID 153) in 6513 ms on 10.0.0.133 (executor driver) (147/208) +26/04/01 06:29:41 INFO Executor: Running task 150.0 in stage 8.0 (TID 158) +26/04/01 06:29:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:41 INFO Executor: Finished task 147.0 in stage 8.0 (TID 155). 6787 bytes result sent to driver +26/04/01 06:29:41 INFO TaskSetManager: Starting task 151.0 in stage 8.0 (TID 159) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:29:41 INFO TaskSetManager: Finished task 147.0 in stage 8.0 (TID 155) in 6523 ms on 10.0.0.133 (executor driver) (148/208) +26/04/01 06:29:41 INFO Executor: Running task 151.0 in stage 8.0 (TID 159) +26/04/01 06:29:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:47 INFO Executor: Finished task 148.0 in stage 8.0 (TID 156). 6787 bytes result sent to driver +26/04/01 06:29:47 INFO TaskSetManager: Starting task 152.0 in stage 8.0 (TID 160) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:47 INFO TaskSetManager: Finished task 148.0 in stage 8.0 (TID 156) in 6532 ms on 10.0.0.133 (executor driver) (149/208) +26/04/01 06:29:47 INFO Executor: Running task 152.0 in stage 8.0 (TID 160) +26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:47 INFO Executor: Finished task 150.0 in stage 8.0 (TID 158). 6787 bytes result sent to driver +26/04/01 06:29:47 INFO Executor: Finished task 149.0 in stage 8.0 (TID 157). 6787 bytes result sent to driver +26/04/01 06:29:47 INFO TaskSetManager: Starting task 153.0 in stage 8.0 (TID 161) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:47 INFO TaskSetManager: Starting task 154.0 in stage 8.0 (TID 162) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:47 INFO Executor: Running task 153.0 in stage 8.0 (TID 161) +26/04/01 06:29:47 INFO TaskSetManager: Finished task 150.0 in stage 8.0 (TID 158) in 6525 ms on 10.0.0.133 (executor driver) (150/208) +26/04/01 06:29:47 INFO Executor: Running task 154.0 in stage 8.0 (TID 162) +26/04/01 06:29:47 INFO TaskSetManager: Finished task 149.0 in stage 8.0 (TID 157) in 6526 ms on 10.0.0.133 (executor driver) (151/208) +26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:47 INFO Executor: Finished task 151.0 in stage 8.0 (TID 159). 6787 bytes result sent to driver +26/04/01 06:29:47 INFO TaskSetManager: Starting task 155.0 in stage 8.0 (TID 163) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:47 INFO TaskSetManager: Finished task 151.0 in stage 8.0 (TID 159) in 6528 ms on 10.0.0.133 (executor driver) (152/208) +26/04/01 06:29:47 INFO Executor: Running task 155.0 in stage 8.0 (TID 163) +26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:53 INFO Executor: Finished task 152.0 in stage 8.0 (TID 160). 6787 bytes result sent to driver +26/04/01 06:29:53 INFO TaskSetManager: Starting task 156.0 in stage 8.0 (TID 164) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:29:53 INFO TaskSetManager: Finished task 152.0 in stage 8.0 (TID 160) in 6544 ms on 10.0.0.133 (executor driver) (153/208) +26/04/01 06:29:53 INFO Executor: Running task 156.0 in stage 8.0 (TID 164) +26/04/01 06:29:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:54 INFO Executor: Finished task 153.0 in stage 8.0 (TID 161). 6787 bytes result sent to driver +26/04/01 06:29:54 INFO TaskSetManager: Starting task 157.0 in stage 8.0 (TID 165) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:29:54 INFO TaskSetManager: Finished task 153.0 in stage 8.0 (TID 161) in 6550 ms on 10.0.0.133 (executor driver) (154/208) +26/04/01 06:29:54 INFO Executor: Running task 157.0 in stage 8.0 (TID 165) +26/04/01 06:29:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:54 INFO Executor: Finished task 154.0 in stage 8.0 (TID 162). 6787 bytes result sent to driver +26/04/01 06:29:54 INFO TaskSetManager: Starting task 158.0 in stage 8.0 (TID 166) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:54 INFO TaskSetManager: Finished task 154.0 in stage 8.0 (TID 162) in 6552 ms on 10.0.0.133 (executor driver) (155/208) +26/04/01 06:29:54 INFO Executor: Running task 158.0 in stage 8.0 (TID 166) +26/04/01 06:29:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:54 INFO Executor: Finished task 155.0 in stage 8.0 (TID 163). 6787 bytes result sent to driver +26/04/01 06:29:54 INFO TaskSetManager: Starting task 159.0 in stage 8.0 (TID 167) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:29:54 INFO TaskSetManager: Finished task 155.0 in stage 8.0 (TID 163) in 6551 ms on 10.0.0.133 (executor driver) (156/208) +26/04/01 06:29:54 INFO Executor: Running task 159.0 in stage 8.0 (TID 167) +26/04/01 06:29:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:29:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:00 INFO Executor: Finished task 156.0 in stage 8.0 (TID 164). 6787 bytes result sent to driver +26/04/01 06:30:00 INFO TaskSetManager: Starting task 160.0 in stage 8.0 (TID 168) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:30:00 INFO Executor: Running task 160.0 in stage 8.0 (TID 168) +26/04/01 06:30:00 INFO TaskSetManager: Finished task 156.0 in stage 8.0 (TID 164) in 6472 ms on 10.0.0.133 (executor driver) (157/208) +26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:00 INFO Executor: Finished task 157.0 in stage 8.0 (TID 165). 6787 bytes result sent to driver +26/04/01 06:30:00 INFO TaskSetManager: Starting task 161.0 in stage 8.0 (TID 169) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:30:00 INFO Executor: Running task 161.0 in stage 8.0 (TID 169) +26/04/01 06:30:00 INFO TaskSetManager: Finished task 157.0 in stage 8.0 (TID 165) in 6469 ms on 10.0.0.133 (executor driver) (158/208) +26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:00 INFO Executor: Finished task 158.0 in stage 8.0 (TID 166). 6787 bytes result sent to driver +26/04/01 06:30:00 INFO TaskSetManager: Starting task 162.0 in stage 8.0 (TID 170) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:30:00 INFO TaskSetManager: Finished task 158.0 in stage 8.0 (TID 166) in 6471 ms on 10.0.0.133 (executor driver) (159/208) +26/04/01 06:30:00 INFO Executor: Running task 162.0 in stage 8.0 (TID 170) +26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:00 INFO Executor: Finished task 159.0 in stage 8.0 (TID 167). 6787 bytes result sent to driver +26/04/01 06:30:00 INFO TaskSetManager: Starting task 163.0 in stage 8.0 (TID 171) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:30:00 INFO TaskSetManager: Finished task 159.0 in stage 8.0 (TID 167) in 6469 ms on 10.0.0.133 (executor driver) (160/208) +26/04/01 06:30:00 INFO Executor: Running task 163.0 in stage 8.0 (TID 171) +26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:06 INFO Executor: Finished task 160.0 in stage 8.0 (TID 168). 6787 bytes result sent to driver +26/04/01 06:30:06 INFO TaskSetManager: Starting task 164.0 in stage 8.0 (TID 172) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:30:06 INFO Executor: Running task 164.0 in stage 8.0 (TID 172) +26/04/01 06:30:06 INFO TaskSetManager: Finished task 160.0 in stage 8.0 (TID 168) in 6477 ms on 10.0.0.133 (executor driver) (161/208) +26/04/01 06:30:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:07 INFO Executor: Finished task 161.0 in stage 8.0 (TID 169). 6787 bytes result sent to driver +26/04/01 06:30:07 INFO TaskSetManager: Starting task 165.0 in stage 8.0 (TID 173) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:30:07 INFO TaskSetManager: Finished task 161.0 in stage 8.0 (TID 169) in 6475 ms on 10.0.0.133 (executor driver) (162/208) +26/04/01 06:30:07 INFO Executor: Running task 165.0 in stage 8.0 (TID 173) +26/04/01 06:30:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:07 INFO Executor: Finished task 162.0 in stage 8.0 (TID 170). 6787 bytes result sent to driver +26/04/01 06:30:07 INFO TaskSetManager: Starting task 166.0 in stage 8.0 (TID 174) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:30:07 INFO Executor: Running task 166.0 in stage 8.0 (TID 174) +26/04/01 06:30:07 INFO TaskSetManager: Finished task 162.0 in stage 8.0 (TID 170) in 6494 ms on 10.0.0.133 (executor driver) (163/208) +26/04/01 06:30:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:07 INFO Executor: Finished task 163.0 in stage 8.0 (TID 171). 6787 bytes result sent to driver +26/04/01 06:30:07 INFO TaskSetManager: Starting task 167.0 in stage 8.0 (TID 175) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:30:07 INFO TaskSetManager: Finished task 163.0 in stage 8.0 (TID 171) in 6497 ms on 10.0.0.133 (executor driver) (164/208) +26/04/01 06:30:07 INFO Executor: Running task 167.0 in stage 8.0 (TID 175) +26/04/01 06:30:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:13 INFO Executor: Finished task 164.0 in stage 8.0 (TID 172). 6873 bytes result sent to driver +26/04/01 06:30:13 INFO TaskSetManager: Starting task 168.0 in stage 8.0 (TID 176) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:30:13 INFO TaskSetManager: Finished task 164.0 in stage 8.0 (TID 172) in 6502 ms on 10.0.0.133 (executor driver) (165/208) +26/04/01 06:30:13 INFO Executor: Running task 168.0 in stage 8.0 (TID 176) +26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:13 INFO Executor: Finished task 165.0 in stage 8.0 (TID 173). 6830 bytes result sent to driver +26/04/01 06:30:13 INFO TaskSetManager: Starting task 169.0 in stage 8.0 (TID 177) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:30:13 INFO TaskSetManager: Finished task 165.0 in stage 8.0 (TID 173) in 6500 ms on 10.0.0.133 (executor driver) (166/208) +26/04/01 06:30:13 INFO Executor: Running task 169.0 in stage 8.0 (TID 177) +26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:13 INFO Executor: Finished task 166.0 in stage 8.0 (TID 174). 6830 bytes result sent to driver +26/04/01 06:30:13 INFO TaskSetManager: Starting task 170.0 in stage 8.0 (TID 178) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:30:13 INFO TaskSetManager: Finished task 166.0 in stage 8.0 (TID 174) in 6478 ms on 10.0.0.133 (executor driver) (167/208) +26/04/01 06:30:13 INFO Executor: Running task 170.0 in stage 8.0 (TID 178) +26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:13 INFO Executor: Finished task 167.0 in stage 8.0 (TID 175). 6830 bytes result sent to driver +26/04/01 06:30:13 INFO TaskSetManager: Starting task 171.0 in stage 8.0 (TID 179) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:30:13 INFO Executor: Running task 171.0 in stage 8.0 (TID 179) +26/04/01 06:30:13 INFO TaskSetManager: Finished task 167.0 in stage 8.0 (TID 175) in 6481 ms on 10.0.0.133 (executor driver) (168/208) +26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:19 INFO Executor: Finished task 168.0 in stage 8.0 (TID 176). 6787 bytes result sent to driver +26/04/01 06:30:19 INFO TaskSetManager: Starting task 172.0 in stage 8.0 (TID 180) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:30:19 INFO TaskSetManager: Finished task 168.0 in stage 8.0 (TID 176) in 6486 ms on 10.0.0.133 (executor driver) (169/208) +26/04/01 06:30:19 INFO Executor: Running task 172.0 in stage 8.0 (TID 180) +26/04/01 06:30:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:20 INFO Executor: Finished task 170.0 in stage 8.0 (TID 178). 6787 bytes result sent to driver +26/04/01 06:30:20 INFO TaskSetManager: Starting task 173.0 in stage 8.0 (TID 181) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:30:20 INFO TaskSetManager: Finished task 170.0 in stage 8.0 (TID 178) in 6486 ms on 10.0.0.133 (executor driver) (170/208) +26/04/01 06:30:20 INFO Executor: Running task 173.0 in stage 8.0 (TID 181) +26/04/01 06:30:20 INFO Executor: Finished task 169.0 in stage 8.0 (TID 177). 6787 bytes result sent to driver +26/04/01 06:30:20 INFO TaskSetManager: Starting task 174.0 in stage 8.0 (TID 182) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:30:20 INFO TaskSetManager: Finished task 169.0 in stage 8.0 (TID 177) in 6489 ms on 10.0.0.133 (executor driver) (171/208) +26/04/01 06:30:20 INFO Executor: Running task 174.0 in stage 8.0 (TID 182) +26/04/01 06:30:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:20 INFO Executor: Finished task 171.0 in stage 8.0 (TID 179). 6787 bytes result sent to driver +26/04/01 06:30:20 INFO TaskSetManager: Starting task 175.0 in stage 8.0 (TID 183) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:30:20 INFO TaskSetManager: Finished task 171.0 in stage 8.0 (TID 179) in 6483 ms on 10.0.0.133 (executor driver) (172/208) +26/04/01 06:30:20 INFO Executor: Running task 175.0 in stage 8.0 (TID 183) +26/04/01 06:30:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:26 INFO Executor: Finished task 172.0 in stage 8.0 (TID 180). 6787 bytes result sent to driver +26/04/01 06:30:26 INFO TaskSetManager: Starting task 176.0 in stage 8.0 (TID 184) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:30:26 INFO TaskSetManager: Finished task 172.0 in stage 8.0 (TID 180) in 6501 ms on 10.0.0.133 (executor driver) (173/208) +26/04/01 06:30:26 INFO Executor: Running task 176.0 in stage 8.0 (TID 184) +26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:26 INFO Executor: Finished task 173.0 in stage 8.0 (TID 181). 6787 bytes result sent to driver +26/04/01 06:30:26 INFO TaskSetManager: Starting task 177.0 in stage 8.0 (TID 185) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:30:26 INFO TaskSetManager: Finished task 173.0 in stage 8.0 (TID 181) in 6503 ms on 10.0.0.133 (executor driver) (174/208) +26/04/01 06:30:26 INFO Executor: Running task 177.0 in stage 8.0 (TID 185) +26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:26 INFO Executor: Finished task 174.0 in stage 8.0 (TID 182). 6787 bytes result sent to driver +26/04/01 06:30:26 INFO TaskSetManager: Starting task 178.0 in stage 8.0 (TID 186) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:30:26 INFO Executor: Running task 178.0 in stage 8.0 (TID 186) +26/04/01 06:30:26 INFO TaskSetManager: Finished task 174.0 in stage 8.0 (TID 182) in 6522 ms on 10.0.0.133 (executor driver) (175/208) +26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:26 INFO Executor: Finished task 175.0 in stage 8.0 (TID 183). 6787 bytes result sent to driver +26/04/01 06:30:26 INFO TaskSetManager: Starting task 179.0 in stage 8.0 (TID 187) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:30:26 INFO TaskSetManager: Finished task 175.0 in stage 8.0 (TID 183) in 6523 ms on 10.0.0.133 (executor driver) (176/208) +26/04/01 06:30:26 INFO Executor: Running task 179.0 in stage 8.0 (TID 187) +26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:32 INFO Executor: Finished task 176.0 in stage 8.0 (TID 184). 6787 bytes result sent to driver +26/04/01 06:30:32 INFO TaskSetManager: Starting task 180.0 in stage 8.0 (TID 188) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:30:32 INFO Executor: Running task 180.0 in stage 8.0 (TID 188) +26/04/01 06:30:32 INFO TaskSetManager: Finished task 176.0 in stage 8.0 (TID 184) in 6525 ms on 10.0.0.133 (executor driver) (177/208) +26/04/01 06:30:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:33 INFO Executor: Finished task 177.0 in stage 8.0 (TID 185). 6787 bytes result sent to driver +26/04/01 06:30:33 INFO Executor: Finished task 178.0 in stage 8.0 (TID 186). 6787 bytes result sent to driver +26/04/01 06:30:33 INFO TaskSetManager: Starting task 181.0 in stage 8.0 (TID 189) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:30:33 INFO TaskSetManager: Starting task 182.0 in stage 8.0 (TID 190) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:30:33 INFO Executor: Running task 181.0 in stage 8.0 (TID 189) +26/04/01 06:30:33 INFO TaskSetManager: Finished task 177.0 in stage 8.0 (TID 185) in 6527 ms on 10.0.0.133 (executor driver) (178/208) +26/04/01 06:30:33 INFO Executor: Running task 182.0 in stage 8.0 (TID 190) +26/04/01 06:30:33 INFO TaskSetManager: Finished task 178.0 in stage 8.0 (TID 186) in 6508 ms on 10.0.0.133 (executor driver) (179/208) +26/04/01 06:30:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:33 INFO Executor: Finished task 179.0 in stage 8.0 (TID 187). 6787 bytes result sent to driver +26/04/01 06:30:33 INFO TaskSetManager: Starting task 183.0 in stage 8.0 (TID 191) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:30:33 INFO TaskSetManager: Finished task 179.0 in stage 8.0 (TID 187) in 6505 ms on 10.0.0.133 (executor driver) (180/208) +26/04/01 06:30:33 INFO Executor: Running task 183.0 in stage 8.0 (TID 191) +26/04/01 06:30:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:39 INFO Executor: Finished task 180.0 in stage 8.0 (TID 188). 6787 bytes result sent to driver +26/04/01 06:30:39 INFO TaskSetManager: Starting task 184.0 in stage 8.0 (TID 192) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:30:39 INFO TaskSetManager: Finished task 180.0 in stage 8.0 (TID 188) in 6512 ms on 10.0.0.133 (executor driver) (181/208) +26/04/01 06:30:39 INFO Executor: Running task 184.0 in stage 8.0 (TID 192) +26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:39 INFO Executor: Finished task 182.0 in stage 8.0 (TID 190). 6787 bytes result sent to driver +26/04/01 06:30:39 INFO TaskSetManager: Starting task 185.0 in stage 8.0 (TID 193) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:30:39 INFO TaskSetManager: Finished task 182.0 in stage 8.0 (TID 190) in 6512 ms on 10.0.0.133 (executor driver) (182/208) +26/04/01 06:30:39 INFO Executor: Running task 185.0 in stage 8.0 (TID 193) +26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:39 INFO Executor: Finished task 181.0 in stage 8.0 (TID 189). 6787 bytes result sent to driver +26/04/01 06:30:39 INFO TaskSetManager: Starting task 186.0 in stage 8.0 (TID 194) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:30:39 INFO Executor: Running task 186.0 in stage 8.0 (TID 194) +26/04/01 06:30:39 INFO TaskSetManager: Finished task 181.0 in stage 8.0 (TID 189) in 6517 ms on 10.0.0.133 (executor driver) (183/208) +26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:39 INFO Executor: Finished task 183.0 in stage 8.0 (TID 191). 6787 bytes result sent to driver +26/04/01 06:30:39 INFO TaskSetManager: Starting task 187.0 in stage 8.0 (TID 195) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:30:39 INFO Executor: Running task 187.0 in stage 8.0 (TID 195) +26/04/01 06:30:39 INFO TaskSetManager: Finished task 183.0 in stage 8.0 (TID 191) in 6513 ms on 10.0.0.133 (executor driver) (184/208) +26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:45 INFO Executor: Finished task 184.0 in stage 8.0 (TID 192). 6787 bytes result sent to driver +26/04/01 06:30:45 INFO TaskSetManager: Starting task 188.0 in stage 8.0 (TID 196) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:30:45 INFO Executor: Running task 188.0 in stage 8.0 (TID 196) +26/04/01 06:30:45 INFO TaskSetManager: Finished task 184.0 in stage 8.0 (TID 192) in 6520 ms on 10.0.0.133 (executor driver) (185/208) +26/04/01 06:30:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:46 INFO Executor: Finished task 185.0 in stage 8.0 (TID 193). 6787 bytes result sent to driver +26/04/01 06:30:46 INFO TaskSetManager: Starting task 189.0 in stage 8.0 (TID 197) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:30:46 INFO TaskSetManager: Finished task 185.0 in stage 8.0 (TID 193) in 6518 ms on 10.0.0.133 (executor driver) (186/208) +26/04/01 06:30:46 INFO Executor: Running task 189.0 in stage 8.0 (TID 197) +26/04/01 06:30:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:46 INFO Executor: Finished task 186.0 in stage 8.0 (TID 194). 6787 bytes result sent to driver +26/04/01 06:30:46 INFO TaskSetManager: Starting task 190.0 in stage 8.0 (TID 198) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:30:46 INFO Executor: Running task 190.0 in stage 8.0 (TID 198) +26/04/01 06:30:46 INFO TaskSetManager: Finished task 186.0 in stage 8.0 (TID 194) in 6514 ms on 10.0.0.133 (executor driver) (187/208) +26/04/01 06:30:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:46 INFO Executor: Finished task 187.0 in stage 8.0 (TID 195). 6787 bytes result sent to driver +26/04/01 06:30:46 INFO TaskSetManager: Starting task 191.0 in stage 8.0 (TID 199) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:30:46 INFO TaskSetManager: Finished task 187.0 in stage 8.0 (TID 195) in 6519 ms on 10.0.0.133 (executor driver) (188/208) +26/04/01 06:30:46 INFO Executor: Running task 191.0 in stage 8.0 (TID 199) +26/04/01 06:30:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:52 INFO Executor: Finished task 188.0 in stage 8.0 (TID 196). 6787 bytes result sent to driver +26/04/01 06:30:52 INFO TaskSetManager: Starting task 192.0 in stage 8.0 (TID 200) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:30:52 INFO TaskSetManager: Finished task 188.0 in stage 8.0 (TID 196) in 6523 ms on 10.0.0.133 (executor driver) (189/208) +26/04/01 06:30:52 INFO Executor: Running task 192.0 in stage 8.0 (TID 200) +26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:52 INFO Executor: Finished task 190.0 in stage 8.0 (TID 198). 6787 bytes result sent to driver +26/04/01 06:30:52 INFO Executor: Finished task 189.0 in stage 8.0 (TID 197). 6787 bytes result sent to driver +26/04/01 06:30:52 INFO TaskSetManager: Starting task 193.0 in stage 8.0 (TID 201) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:30:52 INFO Executor: Running task 193.0 in stage 8.0 (TID 201) +26/04/01 06:30:52 INFO TaskSetManager: Starting task 194.0 in stage 8.0 (TID 202) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:30:52 INFO TaskSetManager: Finished task 190.0 in stage 8.0 (TID 198) in 6525 ms on 10.0.0.133 (executor driver) (190/208) +26/04/01 06:30:52 INFO Executor: Running task 194.0 in stage 8.0 (TID 202) +26/04/01 06:30:52 INFO TaskSetManager: Finished task 189.0 in stage 8.0 (TID 197) in 6527 ms on 10.0.0.133 (executor driver) (191/208) +26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:52 INFO Executor: Finished task 191.0 in stage 8.0 (TID 199). 6787 bytes result sent to driver +26/04/01 06:30:52 INFO TaskSetManager: Starting task 195.0 in stage 8.0 (TID 203) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:30:52 INFO Executor: Running task 195.0 in stage 8.0 (TID 203) +26/04/01 06:30:52 INFO TaskSetManager: Finished task 191.0 in stage 8.0 (TID 199) in 6520 ms on 10.0.0.133 (executor driver) (192/208) +26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:56 INFO Executor: Finished task 192.0 in stage 8.0 (TID 200). 6787 bytes result sent to driver +26/04/01 06:30:56 INFO TaskSetManager: Starting task 196.0 in stage 8.0 (TID 204) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:30:56 INFO Executor: Running task 196.0 in stage 8.0 (TID 204) +26/04/01 06:30:56 INFO TaskSetManager: Finished task 192.0 in stage 8.0 (TID 200) in 4012 ms on 10.0.0.133 (executor driver) (193/208) +26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:56 INFO Executor: Finished task 193.0 in stage 8.0 (TID 201). 6787 bytes result sent to driver +26/04/01 06:30:56 INFO TaskSetManager: Starting task 197.0 in stage 8.0 (TID 205) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:30:56 INFO TaskSetManager: Finished task 193.0 in stage 8.0 (TID 201) in 4007 ms on 10.0.0.133 (executor driver) (194/208) +26/04/01 06:30:56 INFO Executor: Running task 197.0 in stage 8.0 (TID 205) +26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:56 INFO Executor: Finished task 194.0 in stage 8.0 (TID 202). 6787 bytes result sent to driver +26/04/01 06:30:56 INFO TaskSetManager: Starting task 198.0 in stage 8.0 (TID 206) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:30:56 INFO Executor: Running task 198.0 in stage 8.0 (TID 206) +26/04/01 06:30:56 INFO TaskSetManager: Finished task 194.0 in stage 8.0 (TID 202) in 4010 ms on 10.0.0.133 (executor driver) (195/208) +26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:56 INFO Executor: Finished task 195.0 in stage 8.0 (TID 203). 6787 bytes result sent to driver +26/04/01 06:30:56 INFO TaskSetManager: Starting task 199.0 in stage 8.0 (TID 207) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:30:56 INFO TaskSetManager: Finished task 195.0 in stage 8.0 (TID 203) in 4009 ms on 10.0.0.133 (executor driver) (196/208) +26/04/01 06:30:56 INFO Executor: Running task 199.0 in stage 8.0 (TID 207) +26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:00 INFO Executor: Finished task 196.0 in stage 8.0 (TID 204). 6787 bytes result sent to driver +26/04/01 06:31:00 INFO TaskSetManager: Starting task 200.0 in stage 8.0 (TID 208) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9346 bytes) +26/04/01 06:31:00 INFO Executor: Running task 200.0 in stage 8.0 (TID 208) +26/04/01 06:31:00 INFO TaskSetManager: Finished task 196.0 in stage 8.0 (TID 204) in 4012 ms on 10.0.0.133 (executor driver) (197/208) +26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:00 INFO Executor: Finished task 197.0 in stage 8.0 (TID 205). 6787 bytes result sent to driver +26/04/01 06:31:00 INFO Executor: Finished task 198.0 in stage 8.0 (TID 206). 6787 bytes result sent to driver +26/04/01 06:31:00 INFO TaskSetManager: Starting task 201.0 in stage 8.0 (TID 209) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:31:00 INFO TaskSetManager: Finished task 197.0 in stage 8.0 (TID 205) in 4017 ms on 10.0.0.133 (executor driver) (198/208) +26/04/01 06:31:00 INFO Executor: Running task 201.0 in stage 8.0 (TID 209) +26/04/01 06:31:00 INFO TaskSetManager: Starting task 202.0 in stage 8.0 (TID 210) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:31:00 INFO Executor: Running task 202.0 in stage 8.0 (TID 210) +26/04/01 06:31:00 INFO TaskSetManager: Finished task 198.0 in stage 8.0 (TID 206) in 4013 ms on 10.0.0.133 (executor driver) (199/208) +26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:00 INFO Executor: Finished task 199.0 in stage 8.0 (TID 207). 6787 bytes result sent to driver +26/04/01 06:31:00 INFO TaskSetManager: Starting task 203.0 in stage 8.0 (TID 211) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:31:00 INFO TaskSetManager: Finished task 199.0 in stage 8.0 (TID 207) in 4018 ms on 10.0.0.133 (executor driver) (200/208) +26/04/01 06:31:00 INFO Executor: Running task 203.0 in stage 8.0 (TID 211) +26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:04 INFO Executor: Finished task 200.0 in stage 8.0 (TID 208). 6787 bytes result sent to driver +26/04/01 06:31:04 INFO TaskSetManager: Starting task 204.0 in stage 8.0 (TID 212) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:31:04 INFO TaskSetManager: Finished task 200.0 in stage 8.0 (TID 208) in 4017 ms on 10.0.0.133 (executor driver) (201/208) +26/04/01 06:31:04 INFO Executor: Running task 204.0 in stage 8.0 (TID 212) +26/04/01 06:31:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:04 INFO Executor: Finished task 201.0 in stage 8.0 (TID 209). 6787 bytes result sent to driver +26/04/01 06:31:04 INFO TaskSetManager: Starting task 205.0 in stage 8.0 (TID 213) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:31:04 INFO Executor: Running task 205.0 in stage 8.0 (TID 213) +26/04/01 06:31:04 INFO TaskSetManager: Finished task 201.0 in stage 8.0 (TID 209) in 4018 ms on 10.0.0.133 (executor driver) (202/208) +26/04/01 06:31:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:04 INFO Executor: Finished task 202.0 in stage 8.0 (TID 210). 6787 bytes result sent to driver +26/04/01 06:31:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:04 INFO TaskSetManager: Starting task 206.0 in stage 8.0 (TID 214) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:31:04 INFO TaskSetManager: Finished task 202.0 in stage 8.0 (TID 210) in 4020 ms on 10.0.0.133 (executor driver) (203/208) +26/04/01 06:31:04 INFO Executor: Running task 206.0 in stage 8.0 (TID 214) +26/04/01 06:31:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:05 INFO Executor: Finished task 203.0 in stage 8.0 (TID 211). 6787 bytes result sent to driver +26/04/01 06:31:05 INFO TaskSetManager: Starting task 207.0 in stage 8.0 (TID 215) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:31:05 INFO TaskSetManager: Finished task 203.0 in stage 8.0 (TID 211) in 4020 ms on 10.0.0.133 (executor driver) (204/208) +26/04/01 06:31:05 INFO Executor: Running task 207.0 in stage 8.0 (TID 215) +26/04/01 06:31:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:08 INFO Executor: Finished task 204.0 in stage 8.0 (TID 212). 6787 bytes result sent to driver +26/04/01 06:31:08 INFO TaskSetManager: Finished task 204.0 in stage 8.0 (TID 212) in 3982 ms on 10.0.0.133 (executor driver) (205/208) +26/04/01 06:31:08 INFO Executor: Finished task 205.0 in stage 8.0 (TID 213). 6787 bytes result sent to driver +26/04/01 06:31:08 INFO TaskSetManager: Finished task 205.0 in stage 8.0 (TID 213) in 3978 ms on 10.0.0.133 (executor driver) (206/208) +26/04/01 06:31:08 INFO Executor: Finished task 206.0 in stage 8.0 (TID 214). 6787 bytes result sent to driver +26/04/01 06:31:08 INFO TaskSetManager: Finished task 206.0 in stage 8.0 (TID 214) in 3978 ms on 10.0.0.133 (executor driver) (207/208) +26/04/01 06:31:08 INFO Executor: Finished task 207.0 in stage 8.0 (TID 215). 6787 bytes result sent to driver +26/04/01 06:31:08 INFO TaskSetManager: Finished task 207.0 in stage 8.0 (TID 215) in 3957 ms on 10.0.0.133 (executor driver) (208/208) +26/04/01 06:31:08 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool +26/04/01 06:31:08 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 329.106 s +26/04/01 06:31:08 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:31:08 INFO DAGScheduler: running: Set() +26/04/01 06:31:08 INFO DAGScheduler: waiting: Set() +26/04/01 06:31:08 INFO DAGScheduler: failed: Set() +26/04/01 06:31:08 INFO ShufflePartitionsUtil: For shuffle(0), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 06:31:09 INFO BlockManagerInfo: Removed broadcast_9_piece0 on 10.0.0.133:58418 in memory (size: 10.1 KiB, free: 12.6 GiB) +26/04/01 06:31:09 INFO CodeGenerator: Code generated in 60.090958 ms +26/04/01 06:31:09 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:31:09 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:31:09 INFO DAGScheduler: Final stage: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:31:09 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 9) +26/04/01 06:31:09 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:09 INFO DAGScheduler: Submitting ResultStage 10 (MapPartitionsRDD[23] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:31:09 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 49.9 KiB, free 12.6 GiB) +26/04/01 06:31:09 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 10.4 KiB, free 12.6 GiB) +26/04/01 06:31:09 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:58418 (size: 10.4 KiB, free: 12.6 GiB) +26/04/01 06:31:09 INFO SparkContext: Created broadcast 10 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:09 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 10 (MapPartitionsRDD[23] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:31:09 INFO TaskSchedulerImpl: Adding task set 10.0 with 1 tasks resource profile 0 +26/04/01 06:31:09 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 216) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) +26/04/01 06:31:09 INFO Executor: Running task 0.0 in stage 10.0 (TID 216) +26/04/01 06:31:09 INFO ShuffleBlockFetcherIterator: Getting 208 (853.9 KiB) non-empty blocks including 208 (853.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:31:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms +26/04/01 06:31:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:09 INFO CodeGenerator: Code generated in 5.743666 ms +26/04/01 06:31:09 INFO Executor: Finished task 0.0 in stage 10.0 (TID 216). 5160 bytes result sent to driver +26/04/01 06:31:09 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 216) in 345 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:31:09 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool +26/04/01 06:31:09 INFO DAGScheduler: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.347 s +26/04/01 06:31:09 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:31:09 INFO TaskSchedulerImpl: Killing all running tasks in stage 10: Stage finished +26/04/01 06:31:09 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.351742 s +26/04/01 06:31:09 INFO DAGScheduler: Registering RDD 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 +26/04/01 06:31:09 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:31:09 INFO DAGScheduler: Final stage: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:31:09 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 11) +26/04/01 06:31:09 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:09 INFO DAGScheduler: Submitting ShuffleMapStage 12 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:31:09 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 51.7 KiB, free 12.6 GiB) +26/04/01 06:31:09 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 11.2 KiB, free 12.6 GiB) +26/04/01 06:31:09 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:58418 (size: 11.2 KiB, free: 12.6 GiB) +26/04/01 06:31:09 INFO SparkContext: Created broadcast 11 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:09 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 12 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:31:09 INFO TaskSchedulerImpl: Adding task set 12.0 with 1 tasks resource profile 0 +26/04/01 06:31:09 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 217) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9404 bytes) +26/04/01 06:31:09 INFO Executor: Running task 0.0 in stage 12.0 (TID 217) +26/04/01 06:31:09 INFO CodeGenerator: Code generated in 3.572958 ms +26/04/01 06:31:09 INFO ShuffleBlockFetcherIterator: Getting 208 (853.9 KiB) non-empty blocks including 208 (853.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:31:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:31:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:09 INFO BlockManagerInfo: Removed broadcast_10_piece0 on 10.0.0.133:58418 in memory (size: 10.4 KiB, free: 12.6 GiB) +26/04/01 06:31:09 INFO Executor: Finished task 0.0 in stage 12.0 (TID 217). 6609 bytes result sent to driver +26/04/01 06:31:09 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 217) in 311 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:31:09 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool +26/04/01 06:31:09 INFO DAGScheduler: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.314 s +26/04/01 06:31:09 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:31:09 INFO DAGScheduler: running: Set() +26/04/01 06:31:09 INFO DAGScheduler: waiting: Set() +26/04/01 06:31:09 INFO DAGScheduler: failed: Set() +26/04/01 06:31:09 INFO ShufflePartitionsUtil: For shuffle(1), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 06:31:09 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:31:09 INFO DAGScheduler: Got job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:31:09 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:31:09 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 14) +26/04/01 06:31:09 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:09 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:31:09 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 20.4 KiB, free 12.6 GiB) +26/04/01 06:31:09 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 9.0 KiB, free 12.6 GiB) +26/04/01 06:31:09 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:58418 (size: 9.0 KiB, free: 12.6 GiB) +26/04/01 06:31:09 INFO SparkContext: Created broadcast 12 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:09 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:31:09 INFO TaskSchedulerImpl: Adding task set 15.0 with 1 tasks resource profile 0 +26/04/01 06:31:09 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 218) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) +26/04/01 06:31:09 INFO Executor: Running task 0.0 in stage 15.0 (TID 218) +26/04/01 06:31:09 INFO ShuffleBlockFetcherIterator: Getting 1 (2.6 KiB) non-empty blocks including 1 (2.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:31:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:31:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:09 INFO Executor: Finished task 0.0 in stage 15.0 (TID 218). 5560 bytes result sent to driver +26/04/01 06:31:09 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 218) in 9 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:31:09 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool +26/04/01 06:31:09 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.011 s +26/04/01 06:31:09 INFO DAGScheduler: Job 11 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:31:09 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished +26/04/01 06:31:09 INFO DAGScheduler: Job 11 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.011849 s +26/04/01 06:31:09 INFO SparkContext: SparkContext is stopping with exitCode 0. +26/04/01 06:31:09 INFO CometDriverPlugin: CometDriverPlugin shutdown +26/04/01 06:31:09 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! +26/04/01 06:31:09 INFO MemoryStore: MemoryStore cleared +26/04/01 06:31:09 INFO BlockManager: BlockManager stopped +26/04/01 06:31:09 INFO BlockManagerMaster: BlockManagerMaster stopped +26/04/01 06:31:09 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! +26/04/01 06:31:09 INFO SparkContext: Successfully stopped SparkContext +26/04/01 06:31:09 INFO ShutdownHookManager: Shutdown hook called +26/04/01 06:31:09 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-3a879eef-8ce8-4940-9fe0-6c284ae0de62 +26/04/01 06:31:09 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-9d0b1c01-f2bb-4fb8-875a-59c49514dd0e +26/04/01 06:31:09 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-9d0b1c01-f2bb-4fb8-875a-59c49514dd0e/pyspark-62c23908-1d1c-43ec-8349-112409da8991 + 335.72 real 1314.97 user 5.43 sys + 697319424 maximum resident set size + 0 average shared memory size + 0 average unshared data size + 0 average unshared stack size + 68500 page reclaims + 50 page faults + 0 swaps + 0 block input operations + 0 block output operations + 1275 messages sent + 2361 messages received + 17 signals received + 35609 voluntary context switches + 165249 involuntary context switches + 20342899865196 instructions retired + 4796070096872 cycles elapsed + 579110544 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.log b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.log new file mode 100644 index 0000000000..f3cdd0b5c4 --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.log @@ -0,0 +1,317 @@ +Registering table customer from /opt/tpch/sf100/customer +Registering table lineitem from /opt/tpch/sf100/lineitem +Registering table nation from /opt/tpch/sf100/nation +Registering table orders from /opt/tpch/sf100/orders +Registering table part from /opt/tpch/sf100/part +Registering table partsupp from /opt/tpch/sf100/partsupp +Registering table region from /opt/tpch/sf100/region +Registering table supplier from /opt/tpch/sf100/supplier + +============================================================ +Starting iteration 1 of 1 +============================================================ + +Running query 5 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q5.sql +Executing: -- CometBench-H query 5 derived from TPC-H query 5 under the terms of the TPC Fair Use Policy. +-- TP... +== Physical Plan == +AdaptiveSparkPlan (44) ++- CometSort (43) + +- CometExchange (42) + +- CometHashAggregate (41) + +- CometExchange (40) + +- CometHashAggregate (39) + +- CometProject (38) + +- CometBroadcastHashJoin (37) + :- CometProject (32) + : +- CometBroadcastHashJoin (31) + : :- CometProject (27) + : : +- CometSortMergeJoin (26) + : : :- CometSort (21) + : : : +- CometExchange (20) + : : : +- CometProject (19) + : : : +- CometSortMergeJoin (18) + : : : :- CometSort (13) + : : : : +- CometExchange (12) + : : : : +- CometProject (11) + : : : : +- CometSortMergeJoin (10) + : : : : :- CometSort (4) + : : : : : +- CometExchange (3) + : : : : : +- CometFilter (2) + : : : : : +- CometNativeScan parquet (1) + : : : : +- CometSort (9) + : : : : +- CometExchange (8) + : : : : +- CometProject (7) + : : : : +- CometFilter (6) + : : : : +- CometNativeScan parquet (5) + : : : +- CometSort (17) + : : : +- CometExchange (16) + : : : +- CometFilter (15) + : : : +- CometNativeScan parquet (14) + : : +- CometSort (25) + : : +- CometExchange (24) + : : +- CometFilter (23) + : : +- CometNativeScan parquet (22) + : +- CometBroadcastExchange (30) + : +- CometFilter (29) + : +- CometNativeScan parquet (28) + +- CometBroadcastExchange (36) + +- CometProject (35) + +- CometFilter (34) + +- CometNativeScan parquet (33) + + +(1) CometNativeScan parquet +Output [2]: [c_custkey#0L, c_nationkey#3L] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/customer] +PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] +ReadSchema: struct + +(2) CometFilter +Input [2]: [c_custkey#0L, c_nationkey#3L] +Condition : (isnotnull(c_custkey#0L) AND isnotnull(c_nationkey#3L)) + +(3) CometExchange +Input [2]: [c_custkey#0L, c_nationkey#3L] +Arguments: hashpartitioning(c_custkey#0L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=412] + +(4) CometSort +Input [2]: [c_custkey#0L, c_nationkey#3L] +Arguments: [c_custkey#0L, c_nationkey#3L], [c_custkey#0L ASC NULLS FIRST] + +(5) CometNativeScan parquet +Output [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/orders] +PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1994-01-01), LessThan(o_orderdate,1995-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] +ReadSchema: struct + +(6) CometFilter +Input [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] +Condition : ((((isnotnull(o_orderdate#60) AND (o_orderdate#60 >= 1994-01-01)) AND (o_orderdate#60 < 1995-01-01)) AND isnotnull(o_custkey#57L)) AND isnotnull(o_orderkey#56L)) + +(7) CometProject +Input [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] +Arguments: [o_orderkey#56L, o_custkey#57L], [o_orderkey#56L, o_custkey#57L] + +(8) CometExchange +Input [2]: [o_orderkey#56L, o_custkey#57L] +Arguments: hashpartitioning(o_custkey#57L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=416] + +(9) CometSort +Input [2]: [o_orderkey#56L, o_custkey#57L] +Arguments: [o_orderkey#56L, o_custkey#57L], [o_custkey#57L ASC NULLS FIRST] + +(10) CometSortMergeJoin +Left output [2]: [c_custkey#0L, c_nationkey#3L] +Right output [2]: [o_orderkey#56L, o_custkey#57L] +Arguments: [c_custkey#0L], [o_custkey#57L], Inner + +(11) CometProject +Input [4]: [c_custkey#0L, c_nationkey#3L, o_orderkey#56L, o_custkey#57L] +Arguments: [c_nationkey#3L, o_orderkey#56L], [c_nationkey#3L, o_orderkey#56L] + +(12) CometExchange +Input [2]: [c_nationkey#3L, o_orderkey#56L] +Arguments: hashpartitioning(o_orderkey#56L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=420] + +(13) CometSort +Input [2]: [c_nationkey#3L, o_orderkey#56L] +Arguments: [c_nationkey#3L, o_orderkey#56L], [o_orderkey#56L ASC NULLS FIRST] + +(14) CometNativeScan parquet +Output [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] +PushedFilters: [IsNotNull(l_orderkey), IsNotNull(l_suppkey)] +ReadSchema: struct + +(15) CometFilter +Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Condition : (isnotnull(l_orderkey#16L) AND isnotnull(l_suppkey#18L)) + +(16) CometExchange +Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: hashpartitioning(l_orderkey#16L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=424] + +(17) CometSort +Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22], [l_orderkey#16L ASC NULLS FIRST] + +(18) CometSortMergeJoin +Left output [2]: [c_nationkey#3L, o_orderkey#56L] +Right output [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: [o_orderkey#56L], [l_orderkey#16L], Inner + +(19) CometProject +Input [6]: [c_nationkey#3L, o_orderkey#56L, l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22], [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] + +(20) CometExchange +Input [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: hashpartitioning(l_suppkey#18L, c_nationkey#3L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=428] + +(21) CometSort +Input [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22], [l_suppkey#18L ASC NULLS FIRST, c_nationkey#3L ASC NULLS FIRST] + +(22) CometNativeScan parquet +Output [2]: [s_suppkey#108L, s_nationkey#111L] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/supplier] +PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] +ReadSchema: struct + +(23) CometFilter +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Condition : (isnotnull(s_suppkey#108L) AND isnotnull(s_nationkey#111L)) + +(24) CometExchange +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: hashpartitioning(s_suppkey#108L, s_nationkey#111L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=432] + +(25) CometSort +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: [s_suppkey#108L, s_nationkey#111L], [s_suppkey#108L ASC NULLS FIRST, s_nationkey#111L ASC NULLS FIRST] + +(26) CometSortMergeJoin +Left output [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Right output [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: [l_suppkey#18L, c_nationkey#3L], [s_suppkey#108L, s_nationkey#111L], Inner + +(27) CometProject +Input [6]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22, s_suppkey#108L, s_nationkey#111L] +Arguments: [l_extendedprice#21, l_discount#22, s_nationkey#111L], [l_extendedprice#21, l_discount#22, s_nationkey#111L] + +(28) CometNativeScan parquet +Output [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/nation] +PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] +ReadSchema: struct + +(29) CometFilter +Input [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] +Condition : ((isnotnull(n_nationkey#48L) AND isnotnull(n_regionkey#50L)) AND might_contain(Subquery subquery#128, [id=#161], xxhash64(n_regionkey#50L, 42))) + +(30) CometBroadcastExchange +Input [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] +Arguments: [n_nationkey#48L, n_name#49, n_regionkey#50L] + +(31) CometBroadcastHashJoin +Left output [3]: [l_extendedprice#21, l_discount#22, s_nationkey#111L] +Right output [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] +Arguments: [s_nationkey#111L], [n_nationkey#48L], Inner, BuildRight + +(32) CometProject +Input [6]: [l_extendedprice#21, l_discount#22, s_nationkey#111L, n_nationkey#48L, n_name#49, n_regionkey#50L] +Arguments: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L], [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L] + +(33) CometNativeScan parquet +Output [2]: [r_regionkey#102L, r_name#103] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/region] +PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AFRICA), IsNotNull(r_regionkey)] +ReadSchema: struct + +(34) CometFilter +Input [2]: [r_regionkey#102L, r_name#103] +Condition : ((isnotnull(r_name#103) AND (r_name#103 = AFRICA)) AND isnotnull(r_regionkey#102L)) + +(35) CometProject +Input [2]: [r_regionkey#102L, r_name#103] +Arguments: [r_regionkey#102L], [r_regionkey#102L] + +(36) CometBroadcastExchange +Input [1]: [r_regionkey#102L] +Arguments: [r_regionkey#102L] + +(37) CometBroadcastHashJoin +Left output [4]: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L] +Right output [1]: [r_regionkey#102L] +Arguments: [n_regionkey#50L], [r_regionkey#102L], Inner, BuildRight + +(38) CometProject +Input [5]: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L, r_regionkey#102L] +Arguments: [l_extendedprice#21, l_discount#22, n_name#49], [l_extendedprice#21, l_discount#22, n_name#49] + +(39) CometHashAggregate +Input [3]: [l_extendedprice#21, l_discount#22, n_name#49] +Keys [1]: [n_name#49] +Functions [1]: [partial_sum((l_extendedprice#21 * (1 - l_discount#22)))] + +(40) CometExchange +Input [3]: [n_name#49, sum#131, isEmpty#132] +Arguments: hashpartitioning(n_name#49, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=445] + +(41) CometHashAggregate +Input [3]: [n_name#49, sum#131, isEmpty#132] +Keys [1]: [n_name#49] +Functions [1]: [sum((l_extendedprice#21 * (1 - l_discount#22)))] + +(42) CometExchange +Input [2]: [n_name#49, revenue#122] +Arguments: rangepartitioning(revenue#122 DESC NULLS LAST, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=447] + +(43) CometSort +Input [2]: [n_name#49, revenue#122] +Arguments: [n_name#49, revenue#122], [revenue#122 DESC NULLS LAST] + +(44) AdaptiveSparkPlan +Output [2]: [n_name#49, revenue#122] +Arguments: isFinalPlan=false + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 29 Hosting Expression = Subquery subquery#128, [id=#161] +AdaptiveSparkPlan (51) ++- CometHashAggregate (50) + +- CometExchange (49) + +- CometHashAggregate (48) + +- CometProject (47) + +- CometFilter (46) + +- CometNativeScan parquet (45) + + +(45) CometNativeScan parquet +Output [2]: [r_regionkey#102L, r_name#103] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/region] +PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AFRICA), IsNotNull(r_regionkey)] +ReadSchema: struct + +(46) CometFilter +Input [2]: [r_regionkey#102L, r_name#103] +Condition : ((isnotnull(r_name#103) AND (r_name#103 = AFRICA)) AND isnotnull(r_regionkey#102L)) + +(47) CometProject +Input [2]: [r_regionkey#102L, r_name#103] +Arguments: [r_regionkey#102L], [r_regionkey#102L] + +(48) CometHashAggregate +Input [1]: [r_regionkey#102L] +Keys: [] +Functions [1]: [partial_bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)] + +(49) CometExchange +Input [1]: [buf#134] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=159] + +(50) CometHashAggregate +Input [1]: [buf#134] +Keys: [] +Functions [1]: [bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)] + +(51) AdaptiveSparkPlan +Output [1]: [bloomFilter#127] +Arguments: isFinalPlan=false + + + +Query 5 returned 5 rows, hash=f1e68aab4aa9d0988709357fc210d775 +Query 5 took 482.04 seconds + +Iteration 1 took 482.04 seconds + +Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/comet-offheap8g-q5-tpch-1775047157157.json diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.time b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.time new file mode 100644 index 0000000000..226129debc --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.time @@ -0,0 +1,5639 @@ +26/04/01 06:31:10 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) +26/04/01 06:31:10 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address +26/04/01 06:31:10 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +26/04/01 06:31:10 INFO SparkContext: Running Spark version 3.5.8 +26/04/01 06:31:10 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 06:31:10 INFO SparkContext: Java version 17.0.17 +26/04/01 06:31:10 INFO ResourceUtils: ============================================================== +26/04/01 06:31:10 INFO ResourceUtils: No custom resources configured for spark.driver. +26/04/01 06:31:10 INFO ResourceUtils: ============================================================== +26/04/01 06:31:10 INFO SparkContext: Submitted application: comet-offheap8g-q5 benchmark derived from tpch +26/04/01 06:31:10 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 8192, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) +26/04/01 06:31:10 INFO ResourceProfile: Limiting resource is cpu +26/04/01 06:31:10 INFO ResourceProfileManager: Added ResourceProfile id: 0 +26/04/01 06:31:10 INFO SecurityManager: Changing view acls to: andy +26/04/01 06:31:10 INFO SecurityManager: Changing modify acls to: andy +26/04/01 06:31:10 INFO SecurityManager: Changing view acls groups to: +26/04/01 06:31:10 INFO SecurityManager: Changing modify acls groups to: +26/04/01 06:31:10 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY +26/04/01 06:31:10 INFO Utils: Successfully started service 'sparkDriver' on port 58440. +26/04/01 06:31:10 INFO SparkEnv: Registering MapOutputTracker +26/04/01 06:31:10 INFO SparkEnv: Registering BlockManagerMaster +26/04/01 06:31:11 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information +26/04/01 06:31:11 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up +26/04/01 06:31:11 INFO SparkEnv: Registering BlockManagerMasterHeartbeat +26/04/01 06:31:11 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-7f73f6ab-0bd7-4a05-bc84-36cd8add9577 +26/04/01 06:31:11 INFO MemoryStore: MemoryStore started with capacity 12.6 GiB +26/04/01 06:31:11 INFO SparkEnv: Registering OutputCommitCoordinator +26/04/01 06:31:11 INFO SparkContext: Added JAR file:///Users/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar at spark://10.0.0.133:58440/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775046670865 +26/04/01 06:31:11 INFO CometDriverPlugin: CometDriverPlugin init +26/04/01 06:31:11 INFO CometDriverPlugin: Setting spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions +26/04/01 06:31:11 INFO CometDriverPlugin: Comet metrics reporting is disabled. Set spark.comet.metrics.enabled=true to enable. +26/04/01 06:31:11 INFO CometDriverPlugin: Comet is running in unified memory mode and sharing off-heap memory with Spark +26/04/01 06:31:11 INFO DriverPluginContainer: Initialized driver component for plugin org.apache.spark.CometPlugin. +26/04/01 06:31:11 INFO Executor: Starting executor ID driver on host 10.0.0.133 +26/04/01 06:31:11 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 06:31:11 INFO Executor: Java version 17.0.17 +26/04/01 06:31:11 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' +26/04/01 06:31:11 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@5e9ec9dd for default. +26/04/01 06:31:11 INFO Executor: Fetching spark://10.0.0.133:58440/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775046670865 +26/04/01 06:31:11 INFO TransportClientFactory: Successfully created connection to /10.0.0.133:58440 after 8 ms (0 ms spent in bootstraps) +26/04/01 06:31:11 INFO Utils: Fetching spark://10.0.0.133:58440/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-3bd87ee3-3f8e-4785-87c2-97740eb236dc/userFiles-e2b85f9c-464e-478c-a520-f16d67759264/fetchFileTemp16161609104732802677.tmp +26/04/01 06:31:11 INFO Executor: Adding file:/private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-3bd87ee3-3f8e-4785-87c2-97740eb236dc/userFiles-e2b85f9c-464e-478c-a520-f16d67759264/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to class loader default +26/04/01 06:31:11 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 58442. +26/04/01 06:31:11 INFO NettyBlockTransferService: Server created on 10.0.0.133:58442 +26/04/01 06:31:11 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy +26/04/01 06:31:11 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 58442, None) +26/04/01 06:31:11 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:58442 with 12.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 58442, None) +26/04/01 06:31:11 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 58442, None) +26/04/01 06:31:11 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 58442, None) +26/04/01 06:31:11 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. +26/04/01 06:31:11 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. +26/04/01 06:31:11 INFO InMemoryFileIndex: It took 17 ms to list leaf files for 1 paths. +26/04/01 06:31:11 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:31:11 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:31:11 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:31:11 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:31:11 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:11 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:31:11 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:31:11 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) +26/04/01 06:31:11 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:11 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:11 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:31:11 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 +26/04/01 06:31:12 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 06:31:12 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) +26/04/01 06:31:12 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 2041 bytes result sent to driver +26/04/01 06:31:12 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 115 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:31:12 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool +26/04/01 06:31:12 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.441 s +26/04/01 06:31:12 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:31:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished +26/04/01 06:31:12 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.451211 s +26/04/01 06:31:12 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:14 INFO NativeBase: Couldn't locate log file from either COMET_CONF_DIR or comet.log.file.path. Using default log configuration with INFO log level which emits to stderr +26/04/01 06:31:14 INFO core/src/lib.rs: Comet native library version 0.15.0 initialized +26/04/01 06:31:14 INFO NativeBase: Setting system property arrow.enable_unsafe_memory_access to true +26/04/01 06:31:14 INFO NativeBase: Setting system property arrow.enable_null_check_for_get to false +26/04/01 06:31:14 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. +26/04/01 06:31:14 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:31:14 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:31:14 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:31:14 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:31:14 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:14 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:31:14 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:31:14 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) +26/04/01 06:31:14 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:14 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:14 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:31:14 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 +26/04/01 06:31:14 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 06:31:14 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) +26/04/01 06:31:14 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2344 bytes result sent to driver +26/04/01 06:31:14 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 13 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:31:14 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool +26/04/01 06:31:14 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.021 s +26/04/01 06:31:14 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:31:14 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished +26/04/01 06:31:14 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.022628 s +26/04/01 06:31:14 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. +26/04/01 06:31:14 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:14 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:31:14 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:31:14 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:31:14 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:31:14 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:14 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:31:14 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:31:14 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) +26/04/01 06:31:14 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:14 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:14 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:31:14 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 +26/04/01 06:31:14 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 06:31:14 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) +26/04/01 06:31:14 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver +26/04/01 06:31:14 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:31:14 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool +26/04/01 06:31:14 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 06:31:14 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:31:14 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished +26/04/01 06:31:14 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.013560 s +26/04/01 06:31:14 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 06:31:14 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:31:14 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:31:14 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:31:14 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:31:14 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:14 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:31:14 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:31:14 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) +26/04/01 06:31:14 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:14 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:14 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:31:14 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 +26/04/01 06:31:14 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 06:31:14 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) +26/04/01 06:31:14 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver +26/04/01 06:31:14 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:31:14 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool +26/04/01 06:31:14 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.012 s +26/04/01 06:31:14 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:31:14 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished +26/04/01 06:31:14 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.013633 s +26/04/01 06:31:14 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 06:31:14 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:31:14 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:31:14 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:31:14 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:31:14 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:14 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:31:14 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:31:14 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) +26/04/01 06:31:14 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:14 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:14 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:14 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:31:14 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 +26/04/01 06:31:14 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:14 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9299 bytes) +26/04/01 06:31:14 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) +26/04/01 06:31:14 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver +26/04/01 06:31:14 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:31:14 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool +26/04/01 06:31:14 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 06:31:14 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:31:14 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished +26/04/01 06:31:14 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.014272 s +26/04/01 06:31:14 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 06:31:14 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:31:14 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:31:14 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:31:14 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:31:14 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:14 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:15 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 +26/04/01 06:31:15 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 06:31:15 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) +26/04/01 06:31:15 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver +26/04/01 06:31:15 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool +26/04/01 06:31:15 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 06:31:15 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:31:15 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished +26/04/01 06:31:15 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.013480 s +26/04/01 06:31:15 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. +26/04/01 06:31:15 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:31:15 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:31:15 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:15 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:15 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 +26/04/01 06:31:15 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 06:31:15 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) +26/04/01 06:31:15 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver +26/04/01 06:31:15 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool +26/04/01 06:31:15 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.012 s +26/04/01 06:31:15 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:31:15 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished +26/04/01 06:31:15 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.013919 s +26/04/01 06:31:15 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 06:31:15 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 06:31:15 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 06:31:15 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) +26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:15 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:15 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 +26/04/01 06:31:15 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 06:31:15 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) +26/04/01 06:31:15 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver +26/04/01 06:31:15 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool +26/04/01 06:31:15 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.011 s +26/04/01 06:31:15 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:31:15 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished +26/04/01 06:31:15 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.012601 s +26/04/01 06:31:15 INFO FileSourceStrategy: Pushed Filters: IsNotNull(c_custkey),IsNotNull(c_nationkey) +26/04/01 06:31:15 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(c_custkey#0L),isnotnull(c_nationkey#3L) +26/04/01 06:31:15 INFO FileSourceStrategy: Pushed Filters: IsNotNull(o_orderdate),GreaterThanOrEqual(o_orderdate,1994-01-01),LessThan(o_orderdate,1995-01-01),IsNotNull(o_custkey),IsNotNull(o_orderkey) +26/04/01 06:31:15 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(o_orderdate#60),(o_orderdate#60 >= 1994-01-01),(o_orderdate#60 < 1995-01-01),isnotnull(o_custkey#57L),isnotnull(o_orderkey#56L) +26/04/01 06:31:15 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_orderkey),IsNotNull(l_suppkey) +26/04/01 06:31:15 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_orderkey#16L),isnotnull(l_suppkey#18L) +26/04/01 06:31:15 INFO FileSourceStrategy: Pushed Filters: IsNotNull(s_suppkey),IsNotNull(s_nationkey) +26/04/01 06:31:15 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(s_suppkey#108L),isnotnull(s_nationkey#111L) +26/04/01 06:31:15 INFO FileSourceStrategy: Pushed Filters: IsNotNull(n_nationkey),IsNotNull(n_regionkey) +26/04/01 06:31:15 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(n_nationkey#48L),isnotnull(n_regionkey#50L),might_contain(scalar-subquery#128 [], xxhash64(n_regionkey#50L, 42)) +26/04/01 06:31:15 INFO FileSourceStrategy: Pushed Filters: IsNotNull(r_name),EqualTo(r_name,AFRICA),IsNotNull(r_regionkey) +26/04/01 06:31:15 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(r_name#103),(r_name#103 = AFRICA),isnotnull(r_regionkey#102L) +26/04/01 06:31:15 INFO FileSourceStrategy: Pushed Filters: IsNotNull(r_name),EqualTo(r_name,AFRICA),IsNotNull(r_regionkey) +26/04/01 06:31:15 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(r_name#103),(r_name#103 = AFRICA),isnotnull(r_regionkey#102L) +26/04/01 06:31:15 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 55920126 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:58442 (size: 35.4 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:31:15 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO DAGScheduler: Registering RDD 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 +26/04/01 06:31:15 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:31:15 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:15 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 23.0 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 8.0 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:58442 (size: 8.0 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 9 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:15 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 8.0 with 1 tasks resource profile 0 +26/04/01 06:31:15 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9260 bytes) +26/04/01 06:31:15 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) +26/04/01 06:31:15 INFO BaseAllocator: Debug mode disabled. Enable with the VM option -Darrow.memory.debug.allocator=true. +26/04/01 06:31:15 INFO DefaultAllocationManagerOption: allocation manager type not specified, using netty as the default type +26/04/01 06:31:15 INFO CheckAllocator: Using DefaultAllocationManager at memory/unsafe/DefaultAllocationManagerFactory.class +26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:15 INFO core/src/execution/jni_api.rs: Comet tokio runtime: using spark.executor.cores=4 worker threads +26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:15 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 6628 bytes result sent to driver +26/04/01 06:31:15 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 269 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool +26/04/01 06:31:15 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.282 s +26/04/01 06:31:15 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:31:15 INFO DAGScheduler: running: Set() +26/04/01 06:31:15 INFO DAGScheduler: waiting: Set() +26/04/01 06:31:15 INFO DAGScheduler: failed: Set() +26/04/01 06:31:15 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:31:15 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:31:15 INFO DAGScheduler: Final stage: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 9) +26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:15 INFO DAGScheduler: Submitting ResultStage 10 (MapPartitionsRDD[22] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 17.6 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 7.5 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:58442 (size: 7.5 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 10 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:15 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 10 (MapPartitionsRDD[22] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 10.0 with 1 tasks resource profile 0 +26/04/01 06:31:15 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 9) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) +26/04/01 06:31:15 INFO Executor: Running task 0.0 in stage 10.0 (TID 9) +26/04/01 06:31:15 INFO ShuffleBlockFetcherIterator: Getting 1 (4.7 KiB) non-empty blocks including 1 (4.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:31:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:15 INFO Executor: Finished task 0.0 in stage 10.0 (TID 9). 9966 bytes result sent to driver +26/04/01 06:31:15 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 9) in 37 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool +26/04/01 06:31:15 INFO DAGScheduler: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.042 s +26/04/01 06:31:15 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:31:15 INFO TaskSchedulerImpl: Killing all running tasks in stage 10: Stage finished +26/04/01 06:31:15 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.045698 s +26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_13 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:58442 (size: 35.4 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 11 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:58442 (size: 35.4 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_13_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 12 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:31:15 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_13_piece0 in memory on 10.0.0.133:58442 (size: 35.4 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 13 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:31:15 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO DAGScheduler: Registering RDD 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 +26/04/01 06:31:15 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 16 output partitions +26/04/01 06:31:15 INFO DAGScheduler: Final stage: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:15 INFO DAGScheduler: Submitting ShuffleMapStage 11 (MapPartitionsRDD[31] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_14 stored as values in memory (estimated size 17.6 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_14_piece0 stored as bytes in memory (estimated size 7.8 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_14_piece0 in memory on 10.0.0.133:58442 (size: 7.8 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 14 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:15 INFO DAGScheduler: Submitting 16 missing tasks from ShuffleMapStage 11 (MapPartitionsRDD[31] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 11.0 with 16 tasks resource profile 0 +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_15 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:31:15 INFO TaskSetManager: Starting task 0.0 in stage 11.0 (TID 10) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:31:15 INFO DAGScheduler: Got job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:31:15 INFO DAGScheduler: Final stage: ResultStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:31:15 INFO TaskSetManager: Starting task 1.0 in stage 11.0 (TID 11) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9334 bytes) +26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:15 INFO TaskSetManager: Starting task 2.0 in stage 11.0 (TID 12) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:31:15 INFO DAGScheduler: Submitting ResultStage 12 (MapPartitionsRDD[29] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:31:15 INFO TaskSetManager: Starting task 3.0 in stage 11.0 (TID 13) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9334 bytes) +26/04/01 06:31:15 INFO Executor: Running task 0.0 in stage 11.0 (TID 10) +26/04/01 06:31:15 INFO Executor: Running task 1.0 in stage 11.0 (TID 11) +26/04/01 06:31:15 INFO Executor: Running task 2.0 in stage 11.0 (TID 12) +26/04/01 06:31:15 INFO Executor: Running task 3.0 in stage 11.0 (TID 13) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_16 stored as values in memory (estimated size 14.0 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_16_piece0 stored as bytes in memory (estimated size 5.4 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_16_piece0 in memory on 10.0.0.133:58442 (size: 5.4 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 16 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:15 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 12 (MapPartitionsRDD[29] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:31:15 INFO BlockManagerInfo: Removed broadcast_9_piece0 on 10.0.0.133:58442 in memory (size: 8.0 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 12.0 with 1 tasks resource profile 0 +26/04/01 06:31:15 INFO BlockManagerInfo: Removed broadcast_10_piece0 on 10.0.0.133:58442 in memory (size: 7.5 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_15_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_15_piece0 in memory on 10.0.0.133:58442 (size: 35.4 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 15 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:31:15 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:15 INFO DAGScheduler: Registering RDD 34 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 2 +26/04/01 06:31:15 INFO DAGScheduler: Got map stage job 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 64 output partitions +26/04/01 06:31:15 INFO DAGScheduler: Final stage: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:15 INFO DAGScheduler: Submitting ShuffleMapStage 13 (MapPartitionsRDD[34] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_17 stored as values in memory (estimated size 23.3 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_17_piece0 stored as bytes in memory (estimated size 8.1 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_17_piece0 in memory on 10.0.0.133:58442 (size: 8.1 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 17 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:15 INFO DAGScheduler: Submitting 64 missing tasks from ShuffleMapStage 13 (MapPartitionsRDD[34] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 13.0 with 64 tasks resource profile 0 +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_18 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_18_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_18_piece0 in memory on 10.0.0.133:58442 (size: 35.4 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 18 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:31:15 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO DAGScheduler: Registering RDD 37 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 3 +26/04/01 06:31:15 INFO DAGScheduler: Got map stage job 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions +26/04/01 06:31:15 INFO DAGScheduler: Final stage: ShuffleMapStage 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 55920126 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 INFO DAGScheduler: Submitting ShuffleMapStage 14 (MapPartitionsRDD[37] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_20 stored as values in memory (estimated size 18.4 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_19 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_20_piece0 stored as bytes in memory (estimated size 8.0 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_20_piece0 in memory on 10.0.0.133:58442 (size: 8.0 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 20 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:15 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 14 (MapPartitionsRDD[37] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 14.0 with 208 tasks resource profile 0 +26/04/01 06:31:15 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:31:15 INFO DAGScheduler: Got job 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:31:15 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_19_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:15 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_19_piece0 in memory on 10.0.0.133:58442 (size: 35.4 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 19 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:31:15 INFO FileSourceScanExec: Planning scan with bin packing, max size: 55920126 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 06:31:15 WARN DAGScheduler: Broadcasting large task binary with size 1113.7 KiB +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_21 stored as values in memory (estimated size 1113.7 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_21_piece0 stored as bytes in memory (estimated size 30.7 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_21_piece0 in memory on 10.0.0.133:58442 (size: 30.7 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 21 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:15 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 15.0 with 1 tasks resource profile 0 +26/04/01 06:31:15 INFO DAGScheduler: Registering RDD 40 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 4 +26/04/01 06:31:15 INFO DAGScheduler: Got map stage job 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 4 output partitions +26/04/01 06:31:15 INFO DAGScheduler: Final stage: ShuffleMapStage 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() +26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:15 INFO DAGScheduler: Submitting ShuffleMapStage 16 (MapPartitionsRDD[40] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_22 stored as values in memory (estimated size 17.6 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO MemoryStore: Block broadcast_22_piece0 stored as bytes in memory (estimated size 7.8 KiB, free 12.6 GiB) +26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_22_piece0 in memory on 10.0.0.133:58442 (size: 7.8 KiB, free: 12.6 GiB) +26/04/01 06:31:15 INFO SparkContext: Created broadcast 22 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:15 INFO DAGScheduler: Submitting 4 missing tasks from ShuffleMapStage 16 (MapPartitionsRDD[40] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3)) +26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 16.0 with 4 tasks resource profile 0 +26/04/01 06:31:16 INFO Executor: Finished task 1.0 in stage 11.0 (TID 11). 6649 bytes result sent to driver +26/04/01 06:31:16 INFO TaskSetManager: Starting task 4.0 in stage 11.0 (TID 14) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:31:16 INFO Executor: Running task 4.0 in stage 11.0 (TID 14) +26/04/01 06:31:16 INFO TaskSetManager: Finished task 1.0 in stage 11.0 (TID 11) in 740 ms on 10.0.0.133 (executor driver) (1/16) +26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:16 INFO Executor: Finished task 2.0 in stage 11.0 (TID 12). 6649 bytes result sent to driver +26/04/01 06:31:16 INFO Executor: Finished task 3.0 in stage 11.0 (TID 13). 6649 bytes result sent to driver +26/04/01 06:31:16 INFO TaskSetManager: Starting task 5.0 in stage 11.0 (TID 15) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:31:16 INFO TaskSetManager: Finished task 2.0 in stage 11.0 (TID 12) in 743 ms on 10.0.0.133 (executor driver) (2/16) +26/04/01 06:31:16 INFO Executor: Running task 5.0 in stage 11.0 (TID 15) +26/04/01 06:31:16 INFO TaskSetManager: Starting task 6.0 in stage 11.0 (TID 16) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:31:16 INFO Executor: Running task 6.0 in stage 11.0 (TID 16) +26/04/01 06:31:16 INFO TaskSetManager: Finished task 3.0 in stage 11.0 (TID 13) in 743 ms on 10.0.0.133 (executor driver) (3/16) +26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:16 INFO Executor: Finished task 0.0 in stage 11.0 (TID 10). 6649 bytes result sent to driver +26/04/01 06:31:16 INFO TaskSetManager: Starting task 7.0 in stage 11.0 (TID 17) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9334 bytes) +26/04/01 06:31:16 INFO Executor: Running task 7.0 in stage 11.0 (TID 17) +26/04/01 06:31:16 INFO TaskSetManager: Finished task 0.0 in stage 11.0 (TID 10) in 745 ms on 10.0.0.133 (executor driver) (4/16) +26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:17 INFO Executor: Finished task 6.0 in stage 11.0 (TID 16). 6606 bytes result sent to driver +26/04/01 06:31:17 INFO TaskSetManager: Starting task 8.0 in stage 11.0 (TID 18) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:31:17 INFO Executor: Finished task 7.0 in stage 11.0 (TID 17). 6563 bytes result sent to driver +26/04/01 06:31:17 INFO Executor: Running task 8.0 in stage 11.0 (TID 18) +26/04/01 06:31:17 INFO TaskSetManager: Starting task 9.0 in stage 11.0 (TID 19) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:31:17 INFO Executor: Finished task 5.0 in stage 11.0 (TID 15). 6606 bytes result sent to driver +26/04/01 06:31:17 INFO TaskSetManager: Finished task 6.0 in stage 11.0 (TID 16) in 743 ms on 10.0.0.133 (executor driver) (5/16) +26/04/01 06:31:17 INFO Executor: Running task 9.0 in stage 11.0 (TID 19) +26/04/01 06:31:17 INFO TaskSetManager: Starting task 10.0 in stage 11.0 (TID 20) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9334 bytes) +26/04/01 06:31:17 INFO Executor: Running task 10.0 in stage 11.0 (TID 20) +26/04/01 06:31:17 INFO TaskSetManager: Finished task 7.0 in stage 11.0 (TID 17) in 741 ms on 10.0.0.133 (executor driver) (6/16) +26/04/01 06:31:17 INFO Executor: Finished task 4.0 in stage 11.0 (TID 14). 6563 bytes result sent to driver +26/04/01 06:31:17 INFO TaskSetManager: Starting task 11.0 in stage 11.0 (TID 21) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:31:17 INFO TaskSetManager: Finished task 5.0 in stage 11.0 (TID 15) in 745 ms on 10.0.0.133 (executor driver) (7/16) +26/04/01 06:31:17 INFO Executor: Running task 11.0 in stage 11.0 (TID 21) +26/04/01 06:31:17 INFO TaskSetManager: Finished task 4.0 in stage 11.0 (TID 14) in 748 ms on 10.0.0.133 (executor driver) (8/16) +26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO Executor: Finished task 9.0 in stage 11.0 (TID 19). 6606 bytes result sent to driver +26/04/01 06:31:18 INFO Executor: Finished task 10.0 in stage 11.0 (TID 20). 6606 bytes result sent to driver +26/04/01 06:31:18 INFO TaskSetManager: Starting task 12.0 in stage 11.0 (TID 22) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9334 bytes) +26/04/01 06:31:18 INFO Executor: Finished task 11.0 in stage 11.0 (TID 21). 6563 bytes result sent to driver +26/04/01 06:31:18 INFO Executor: Running task 12.0 in stage 11.0 (TID 22) +26/04/01 06:31:18 INFO TaskSetManager: Starting task 13.0 in stage 11.0 (TID 23) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9333 bytes) +26/04/01 06:31:18 INFO TaskSetManager: Finished task 9.0 in stage 11.0 (TID 19) in 724 ms on 10.0.0.133 (executor driver) (9/16) +26/04/01 06:31:18 INFO Executor: Running task 13.0 in stage 11.0 (TID 23) +26/04/01 06:31:18 INFO TaskSetManager: Finished task 10.0 in stage 11.0 (TID 20) in 724 ms on 10.0.0.133 (executor driver) (10/16) +26/04/01 06:31:18 INFO Executor: Finished task 8.0 in stage 11.0 (TID 18). 6606 bytes result sent to driver +26/04/01 06:31:18 INFO TaskSetManager: Starting task 14.0 in stage 11.0 (TID 24) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9334 bytes) +26/04/01 06:31:18 INFO TaskSetManager: Finished task 11.0 in stage 11.0 (TID 21) in 724 ms on 10.0.0.133 (executor driver) (11/16) +26/04/01 06:31:18 INFO Executor: Running task 14.0 in stage 11.0 (TID 24) +26/04/01 06:31:18 INFO TaskSetManager: Starting task 15.0 in stage 11.0 (TID 25) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9334 bytes) +26/04/01 06:31:18 INFO Executor: Running task 15.0 in stage 11.0 (TID 25) +26/04/01 06:31:18 INFO TaskSetManager: Finished task 8.0 in stage 11.0 (TID 18) in 727 ms on 10.0.0.133 (executor driver) (12/16) +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO Executor: Finished task 13.0 in stage 11.0 (TID 23). 6606 bytes result sent to driver +26/04/01 06:31:18 INFO Executor: Finished task 12.0 in stage 11.0 (TID 22). 6606 bytes result sent to driver +26/04/01 06:31:18 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 26) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9271 bytes) +26/04/01 06:31:18 INFO Executor: Running task 0.0 in stage 12.0 (TID 26) +26/04/01 06:31:18 INFO TaskSetManager: Starting task 0.0 in stage 13.0 (TID 27) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:18 INFO TaskSetManager: Finished task 13.0 in stage 11.0 (TID 23) in 719 ms on 10.0.0.133 (executor driver) (13/16) +26/04/01 06:31:18 INFO Executor: Running task 0.0 in stage 13.0 (TID 27) +26/04/01 06:31:18 INFO TaskSetManager: Finished task 12.0 in stage 11.0 (TID 22) in 720 ms on 10.0.0.133 (executor driver) (14/16) +26/04/01 06:31:18 INFO Executor: Finished task 14.0 in stage 11.0 (TID 24). 6563 bytes result sent to driver +26/04/01 06:31:18 INFO TaskSetManager: Starting task 1.0 in stage 13.0 (TID 28) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:31:18 INFO Executor: Running task 1.0 in stage 13.0 (TID 28) +26/04/01 06:31:18 INFO TaskSetManager: Finished task 14.0 in stage 11.0 (TID 24) in 720 ms on 10.0.0.133 (executor driver) (15/16) +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO Executor: Finished task 15.0 in stage 11.0 (TID 25). 6563 bytes result sent to driver +26/04/01 06:31:18 INFO TaskSetManager: Starting task 2.0 in stage 13.0 (TID 29) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:18 INFO TaskSetManager: Finished task 15.0 in stage 11.0 (TID 25) in 721 ms on 10.0.0.133 (executor driver) (16/16) +26/04/01 06:31:18 INFO Executor: Running task 2.0 in stage 13.0 (TID 29) +26/04/01 06:31:18 INFO TaskSchedulerImpl: Removed TaskSet 11.0, whose tasks have all completed, from pool +26/04/01 06:31:18 INFO DAGScheduler: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.935 s +26/04/01 06:31:18 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:31:18 INFO DAGScheduler: running: Set(ResultStage 15, ResultStage 12, ShuffleMapStage 16, ShuffleMapStage 13, ShuffleMapStage 14) +26/04/01 06:31:18 INFO DAGScheduler: waiting: Set() +26/04/01 06:31:18 INFO DAGScheduler: failed: Set() +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO Executor: Finished task 0.0 in stage 12.0 (TID 26). 4405 bytes result sent to driver +26/04/01 06:31:18 INFO TaskSetManager: Starting task 3.0 in stage 13.0 (TID 30) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:18 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 26) in 25 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:31:18 INFO Executor: Running task 3.0 in stage 13.0 (TID 30) +26/04/01 06:31:18 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool +26/04/01 06:31:18 INFO DAGScheduler: ResultStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.955 s +26/04/01 06:31:18 INFO DAGScheduler: Job 11 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:31:18 INFO TaskSchedulerImpl: Killing all running tasks in stage 12: Stage finished +26/04/01 06:31:18 INFO DAGScheduler: Job 11 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 2.955010 s +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:18 INFO Utils: Coalesced 1 broadcast batches into 1 (1 rows) +26/04/01 06:31:18 INFO MemoryStore: Block broadcast_23 stored as values in memory (estimated size 416.0 B, free 12.6 GiB) +26/04/01 06:31:18 INFO MemoryStore: Block broadcast_23_piece0 stored as bytes in memory (estimated size 367.0 B, free 12.6 GiB) +26/04/01 06:31:18 INFO BlockManagerInfo: Added broadcast_23_piece0 in memory on 10.0.0.133:58442 (size: 367.0 B, free: 12.6 GiB) +26/04/01 06:31:18 INFO SparkContext: Created broadcast 23 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:31:20 INFO Executor: Finished task 2.0 in stage 13.0 (TID 29). 6675 bytes result sent to driver +26/04/01 06:31:20 INFO Executor: Finished task 1.0 in stage 13.0 (TID 28). 6675 bytes result sent to driver +26/04/01 06:31:20 INFO TaskSetManager: Starting task 4.0 in stage 13.0 (TID 31) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:20 INFO Executor: Running task 4.0 in stage 13.0 (TID 31) +26/04/01 06:31:20 INFO TaskSetManager: Starting task 5.0 in stage 13.0 (TID 32) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:31:20 INFO TaskSetManager: Finished task 2.0 in stage 13.0 (TID 29) in 1271 ms on 10.0.0.133 (executor driver) (1/64) +26/04/01 06:31:20 INFO Executor: Running task 5.0 in stage 13.0 (TID 32) +26/04/01 06:31:20 INFO TaskSetManager: Finished task 1.0 in stage 13.0 (TID 28) in 1273 ms on 10.0.0.133 (executor driver) (2/64) +26/04/01 06:31:20 INFO Executor: Finished task 0.0 in stage 13.0 (TID 27). 6675 bytes result sent to driver +26/04/01 06:31:20 INFO TaskSetManager: Starting task 6.0 in stage 13.0 (TID 33) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:20 INFO TaskSetManager: Finished task 0.0 in stage 13.0 (TID 27) in 1275 ms on 10.0.0.133 (executor driver) (3/64) +26/04/01 06:31:20 INFO Executor: Running task 6.0 in stage 13.0 (TID 33) +26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:20 INFO Executor: Finished task 3.0 in stage 13.0 (TID 30). 6675 bytes result sent to driver +26/04/01 06:31:20 INFO TaskSetManager: Starting task 7.0 in stage 13.0 (TID 34) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:20 INFO TaskSetManager: Finished task 3.0 in stage 13.0 (TID 30) in 1270 ms on 10.0.0.133 (executor driver) (4/64) +26/04/01 06:31:20 INFO Executor: Running task 7.0 in stage 13.0 (TID 34) +26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:21 INFO Executor: Finished task 6.0 in stage 13.0 (TID 33). 6675 bytes result sent to driver +26/04/01 06:31:21 INFO TaskSetManager: Starting task 8.0 in stage 13.0 (TID 35) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:31:21 INFO Executor: Finished task 4.0 in stage 13.0 (TID 31). 6675 bytes result sent to driver +26/04/01 06:31:21 INFO Executor: Running task 8.0 in stage 13.0 (TID 35) +26/04/01 06:31:21 INFO TaskSetManager: Starting task 9.0 in stage 13.0 (TID 36) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:21 INFO TaskSetManager: Finished task 6.0 in stage 13.0 (TID 33) in 1265 ms on 10.0.0.133 (executor driver) (5/64) +26/04/01 06:31:21 INFO TaskSetManager: Finished task 4.0 in stage 13.0 (TID 31) in 1265 ms on 10.0.0.133 (executor driver) (6/64) +26/04/01 06:31:21 INFO Executor: Running task 9.0 in stage 13.0 (TID 36) +26/04/01 06:31:21 INFO Executor: Finished task 5.0 in stage 13.0 (TID 32). 6761 bytes result sent to driver +26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:21 INFO TaskSetManager: Starting task 10.0 in stage 13.0 (TID 37) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:31:21 INFO TaskSetManager: Finished task 5.0 in stage 13.0 (TID 32) in 1269 ms on 10.0.0.133 (executor driver) (7/64) +26/04/01 06:31:21 INFO Executor: Running task 10.0 in stage 13.0 (TID 37) +26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:21 INFO Executor: Finished task 7.0 in stage 13.0 (TID 34). 6718 bytes result sent to driver +26/04/01 06:31:21 INFO TaskSetManager: Starting task 11.0 in stage 13.0 (TID 38) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:21 INFO TaskSetManager: Finished task 7.0 in stage 13.0 (TID 34) in 1271 ms on 10.0.0.133 (executor driver) (8/64) +26/04/01 06:31:21 INFO Executor: Running task 11.0 in stage 13.0 (TID 38) +26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:22 INFO Executor: Finished task 8.0 in stage 13.0 (TID 35). 6718 bytes result sent to driver +26/04/01 06:31:22 INFO Executor: Finished task 9.0 in stage 13.0 (TID 36). 6718 bytes result sent to driver +26/04/01 06:31:22 INFO TaskSetManager: Starting task 12.0 in stage 13.0 (TID 39) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:22 INFO Executor: Running task 12.0 in stage 13.0 (TID 39) +26/04/01 06:31:22 INFO TaskSetManager: Starting task 13.0 in stage 13.0 (TID 40) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:22 INFO TaskSetManager: Finished task 8.0 in stage 13.0 (TID 35) in 1265 ms on 10.0.0.133 (executor driver) (9/64) +26/04/01 06:31:22 INFO Executor: Running task 13.0 in stage 13.0 (TID 40) +26/04/01 06:31:22 INFO TaskSetManager: Finished task 9.0 in stage 13.0 (TID 36) in 1265 ms on 10.0.0.133 (executor driver) (10/64) +26/04/01 06:31:22 INFO Executor: Finished task 10.0 in stage 13.0 (TID 37). 6675 bytes result sent to driver +26/04/01 06:31:22 INFO TaskSetManager: Starting task 14.0 in stage 13.0 (TID 41) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:22 INFO TaskSetManager: Finished task 10.0 in stage 13.0 (TID 37) in 1262 ms on 10.0.0.133 (executor driver) (11/64) +26/04/01 06:31:22 INFO Executor: Running task 14.0 in stage 13.0 (TID 41) +26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:22 INFO Executor: Finished task 11.0 in stage 13.0 (TID 38). 6675 bytes result sent to driver +26/04/01 06:31:22 INFO TaskSetManager: Starting task 15.0 in stage 13.0 (TID 42) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:22 INFO TaskSetManager: Finished task 11.0 in stage 13.0 (TID 38) in 1266 ms on 10.0.0.133 (executor driver) (12/64) +26/04/01 06:31:22 INFO Executor: Running task 15.0 in stage 13.0 (TID 42) +26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:23 INFO Executor: Finished task 13.0 in stage 13.0 (TID 40). 6675 bytes result sent to driver +26/04/01 06:31:23 INFO TaskSetManager: Starting task 16.0 in stage 13.0 (TID 43) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:31:23 INFO Executor: Running task 16.0 in stage 13.0 (TID 43) +26/04/01 06:31:23 INFO TaskSetManager: Finished task 13.0 in stage 13.0 (TID 40) in 1270 ms on 10.0.0.133 (executor driver) (13/64) +26/04/01 06:31:23 INFO Executor: Finished task 14.0 in stage 13.0 (TID 41). 6675 bytes result sent to driver +26/04/01 06:31:23 INFO TaskSetManager: Starting task 17.0 in stage 13.0 (TID 44) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:23 INFO Executor: Finished task 12.0 in stage 13.0 (TID 39). 6675 bytes result sent to driver +26/04/01 06:31:23 INFO TaskSetManager: Finished task 14.0 in stage 13.0 (TID 41) in 1270 ms on 10.0.0.133 (executor driver) (14/64) +26/04/01 06:31:23 INFO Executor: Running task 17.0 in stage 13.0 (TID 44) +26/04/01 06:31:23 INFO TaskSetManager: Starting task 18.0 in stage 13.0 (TID 45) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:31:23 INFO Executor: Running task 18.0 in stage 13.0 (TID 45) +26/04/01 06:31:23 INFO TaskSetManager: Finished task 12.0 in stage 13.0 (TID 39) in 1271 ms on 10.0.0.133 (executor driver) (15/64) +26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:23 INFO Executor: Finished task 15.0 in stage 13.0 (TID 42). 6675 bytes result sent to driver +26/04/01 06:31:23 INFO TaskSetManager: Starting task 19.0 in stage 13.0 (TID 46) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:23 INFO TaskSetManager: Finished task 15.0 in stage 13.0 (TID 42) in 1269 ms on 10.0.0.133 (executor driver) (16/64) +26/04/01 06:31:23 INFO Executor: Running task 19.0 in stage 13.0 (TID 46) +26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:25 INFO Executor: Finished task 17.0 in stage 13.0 (TID 44). 6675 bytes result sent to driver +26/04/01 06:31:25 INFO Executor: Finished task 16.0 in stage 13.0 (TID 43). 6675 bytes result sent to driver +26/04/01 06:31:25 INFO TaskSetManager: Starting task 20.0 in stage 13.0 (TID 47) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:25 INFO TaskSetManager: Starting task 21.0 in stage 13.0 (TID 48) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:31:25 INFO Executor: Running task 20.0 in stage 13.0 (TID 47) +26/04/01 06:31:25 INFO TaskSetManager: Finished task 16.0 in stage 13.0 (TID 43) in 1270 ms on 10.0.0.133 (executor driver) (17/64) +26/04/01 06:31:25 INFO Executor: Running task 21.0 in stage 13.0 (TID 48) +26/04/01 06:31:25 INFO TaskSetManager: Finished task 17.0 in stage 13.0 (TID 44) in 1268 ms on 10.0.0.133 (executor driver) (18/64) +26/04/01 06:31:25 INFO Executor: Finished task 18.0 in stage 13.0 (TID 45). 6675 bytes result sent to driver +26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:25 INFO TaskSetManager: Starting task 22.0 in stage 13.0 (TID 49) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:25 INFO TaskSetManager: Finished task 18.0 in stage 13.0 (TID 45) in 1270 ms on 10.0.0.133 (executor driver) (19/64) +26/04/01 06:31:25 INFO Executor: Running task 22.0 in stage 13.0 (TID 49) +26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:25 INFO Executor: Finished task 19.0 in stage 13.0 (TID 46). 6675 bytes result sent to driver +26/04/01 06:31:25 INFO TaskSetManager: Starting task 23.0 in stage 13.0 (TID 50) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:25 INFO Executor: Running task 23.0 in stage 13.0 (TID 50) +26/04/01 06:31:25 INFO TaskSetManager: Finished task 19.0 in stage 13.0 (TID 46) in 1268 ms on 10.0.0.133 (executor driver) (20/64) +26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:26 INFO Executor: Finished task 22.0 in stage 13.0 (TID 49). 6675 bytes result sent to driver +26/04/01 06:31:26 INFO TaskSetManager: Starting task 24.0 in stage 13.0 (TID 51) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:31:26 INFO TaskSetManager: Finished task 22.0 in stage 13.0 (TID 49) in 1259 ms on 10.0.0.133 (executor driver) (21/64) +26/04/01 06:31:26 INFO Executor: Running task 24.0 in stage 13.0 (TID 51) +26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:26 INFO Executor: Finished task 21.0 in stage 13.0 (TID 48). 6675 bytes result sent to driver +26/04/01 06:31:26 INFO TaskSetManager: Starting task 25.0 in stage 13.0 (TID 52) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:26 INFO Executor: Running task 25.0 in stage 13.0 (TID 52) +26/04/01 06:31:26 INFO TaskSetManager: Finished task 21.0 in stage 13.0 (TID 48) in 1264 ms on 10.0.0.133 (executor driver) (22/64) +26/04/01 06:31:26 INFO Executor: Finished task 20.0 in stage 13.0 (TID 47). 6675 bytes result sent to driver +26/04/01 06:31:26 INFO TaskSetManager: Starting task 26.0 in stage 13.0 (TID 53) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:26 INFO TaskSetManager: Finished task 20.0 in stage 13.0 (TID 47) in 1265 ms on 10.0.0.133 (executor driver) (23/64) +26/04/01 06:31:26 INFO Executor: Running task 26.0 in stage 13.0 (TID 53) +26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:26 INFO Executor: Finished task 23.0 in stage 13.0 (TID 50). 6675 bytes result sent to driver +26/04/01 06:31:26 INFO TaskSetManager: Starting task 27.0 in stage 13.0 (TID 54) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:26 INFO Executor: Running task 27.0 in stage 13.0 (TID 54) +26/04/01 06:31:26 INFO TaskSetManager: Finished task 23.0 in stage 13.0 (TID 50) in 1266 ms on 10.0.0.133 (executor driver) (24/64) +26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:27 INFO Executor: Finished task 24.0 in stage 13.0 (TID 51). 6675 bytes result sent to driver +26/04/01 06:31:27 INFO Executor: Finished task 25.0 in stage 13.0 (TID 52). 6675 bytes result sent to driver +26/04/01 06:31:27 INFO Executor: Finished task 26.0 in stage 13.0 (TID 53). 6675 bytes result sent to driver +26/04/01 06:31:27 INFO TaskSetManager: Starting task 28.0 in stage 13.0 (TID 55) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:27 INFO Executor: Running task 28.0 in stage 13.0 (TID 55) +26/04/01 06:31:27 INFO TaskSetManager: Starting task 29.0 in stage 13.0 (TID 56) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:27 INFO TaskSetManager: Finished task 24.0 in stage 13.0 (TID 51) in 1271 ms on 10.0.0.133 (executor driver) (25/64) +26/04/01 06:31:27 INFO TaskSetManager: Finished task 25.0 in stage 13.0 (TID 52) in 1268 ms on 10.0.0.133 (executor driver) (26/64) +26/04/01 06:31:27 INFO Executor: Running task 29.0 in stage 13.0 (TID 56) +26/04/01 06:31:27 INFO TaskSetManager: Starting task 30.0 in stage 13.0 (TID 57) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9264 bytes) +26/04/01 06:31:27 INFO TaskSetManager: Finished task 26.0 in stage 13.0 (TID 53) in 1267 ms on 10.0.0.133 (executor driver) (27/64) +26/04/01 06:31:27 INFO Executor: Running task 30.0 in stage 13.0 (TID 57) +26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:27 INFO Executor: Finished task 27.0 in stage 13.0 (TID 54). 6675 bytes result sent to driver +26/04/01 06:31:27 INFO TaskSetManager: Starting task 31.0 in stage 13.0 (TID 58) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9263 bytes) +26/04/01 06:31:27 INFO TaskSetManager: Finished task 27.0 in stage 13.0 (TID 54) in 1271 ms on 10.0.0.133 (executor driver) (28/64) +26/04/01 06:31:27 INFO Executor: Running task 31.0 in stage 13.0 (TID 58) +26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:28 INFO Executor: Finished task 30.0 in stage 13.0 (TID 57). 6675 bytes result sent to driver +26/04/01 06:31:28 INFO TaskSetManager: Starting task 32.0 in stage 13.0 (TID 59) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:28 INFO TaskSetManager: Finished task 30.0 in stage 13.0 (TID 57) in 1274 ms on 10.0.0.133 (executor driver) (29/64) +26/04/01 06:31:28 INFO Executor: Running task 32.0 in stage 13.0 (TID 59) +26/04/01 06:31:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:28 INFO Executor: Finished task 28.0 in stage 13.0 (TID 55). 6675 bytes result sent to driver +26/04/01 06:31:28 INFO TaskSetManager: Starting task 33.0 in stage 13.0 (TID 60) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:28 INFO Executor: Running task 33.0 in stage 13.0 (TID 60) +26/04/01 06:31:28 INFO TaskSetManager: Finished task 28.0 in stage 13.0 (TID 55) in 1277 ms on 10.0.0.133 (executor driver) (30/64) +26/04/01 06:31:28 INFO Executor: Finished task 29.0 in stage 13.0 (TID 56). 6675 bytes result sent to driver +26/04/01 06:31:28 INFO TaskSetManager: Starting task 34.0 in stage 13.0 (TID 61) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:28 INFO Executor: Running task 34.0 in stage 13.0 (TID 61) +26/04/01 06:31:28 INFO TaskSetManager: Finished task 29.0 in stage 13.0 (TID 56) in 1276 ms on 10.0.0.133 (executor driver) (31/64) +26/04/01 06:31:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:29 INFO Executor: Finished task 31.0 in stage 13.0 (TID 58). 6675 bytes result sent to driver +26/04/01 06:31:29 INFO TaskSetManager: Starting task 35.0 in stage 13.0 (TID 62) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:31:29 INFO TaskSetManager: Finished task 31.0 in stage 13.0 (TID 58) in 1276 ms on 10.0.0.133 (executor driver) (32/64) +26/04/01 06:31:29 INFO Executor: Running task 35.0 in stage 13.0 (TID 62) +26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:29 INFO Executor: Finished task 33.0 in stage 13.0 (TID 60). 6675 bytes result sent to driver +26/04/01 06:31:29 INFO Executor: Finished task 32.0 in stage 13.0 (TID 59). 6675 bytes result sent to driver +26/04/01 06:31:29 INFO TaskSetManager: Starting task 36.0 in stage 13.0 (TID 63) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:29 INFO Executor: Running task 36.0 in stage 13.0 (TID 63) +26/04/01 06:31:29 INFO TaskSetManager: Starting task 37.0 in stage 13.0 (TID 64) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:29 INFO TaskSetManager: Finished task 33.0 in stage 13.0 (TID 60) in 903 ms on 10.0.0.133 (executor driver) (33/64) +26/04/01 06:31:29 INFO Executor: Running task 37.0 in stage 13.0 (TID 64) +26/04/01 06:31:29 INFO TaskSetManager: Finished task 32.0 in stage 13.0 (TID 59) in 905 ms on 10.0.0.133 (executor driver) (34/64) +26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:29 INFO Executor: Finished task 34.0 in stage 13.0 (TID 61). 6675 bytes result sent to driver +26/04/01 06:31:29 INFO TaskSetManager: Starting task 38.0 in stage 13.0 (TID 65) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:29 INFO Executor: Running task 38.0 in stage 13.0 (TID 65) +26/04/01 06:31:29 INFO TaskSetManager: Finished task 34.0 in stage 13.0 (TID 61) in 905 ms on 10.0.0.133 (executor driver) (35/64) +26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:29 INFO Executor: Finished task 35.0 in stage 13.0 (TID 62). 6675 bytes result sent to driver +26/04/01 06:31:29 INFO TaskSetManager: Starting task 39.0 in stage 13.0 (TID 66) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:29 INFO TaskSetManager: Finished task 35.0 in stage 13.0 (TID 62) in 905 ms on 10.0.0.133 (executor driver) (36/64) +26/04/01 06:31:29 INFO Executor: Running task 39.0 in stage 13.0 (TID 66) +26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:30 INFO Executor: Finished task 36.0 in stage 13.0 (TID 63). 6675 bytes result sent to driver +26/04/01 06:31:30 INFO TaskSetManager: Starting task 40.0 in stage 13.0 (TID 67) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:31:30 INFO TaskSetManager: Finished task 36.0 in stage 13.0 (TID 63) in 900 ms on 10.0.0.133 (executor driver) (37/64) +26/04/01 06:31:30 INFO Executor: Running task 40.0 in stage 13.0 (TID 67) +26/04/01 06:31:30 INFO Executor: Finished task 37.0 in stage 13.0 (TID 64). 6675 bytes result sent to driver +26/04/01 06:31:30 INFO TaskSetManager: Starting task 41.0 in stage 13.0 (TID 68) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:30 INFO TaskSetManager: Finished task 37.0 in stage 13.0 (TID 64) in 901 ms on 10.0.0.133 (executor driver) (38/64) +26/04/01 06:31:30 INFO Executor: Running task 41.0 in stage 13.0 (TID 68) +26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:30 INFO Executor: Finished task 38.0 in stage 13.0 (TID 65). 6675 bytes result sent to driver +26/04/01 06:31:30 INFO TaskSetManager: Starting task 42.0 in stage 13.0 (TID 69) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:30 INFO TaskSetManager: Finished task 38.0 in stage 13.0 (TID 65) in 906 ms on 10.0.0.133 (executor driver) (39/64) +26/04/01 06:31:30 INFO Executor: Running task 42.0 in stage 13.0 (TID 69) +26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:30 INFO Executor: Finished task 39.0 in stage 13.0 (TID 66). 6675 bytes result sent to driver +26/04/01 06:31:30 INFO TaskSetManager: Starting task 43.0 in stage 13.0 (TID 70) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:31:30 INFO TaskSetManager: Finished task 39.0 in stage 13.0 (TID 66) in 907 ms on 10.0.0.133 (executor driver) (40/64) +26/04/01 06:31:30 INFO Executor: Running task 43.0 in stage 13.0 (TID 70) +26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:31 INFO Executor: Finished task 40.0 in stage 13.0 (TID 67). 6675 bytes result sent to driver +26/04/01 06:31:31 INFO TaskSetManager: Starting task 44.0 in stage 13.0 (TID 71) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:31:31 INFO Executor: Finished task 41.0 in stage 13.0 (TID 68). 6675 bytes result sent to driver +26/04/01 06:31:31 INFO TaskSetManager: Finished task 40.0 in stage 13.0 (TID 67) in 906 ms on 10.0.0.133 (executor driver) (41/64) +26/04/01 06:31:31 INFO Executor: Running task 44.0 in stage 13.0 (TID 71) +26/04/01 06:31:31 INFO TaskSetManager: Starting task 45.0 in stage 13.0 (TID 72) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:31 INFO TaskSetManager: Finished task 41.0 in stage 13.0 (TID 68) in 905 ms on 10.0.0.133 (executor driver) (42/64) +26/04/01 06:31:31 INFO Executor: Running task 45.0 in stage 13.0 (TID 72) +26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:31 INFO Executor: Finished task 42.0 in stage 13.0 (TID 69). 6675 bytes result sent to driver +26/04/01 06:31:31 INFO TaskSetManager: Starting task 46.0 in stage 13.0 (TID 73) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:31 INFO TaskSetManager: Finished task 42.0 in stage 13.0 (TID 69) in 905 ms on 10.0.0.133 (executor driver) (43/64) +26/04/01 06:31:31 INFO Executor: Running task 46.0 in stage 13.0 (TID 73) +26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:31 INFO Executor: Finished task 43.0 in stage 13.0 (TID 70). 6675 bytes result sent to driver +26/04/01 06:31:31 INFO TaskSetManager: Starting task 47.0 in stage 13.0 (TID 74) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:31:31 INFO TaskSetManager: Finished task 43.0 in stage 13.0 (TID 70) in 908 ms on 10.0.0.133 (executor driver) (44/64) +26/04/01 06:31:31 INFO Executor: Running task 47.0 in stage 13.0 (TID 74) +26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:32 INFO Executor: Finished task 45.0 in stage 13.0 (TID 72). 6675 bytes result sent to driver +26/04/01 06:31:32 INFO TaskSetManager: Starting task 48.0 in stage 13.0 (TID 75) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:32 INFO TaskSetManager: Finished task 45.0 in stage 13.0 (TID 72) in 906 ms on 10.0.0.133 (executor driver) (45/64) +26/04/01 06:31:32 INFO Executor: Running task 48.0 in stage 13.0 (TID 75) +26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:32 INFO Executor: Finished task 46.0 in stage 13.0 (TID 73). 6675 bytes result sent to driver +26/04/01 06:31:32 INFO Executor: Finished task 44.0 in stage 13.0 (TID 71). 6675 bytes result sent to driver +26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:32 INFO TaskSetManager: Starting task 49.0 in stage 13.0 (TID 76) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:32 INFO TaskSetManager: Finished task 46.0 in stage 13.0 (TID 73) in 903 ms on 10.0.0.133 (executor driver) (46/64) +26/04/01 06:31:32 INFO Executor: Running task 49.0 in stage 13.0 (TID 76) +26/04/01 06:31:32 INFO TaskSetManager: Starting task 50.0 in stage 13.0 (TID 77) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:32 INFO TaskSetManager: Finished task 44.0 in stage 13.0 (TID 71) in 909 ms on 10.0.0.133 (executor driver) (47/64) +26/04/01 06:31:32 INFO Executor: Running task 50.0 in stage 13.0 (TID 77) +26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:32 INFO Executor: Finished task 47.0 in stage 13.0 (TID 74). 6675 bytes result sent to driver +26/04/01 06:31:32 INFO TaskSetManager: Starting task 51.0 in stage 13.0 (TID 78) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:32 INFO TaskSetManager: Finished task 47.0 in stage 13.0 (TID 74) in 906 ms on 10.0.0.133 (executor driver) (48/64) +26/04/01 06:31:32 INFO Executor: Running task 51.0 in stage 13.0 (TID 78) +26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:33 INFO Executor: Finished task 48.0 in stage 13.0 (TID 75). 6675 bytes result sent to driver +26/04/01 06:31:33 INFO TaskSetManager: Starting task 52.0 in stage 13.0 (TID 79) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:33 INFO Executor: Running task 52.0 in stage 13.0 (TID 79) +26/04/01 06:31:33 INFO TaskSetManager: Finished task 48.0 in stage 13.0 (TID 75) in 914 ms on 10.0.0.133 (executor driver) (49/64) +26/04/01 06:31:33 INFO Executor: Finished task 49.0 in stage 13.0 (TID 76). 6675 bytes result sent to driver +26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:33 INFO TaskSetManager: Starting task 53.0 in stage 13.0 (TID 80) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:33 INFO TaskSetManager: Finished task 49.0 in stage 13.0 (TID 76) in 912 ms on 10.0.0.133 (executor driver) (50/64) +26/04/01 06:31:33 INFO Executor: Finished task 50.0 in stage 13.0 (TID 77). 6675 bytes result sent to driver +26/04/01 06:31:33 INFO Executor: Running task 53.0 in stage 13.0 (TID 80) +26/04/01 06:31:33 INFO TaskSetManager: Starting task 54.0 in stage 13.0 (TID 81) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:31:33 INFO TaskSetManager: Finished task 50.0 in stage 13.0 (TID 77) in 913 ms on 10.0.0.133 (executor driver) (51/64) +26/04/01 06:31:33 INFO Executor: Running task 54.0 in stage 13.0 (TID 81) +26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:33 INFO Executor: Finished task 51.0 in stage 13.0 (TID 78). 6675 bytes result sent to driver +26/04/01 06:31:33 INFO TaskSetManager: Starting task 55.0 in stage 13.0 (TID 82) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:33 INFO TaskSetManager: Finished task 51.0 in stage 13.0 (TID 78) in 915 ms on 10.0.0.133 (executor driver) (52/64) +26/04/01 06:31:33 INFO Executor: Running task 55.0 in stage 13.0 (TID 82) +26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:34 INFO Executor: Finished task 52.0 in stage 13.0 (TID 79). 6675 bytes result sent to driver +26/04/01 06:31:34 INFO TaskSetManager: Starting task 56.0 in stage 13.0 (TID 83) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:31:34 INFO TaskSetManager: Finished task 52.0 in stage 13.0 (TID 79) in 909 ms on 10.0.0.133 (executor driver) (53/64) +26/04/01 06:31:34 INFO Executor: Running task 56.0 in stage 13.0 (TID 83) +26/04/01 06:31:34 INFO Executor: Finished task 54.0 in stage 13.0 (TID 81). 6675 bytes result sent to driver +26/04/01 06:31:34 INFO TaskSetManager: Starting task 57.0 in stage 13.0 (TID 84) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:34 INFO Executor: Finished task 53.0 in stage 13.0 (TID 80). 6675 bytes result sent to driver +26/04/01 06:31:34 INFO TaskSetManager: Finished task 54.0 in stage 13.0 (TID 81) in 907 ms on 10.0.0.133 (executor driver) (54/64) +26/04/01 06:31:34 INFO Executor: Running task 57.0 in stage 13.0 (TID 84) +26/04/01 06:31:34 INFO TaskSetManager: Starting task 58.0 in stage 13.0 (TID 85) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:34 INFO TaskSetManager: Finished task 53.0 in stage 13.0 (TID 80) in 908 ms on 10.0.0.133 (executor driver) (55/64) +26/04/01 06:31:34 INFO Executor: Running task 58.0 in stage 13.0 (TID 85) +26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:34 INFO Executor: Finished task 55.0 in stage 13.0 (TID 82). 6675 bytes result sent to driver +26/04/01 06:31:34 INFO TaskSetManager: Starting task 59.0 in stage 13.0 (TID 86) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:34 INFO TaskSetManager: Finished task 55.0 in stage 13.0 (TID 82) in 905 ms on 10.0.0.133 (executor driver) (56/64) +26/04/01 06:31:34 INFO Executor: Running task 59.0 in stage 13.0 (TID 86) +26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:35 INFO Executor: Finished task 56.0 in stage 13.0 (TID 83). 6675 bytes result sent to driver +26/04/01 06:31:35 INFO TaskSetManager: Starting task 60.0 in stage 13.0 (TID 87) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:35 INFO Executor: Running task 60.0 in stage 13.0 (TID 87) +26/04/01 06:31:35 INFO TaskSetManager: Finished task 56.0 in stage 13.0 (TID 83) in 907 ms on 10.0.0.133 (executor driver) (57/64) +26/04/01 06:31:35 INFO Executor: Finished task 58.0 in stage 13.0 (TID 85). 6675 bytes result sent to driver +26/04/01 06:31:35 INFO TaskSetManager: Starting task 61.0 in stage 13.0 (TID 88) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:35 INFO TaskSetManager: Finished task 58.0 in stage 13.0 (TID 85) in 906 ms on 10.0.0.133 (executor driver) (58/64) +26/04/01 06:31:35 INFO Executor: Running task 61.0 in stage 13.0 (TID 88) +26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:35 INFO Executor: Finished task 57.0 in stage 13.0 (TID 84). 6675 bytes result sent to driver +26/04/01 06:31:35 INFO TaskSetManager: Starting task 62.0 in stage 13.0 (TID 89) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:31:35 INFO Executor: Running task 62.0 in stage 13.0 (TID 89) +26/04/01 06:31:35 INFO TaskSetManager: Finished task 57.0 in stage 13.0 (TID 84) in 908 ms on 10.0.0.133 (executor driver) (59/64) +26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:35 INFO Executor: Finished task 59.0 in stage 13.0 (TID 86). 6675 bytes result sent to driver +26/04/01 06:31:35 INFO TaskSetManager: Starting task 63.0 in stage 13.0 (TID 90) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:31:35 INFO TaskSetManager: Finished task 59.0 in stage 13.0 (TID 86) in 907 ms on 10.0.0.133 (executor driver) (60/64) +26/04/01 06:31:35 INFO Executor: Running task 63.0 in stage 13.0 (TID 90) +26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:36 INFO Executor: Finished task 62.0 in stage 13.0 (TID 89). 6675 bytes result sent to driver +26/04/01 06:31:36 INFO Executor: Finished task 61.0 in stage 13.0 (TID 88). 6675 bytes result sent to driver +26/04/01 06:31:36 INFO TaskSetManager: Starting task 0.0 in stage 14.0 (TID 91) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:36 INFO Executor: Running task 0.0 in stage 14.0 (TID 91) +26/04/01 06:31:36 INFO TaskSetManager: Starting task 1.0 in stage 14.0 (TID 92) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:31:36 INFO Executor: Running task 1.0 in stage 14.0 (TID 92) +26/04/01 06:31:36 INFO TaskSetManager: Finished task 62.0 in stage 13.0 (TID 89) in 901 ms on 10.0.0.133 (executor driver) (61/64) +26/04/01 06:31:36 INFO TaskSetManager: Finished task 61.0 in stage 13.0 (TID 88) in 902 ms on 10.0.0.133 (executor driver) (62/64) +26/04/01 06:31:36 INFO Executor: Finished task 60.0 in stage 13.0 (TID 87). 6675 bytes result sent to driver +26/04/01 06:31:36 INFO TaskSetManager: Starting task 2.0 in stage 14.0 (TID 93) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:31:36 INFO TaskSetManager: Finished task 60.0 in stage 13.0 (TID 87) in 903 ms on 10.0.0.133 (executor driver) (63/64) +26/04/01 06:31:36 INFO Executor: Running task 2.0 in stage 14.0 (TID 93) +26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:36 INFO Executor: Finished task 63.0 in stage 13.0 (TID 90). 6675 bytes result sent to driver +26/04/01 06:31:36 INFO TaskSetManager: Starting task 3.0 in stage 14.0 (TID 94) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:31:36 INFO Executor: Running task 3.0 in stage 14.0 (TID 94) +26/04/01 06:31:36 INFO TaskSetManager: Finished task 63.0 in stage 13.0 (TID 90) in 906 ms on 10.0.0.133 (executor driver) (64/64) +26/04/01 06:31:36 INFO TaskSchedulerImpl: Removed TaskSet 13.0, whose tasks have all completed, from pool +26/04/01 06:31:36 INFO DAGScheduler: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 20.348 s +26/04/01 06:31:36 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:31:36 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 16, ShuffleMapStage 14) +26/04/01 06:31:36 INFO DAGScheduler: waiting: Set() +26/04/01 06:31:36 INFO DAGScheduler: failed: Set() +26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:36 INFO ShufflePartitionsUtil: For shuffle(1, 2), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 06:31:36 INFO DAGScheduler: Registering RDD 44 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 5 +26/04/01 06:31:36 INFO DAGScheduler: Got map stage job 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 6 output partitions +26/04/01 06:31:36 INFO DAGScheduler: Final stage: ShuffleMapStage 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:31:36 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 17, ShuffleMapStage 18) +26/04/01 06:31:36 INFO DAGScheduler: Missing parents: List() +26/04/01 06:31:36 INFO DAGScheduler: Submitting ShuffleMapStage 19 (MapPartitionsRDD[44] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:31:36 INFO MemoryStore: Block broadcast_24 stored as values in memory (estimated size 22.7 KiB, free 12.6 GiB) +26/04/01 06:31:36 INFO MemoryStore: Block broadcast_24_piece0 stored as bytes in memory (estimated size 9.5 KiB, free 12.6 GiB) +26/04/01 06:31:36 INFO BlockManagerInfo: Added broadcast_24_piece0 in memory on 10.0.0.133:58442 (size: 9.5 KiB, free: 12.6 GiB) +26/04/01 06:31:36 INFO SparkContext: Created broadcast 24 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:31:36 INFO DAGScheduler: Submitting 6 missing tasks from ShuffleMapStage 19 (MapPartitionsRDD[44] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5)) +26/04/01 06:31:36 INFO TaskSchedulerImpl: Adding task set 19.0 with 6 tasks resource profile 0 +26/04/01 06:31:40 INFO Executor: Finished task 1.0 in stage 14.0 (TID 92). 6563 bytes result sent to driver +26/04/01 06:31:40 INFO Executor: Finished task 0.0 in stage 14.0 (TID 91). 6563 bytes result sent to driver +26/04/01 06:31:40 INFO TaskSetManager: Starting task 4.0 in stage 14.0 (TID 95) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:31:40 INFO Executor: Finished task 2.0 in stage 14.0 (TID 93). 6563 bytes result sent to driver +26/04/01 06:31:40 INFO Executor: Running task 4.0 in stage 14.0 (TID 95) +26/04/01 06:31:40 INFO TaskSetManager: Starting task 5.0 in stage 14.0 (TID 96) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:31:40 INFO TaskSetManager: Finished task 1.0 in stage 14.0 (TID 92) in 4141 ms on 10.0.0.133 (executor driver) (1/208) +26/04/01 06:31:40 INFO Executor: Running task 5.0 in stage 14.0 (TID 96) +26/04/01 06:31:40 INFO TaskSetManager: Starting task 6.0 in stage 14.0 (TID 97) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:40 INFO TaskSetManager: Finished task 2.0 in stage 14.0 (TID 93) in 4141 ms on 10.0.0.133 (executor driver) (2/208) +26/04/01 06:31:40 INFO Executor: Running task 6.0 in stage 14.0 (TID 97) +26/04/01 06:31:40 INFO TaskSetManager: Finished task 0.0 in stage 14.0 (TID 91) in 4143 ms on 10.0.0.133 (executor driver) (3/208) +26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:40 INFO Executor: Finished task 3.0 in stage 14.0 (TID 94). 6563 bytes result sent to driver +26/04/01 06:31:40 INFO TaskSetManager: Starting task 7.0 in stage 14.0 (TID 98) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:31:40 INFO TaskSetManager: Finished task 3.0 in stage 14.0 (TID 94) in 4144 ms on 10.0.0.133 (executor driver) (4/208) +26/04/01 06:31:40 INFO Executor: Running task 7.0 in stage 14.0 (TID 98) +26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:44 INFO Executor: Finished task 4.0 in stage 14.0 (TID 95). 6563 bytes result sent to driver +26/04/01 06:31:44 INFO TaskSetManager: Starting task 8.0 in stage 14.0 (TID 99) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:31:44 INFO Executor: Running task 8.0 in stage 14.0 (TID 99) +26/04/01 06:31:44 INFO TaskSetManager: Finished task 4.0 in stage 14.0 (TID 95) in 4138 ms on 10.0.0.133 (executor driver) (5/208) +26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:44 INFO Executor: Finished task 6.0 in stage 14.0 (TID 97). 6563 bytes result sent to driver +26/04/01 06:31:44 INFO TaskSetManager: Starting task 9.0 in stage 14.0 (TID 100) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:31:44 INFO Executor: Running task 9.0 in stage 14.0 (TID 100) +26/04/01 06:31:44 INFO TaskSetManager: Finished task 6.0 in stage 14.0 (TID 97) in 4140 ms on 10.0.0.133 (executor driver) (6/208) +26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:44 INFO Executor: Finished task 5.0 in stage 14.0 (TID 96). 6563 bytes result sent to driver +26/04/01 06:31:44 INFO TaskSetManager: Starting task 10.0 in stage 14.0 (TID 101) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:31:44 INFO TaskSetManager: Finished task 5.0 in stage 14.0 (TID 96) in 4143 ms on 10.0.0.133 (executor driver) (7/208) +26/04/01 06:31:44 INFO Executor: Running task 10.0 in stage 14.0 (TID 101) +26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:44 INFO Executor: Finished task 7.0 in stage 14.0 (TID 98). 6563 bytes result sent to driver +26/04/01 06:31:44 INFO TaskSetManager: Starting task 11.0 in stage 14.0 (TID 102) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:31:44 INFO TaskSetManager: Finished task 7.0 in stage 14.0 (TID 98) in 4140 ms on 10.0.0.133 (executor driver) (8/208) +26/04/01 06:31:44 INFO Executor: Running task 11.0 in stage 14.0 (TID 102) +26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:48 INFO Executor: Finished task 9.0 in stage 14.0 (TID 100). 6563 bytes result sent to driver +26/04/01 06:31:48 INFO TaskSetManager: Starting task 12.0 in stage 14.0 (TID 103) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:31:48 INFO Executor: Running task 12.0 in stage 14.0 (TID 103) +26/04/01 06:31:48 INFO TaskSetManager: Finished task 9.0 in stage 14.0 (TID 100) in 4138 ms on 10.0.0.133 (executor driver) (9/208) +26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:48 INFO Executor: Finished task 8.0 in stage 14.0 (TID 99). 6563 bytes result sent to driver +26/04/01 06:31:48 INFO TaskSetManager: Starting task 13.0 in stage 14.0 (TID 104) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:31:48 INFO TaskSetManager: Finished task 8.0 in stage 14.0 (TID 99) in 4143 ms on 10.0.0.133 (executor driver) (10/208) +26/04/01 06:31:48 INFO Executor: Running task 13.0 in stage 14.0 (TID 104) +26/04/01 06:31:48 INFO Executor: Finished task 10.0 in stage 14.0 (TID 101). 6563 bytes result sent to driver +26/04/01 06:31:48 INFO TaskSetManager: Starting task 14.0 in stage 14.0 (TID 105) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:31:48 INFO Executor: Running task 14.0 in stage 14.0 (TID 105) +26/04/01 06:31:48 INFO TaskSetManager: Finished task 10.0 in stage 14.0 (TID 101) in 4139 ms on 10.0.0.133 (executor driver) (11/208) +26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:48 INFO Executor: Finished task 11.0 in stage 14.0 (TID 102). 6563 bytes result sent to driver +26/04/01 06:31:48 INFO TaskSetManager: Starting task 15.0 in stage 14.0 (TID 106) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:31:48 INFO Executor: Running task 15.0 in stage 14.0 (TID 106) +26/04/01 06:31:48 INFO TaskSetManager: Finished task 11.0 in stage 14.0 (TID 102) in 4152 ms on 10.0.0.133 (executor driver) (12/208) +26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:52 INFO Executor: Finished task 12.0 in stage 14.0 (TID 103). 6563 bytes result sent to driver +26/04/01 06:31:52 INFO Executor: Finished task 13.0 in stage 14.0 (TID 104). 6563 bytes result sent to driver +26/04/01 06:31:52 INFO TaskSetManager: Starting task 16.0 in stage 14.0 (TID 107) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:31:52 INFO TaskSetManager: Starting task 17.0 in stage 14.0 (TID 108) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:31:52 INFO Executor: Running task 16.0 in stage 14.0 (TID 107) +26/04/01 06:31:52 INFO TaskSetManager: Finished task 12.0 in stage 14.0 (TID 103) in 4152 ms on 10.0.0.133 (executor driver) (13/208) +26/04/01 06:31:52 INFO Executor: Running task 17.0 in stage 14.0 (TID 108) +26/04/01 06:31:52 INFO TaskSetManager: Finished task 13.0 in stage 14.0 (TID 104) in 4149 ms on 10.0.0.133 (executor driver) (14/208) +26/04/01 06:31:52 INFO Executor: Finished task 14.0 in stage 14.0 (TID 105). 6563 bytes result sent to driver +26/04/01 06:31:52 INFO TaskSetManager: Starting task 18.0 in stage 14.0 (TID 109) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:31:52 INFO TaskSetManager: Finished task 14.0 in stage 14.0 (TID 105) in 4149 ms on 10.0.0.133 (executor driver) (15/208) +26/04/01 06:31:52 INFO Executor: Running task 18.0 in stage 14.0 (TID 109) +26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:52 INFO Executor: Finished task 15.0 in stage 14.0 (TID 106). 6563 bytes result sent to driver +26/04/01 06:31:52 INFO TaskSetManager: Starting task 19.0 in stage 14.0 (TID 110) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:31:52 INFO TaskSetManager: Finished task 15.0 in stage 14.0 (TID 106) in 4147 ms on 10.0.0.133 (executor driver) (16/208) +26/04/01 06:31:52 INFO Executor: Running task 19.0 in stage 14.0 (TID 110) +26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:56 INFO Executor: Finished task 17.0 in stage 14.0 (TID 108). 6563 bytes result sent to driver +26/04/01 06:31:56 INFO Executor: Finished task 18.0 in stage 14.0 (TID 109). 6563 bytes result sent to driver +26/04/01 06:31:56 INFO TaskSetManager: Starting task 20.0 in stage 14.0 (TID 111) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:31:56 INFO Executor: Finished task 16.0 in stage 14.0 (TID 107). 6563 bytes result sent to driver +26/04/01 06:31:56 INFO Executor: Running task 20.0 in stage 14.0 (TID 111) +26/04/01 06:31:56 INFO TaskSetManager: Starting task 21.0 in stage 14.0 (TID 112) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:31:56 INFO TaskSetManager: Finished task 17.0 in stage 14.0 (TID 108) in 4143 ms on 10.0.0.133 (executor driver) (17/208) +26/04/01 06:31:56 INFO Executor: Running task 21.0 in stage 14.0 (TID 112) +26/04/01 06:31:56 INFO TaskSetManager: Finished task 18.0 in stage 14.0 (TID 109) in 4142 ms on 10.0.0.133 (executor driver) (18/208) +26/04/01 06:31:56 INFO TaskSetManager: Starting task 22.0 in stage 14.0 (TID 113) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:31:56 INFO TaskSetManager: Finished task 16.0 in stage 14.0 (TID 107) in 4143 ms on 10.0.0.133 (executor driver) (19/208) +26/04/01 06:31:56 INFO Executor: Running task 22.0 in stage 14.0 (TID 113) +26/04/01 06:31:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:56 INFO Executor: Finished task 19.0 in stage 14.0 (TID 110). 6563 bytes result sent to driver +26/04/01 06:31:56 INFO TaskSetManager: Starting task 23.0 in stage 14.0 (TID 114) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:31:56 INFO TaskSetManager: Finished task 19.0 in stage 14.0 (TID 110) in 4136 ms on 10.0.0.133 (executor driver) (20/208) +26/04/01 06:31:56 INFO Executor: Running task 23.0 in stage 14.0 (TID 114) +26/04/01 06:31:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:31:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:01 INFO Executor: Finished task 21.0 in stage 14.0 (TID 112). 6563 bytes result sent to driver +26/04/01 06:32:01 INFO TaskSetManager: Starting task 24.0 in stage 14.0 (TID 115) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:32:01 INFO TaskSetManager: Finished task 21.0 in stage 14.0 (TID 112) in 4125 ms on 10.0.0.133 (executor driver) (21/208) +26/04/01 06:32:01 INFO Executor: Running task 24.0 in stage 14.0 (TID 115) +26/04/01 06:32:01 INFO Executor: Finished task 20.0 in stage 14.0 (TID 111). 6563 bytes result sent to driver +26/04/01 06:32:01 INFO TaskSetManager: Starting task 25.0 in stage 14.0 (TID 116) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:01 INFO TaskSetManager: Finished task 20.0 in stage 14.0 (TID 111) in 4126 ms on 10.0.0.133 (executor driver) (22/208) +26/04/01 06:32:01 INFO Executor: Running task 25.0 in stage 14.0 (TID 116) +26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:01 INFO Executor: Finished task 22.0 in stage 14.0 (TID 113). 6563 bytes result sent to driver +26/04/01 06:32:01 INFO TaskSetManager: Starting task 26.0 in stage 14.0 (TID 117) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:01 INFO TaskSetManager: Finished task 22.0 in stage 14.0 (TID 113) in 4129 ms on 10.0.0.133 (executor driver) (23/208) +26/04/01 06:32:01 INFO Executor: Running task 26.0 in stage 14.0 (TID 117) +26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:01 INFO Executor: Finished task 23.0 in stage 14.0 (TID 114). 6563 bytes result sent to driver +26/04/01 06:32:01 INFO TaskSetManager: Starting task 27.0 in stage 14.0 (TID 118) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:01 INFO TaskSetManager: Finished task 23.0 in stage 14.0 (TID 114) in 4123 ms on 10.0.0.133 (executor driver) (24/208) +26/04/01 06:32:01 INFO Executor: Running task 27.0 in stage 14.0 (TID 118) +26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:01 INFO BlockManagerInfo: Removed broadcast_17_piece0 on 10.0.0.133:58442 in memory (size: 8.1 KiB, free: 12.6 GiB) +26/04/01 06:32:05 INFO Executor: Finished task 24.0 in stage 14.0 (TID 115). 6606 bytes result sent to driver +26/04/01 06:32:05 INFO TaskSetManager: Starting task 28.0 in stage 14.0 (TID 119) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:05 INFO TaskSetManager: Finished task 24.0 in stage 14.0 (TID 115) in 4070 ms on 10.0.0.133 (executor driver) (25/208) +26/04/01 06:32:05 INFO Executor: Running task 28.0 in stage 14.0 (TID 119) +26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:05 INFO Executor: Finished task 25.0 in stage 14.0 (TID 116). 6606 bytes result sent to driver +26/04/01 06:32:05 INFO TaskSetManager: Starting task 29.0 in stage 14.0 (TID 120) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:05 INFO Executor: Running task 29.0 in stage 14.0 (TID 120) +26/04/01 06:32:05 INFO TaskSetManager: Finished task 25.0 in stage 14.0 (TID 116) in 4078 ms on 10.0.0.133 (executor driver) (26/208) +26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:05 INFO Executor: Finished task 26.0 in stage 14.0 (TID 117). 6606 bytes result sent to driver +26/04/01 06:32:05 INFO TaskSetManager: Starting task 30.0 in stage 14.0 (TID 121) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:32:05 INFO TaskSetManager: Finished task 26.0 in stage 14.0 (TID 117) in 4079 ms on 10.0.0.133 (executor driver) (27/208) +26/04/01 06:32:05 INFO Executor: Running task 30.0 in stage 14.0 (TID 121) +26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:05 INFO Executor: Finished task 27.0 in stage 14.0 (TID 118). 6606 bytes result sent to driver +26/04/01 06:32:05 INFO TaskSetManager: Starting task 31.0 in stage 14.0 (TID 122) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:05 INFO TaskSetManager: Finished task 27.0 in stage 14.0 (TID 118) in 4078 ms on 10.0.0.133 (executor driver) (28/208) +26/04/01 06:32:05 INFO Executor: Running task 31.0 in stage 14.0 (TID 122) +26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:09 INFO Executor: Finished task 28.0 in stage 14.0 (TID 119). 6563 bytes result sent to driver +26/04/01 06:32:09 INFO TaskSetManager: Starting task 32.0 in stage 14.0 (TID 123) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:09 INFO TaskSetManager: Finished task 28.0 in stage 14.0 (TID 119) in 4079 ms on 10.0.0.133 (executor driver) (29/208) +26/04/01 06:32:09 INFO Executor: Running task 32.0 in stage 14.0 (TID 123) +26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:09 INFO Executor: Finished task 29.0 in stage 14.0 (TID 120). 6563 bytes result sent to driver +26/04/01 06:32:09 INFO TaskSetManager: Starting task 33.0 in stage 14.0 (TID 124) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:09 INFO TaskSetManager: Finished task 29.0 in stage 14.0 (TID 120) in 4085 ms on 10.0.0.133 (executor driver) (30/208) +26/04/01 06:32:09 INFO Executor: Running task 33.0 in stage 14.0 (TID 124) +26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:09 INFO Executor: Finished task 30.0 in stage 14.0 (TID 121). 6563 bytes result sent to driver +26/04/01 06:32:09 INFO TaskSetManager: Starting task 34.0 in stage 14.0 (TID 125) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:09 INFO TaskSetManager: Finished task 30.0 in stage 14.0 (TID 121) in 4083 ms on 10.0.0.133 (executor driver) (31/208) +26/04/01 06:32:09 INFO Executor: Running task 34.0 in stage 14.0 (TID 125) +26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:09 INFO Executor: Finished task 31.0 in stage 14.0 (TID 122). 6563 bytes result sent to driver +26/04/01 06:32:09 INFO TaskSetManager: Starting task 35.0 in stage 14.0 (TID 126) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:09 INFO TaskSetManager: Finished task 31.0 in stage 14.0 (TID 122) in 4089 ms on 10.0.0.133 (executor driver) (32/208) +26/04/01 06:32:09 INFO Executor: Running task 35.0 in stage 14.0 (TID 126) +26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:13 INFO Executor: Finished task 32.0 in stage 14.0 (TID 123). 6563 bytes result sent to driver +26/04/01 06:32:13 INFO Executor: Finished task 33.0 in stage 14.0 (TID 124). 6563 bytes result sent to driver +26/04/01 06:32:13 INFO TaskSetManager: Starting task 36.0 in stage 14.0 (TID 127) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:32:13 INFO TaskSetManager: Finished task 32.0 in stage 14.0 (TID 123) in 4077 ms on 10.0.0.133 (executor driver) (33/208) +26/04/01 06:32:13 INFO Executor: Running task 36.0 in stage 14.0 (TID 127) +26/04/01 06:32:13 INFO TaskSetManager: Starting task 37.0 in stage 14.0 (TID 128) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:13 INFO TaskSetManager: Finished task 33.0 in stage 14.0 (TID 124) in 4063 ms on 10.0.0.133 (executor driver) (34/208) +26/04/01 06:32:13 INFO Executor: Running task 37.0 in stage 14.0 (TID 128) +26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:13 INFO Executor: Finished task 34.0 in stage 14.0 (TID 125). 6563 bytes result sent to driver +26/04/01 06:32:13 INFO TaskSetManager: Starting task 38.0 in stage 14.0 (TID 129) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:13 INFO Executor: Running task 38.0 in stage 14.0 (TID 129) +26/04/01 06:32:13 INFO TaskSetManager: Finished task 34.0 in stage 14.0 (TID 125) in 4075 ms on 10.0.0.133 (executor driver) (35/208) +26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:13 INFO Executor: Finished task 35.0 in stage 14.0 (TID 126). 6563 bytes result sent to driver +26/04/01 06:32:13 INFO TaskSetManager: Starting task 39.0 in stage 14.0 (TID 130) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:13 INFO Executor: Running task 39.0 in stage 14.0 (TID 130) +26/04/01 06:32:13 INFO TaskSetManager: Finished task 35.0 in stage 14.0 (TID 126) in 4072 ms on 10.0.0.133 (executor driver) (36/208) +26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:17 INFO Executor: Finished task 36.0 in stage 14.0 (TID 127). 6563 bytes result sent to driver +26/04/01 06:32:17 INFO TaskSetManager: Starting task 40.0 in stage 14.0 (TID 131) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:17 INFO Executor: Finished task 37.0 in stage 14.0 (TID 128). 6563 bytes result sent to driver +26/04/01 06:32:17 INFO Executor: Running task 40.0 in stage 14.0 (TID 131) +26/04/01 06:32:17 INFO TaskSetManager: Starting task 41.0 in stage 14.0 (TID 132) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:17 INFO TaskSetManager: Finished task 36.0 in stage 14.0 (TID 127) in 4131 ms on 10.0.0.133 (executor driver) (37/208) +26/04/01 06:32:17 INFO TaskSetManager: Finished task 37.0 in stage 14.0 (TID 128) in 4130 ms on 10.0.0.133 (executor driver) (38/208) +26/04/01 06:32:17 INFO Executor: Running task 41.0 in stage 14.0 (TID 132) +26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:17 INFO Executor: Finished task 38.0 in stage 14.0 (TID 129). 6563 bytes result sent to driver +26/04/01 06:32:17 INFO TaskSetManager: Starting task 42.0 in stage 14.0 (TID 133) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:32:17 INFO Executor: Running task 42.0 in stage 14.0 (TID 133) +26/04/01 06:32:17 INFO TaskSetManager: Finished task 38.0 in stage 14.0 (TID 129) in 4119 ms on 10.0.0.133 (executor driver) (39/208) +26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:17 INFO Executor: Finished task 39.0 in stage 14.0 (TID 130). 6563 bytes result sent to driver +26/04/01 06:32:17 INFO TaskSetManager: Starting task 43.0 in stage 14.0 (TID 134) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:32:17 INFO TaskSetManager: Finished task 39.0 in stage 14.0 (TID 130) in 4134 ms on 10.0.0.133 (executor driver) (40/208) +26/04/01 06:32:17 INFO Executor: Running task 43.0 in stage 14.0 (TID 134) +26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:21 INFO Executor: Finished task 42.0 in stage 14.0 (TID 133). 6563 bytes result sent to driver +26/04/01 06:32:21 INFO TaskSetManager: Starting task 44.0 in stage 14.0 (TID 135) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:21 INFO TaskSetManager: Finished task 42.0 in stage 14.0 (TID 133) in 4075 ms on 10.0.0.133 (executor driver) (41/208) +26/04/01 06:32:21 INFO Executor: Running task 44.0 in stage 14.0 (TID 135) +26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:21 INFO Executor: Finished task 43.0 in stage 14.0 (TID 134). 6563 bytes result sent to driver +26/04/01 06:32:21 INFO TaskSetManager: Starting task 45.0 in stage 14.0 (TID 136) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:21 INFO TaskSetManager: Finished task 43.0 in stage 14.0 (TID 134) in 4075 ms on 10.0.0.133 (executor driver) (42/208) +26/04/01 06:32:21 INFO Executor: Running task 45.0 in stage 14.0 (TID 136) +26/04/01 06:32:21 INFO Executor: Finished task 40.0 in stage 14.0 (TID 131). 6563 bytes result sent to driver +26/04/01 06:32:21 INFO TaskSetManager: Starting task 46.0 in stage 14.0 (TID 137) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:21 INFO Executor: Running task 46.0 in stage 14.0 (TID 137) +26/04/01 06:32:21 INFO TaskSetManager: Finished task 40.0 in stage 14.0 (TID 131) in 4136 ms on 10.0.0.133 (executor driver) (43/208) +26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:21 INFO Executor: Finished task 41.0 in stage 14.0 (TID 132). 6563 bytes result sent to driver +26/04/01 06:32:21 INFO TaskSetManager: Starting task 47.0 in stage 14.0 (TID 138) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:21 INFO TaskSetManager: Finished task 41.0 in stage 14.0 (TID 132) in 4153 ms on 10.0.0.133 (executor driver) (44/208) +26/04/01 06:32:21 INFO Executor: Running task 47.0 in stage 14.0 (TID 138) +26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:25 INFO Executor: Finished task 45.0 in stage 14.0 (TID 136). 6563 bytes result sent to driver +26/04/01 06:32:25 INFO TaskSetManager: Starting task 48.0 in stage 14.0 (TID 139) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:32:25 INFO TaskSetManager: Finished task 45.0 in stage 14.0 (TID 136) in 3930 ms on 10.0.0.133 (executor driver) (45/208) +26/04/01 06:32:25 INFO Executor: Running task 48.0 in stage 14.0 (TID 139) +26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:25 INFO Executor: Finished task 46.0 in stage 14.0 (TID 137). 6563 bytes result sent to driver +26/04/01 06:32:25 INFO TaskSetManager: Starting task 49.0 in stage 14.0 (TID 140) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:25 INFO TaskSetManager: Finished task 46.0 in stage 14.0 (TID 137) in 4082 ms on 10.0.0.133 (executor driver) (46/208) +26/04/01 06:32:25 INFO Executor: Running task 49.0 in stage 14.0 (TID 140) +26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:25 INFO Executor: Finished task 47.0 in stage 14.0 (TID 138). 6563 bytes result sent to driver +26/04/01 06:32:25 INFO TaskSetManager: Starting task 50.0 in stage 14.0 (TID 141) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:25 INFO Executor: Running task 50.0 in stage 14.0 (TID 141) +26/04/01 06:32:25 INFO TaskSetManager: Finished task 47.0 in stage 14.0 (TID 138) in 4067 ms on 10.0.0.133 (executor driver) (47/208) +26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:25 INFO Executor: Finished task 44.0 in stage 14.0 (TID 135). 6563 bytes result sent to driver +26/04/01 06:32:25 INFO TaskSetManager: Starting task 51.0 in stage 14.0 (TID 142) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:25 INFO TaskSetManager: Finished task 44.0 in stage 14.0 (TID 135) in 4221 ms on 10.0.0.133 (executor driver) (48/208) +26/04/01 06:32:25 INFO Executor: Running task 51.0 in stage 14.0 (TID 142) +26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:29 INFO Executor: Finished task 48.0 in stage 14.0 (TID 139). 6563 bytes result sent to driver +26/04/01 06:32:29 INFO TaskSetManager: Starting task 52.0 in stage 14.0 (TID 143) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:29 INFO Executor: Running task 52.0 in stage 14.0 (TID 143) +26/04/01 06:32:29 INFO TaskSetManager: Finished task 48.0 in stage 14.0 (TID 139) in 4132 ms on 10.0.0.133 (executor driver) (49/208) +26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:29 INFO Executor: Finished task 49.0 in stage 14.0 (TID 140). 6563 bytes result sent to driver +26/04/01 06:32:29 INFO TaskSetManager: Starting task 53.0 in stage 14.0 (TID 144) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:29 INFO TaskSetManager: Finished task 49.0 in stage 14.0 (TID 140) in 4121 ms on 10.0.0.133 (executor driver) (50/208) +26/04/01 06:32:29 INFO Executor: Running task 53.0 in stage 14.0 (TID 144) +26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:29 INFO Executor: Finished task 50.0 in stage 14.0 (TID 141). 6563 bytes result sent to driver +26/04/01 06:32:29 INFO TaskSetManager: Starting task 54.0 in stage 14.0 (TID 145) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:32:29 INFO TaskSetManager: Finished task 50.0 in stage 14.0 (TID 141) in 4121 ms on 10.0.0.133 (executor driver) (51/208) +26/04/01 06:32:29 INFO Executor: Running task 54.0 in stage 14.0 (TID 145) +26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:29 INFO Executor: Finished task 51.0 in stage 14.0 (TID 142). 6563 bytes result sent to driver +26/04/01 06:32:29 INFO TaskSetManager: Starting task 55.0 in stage 14.0 (TID 146) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:29 INFO TaskSetManager: Finished task 51.0 in stage 14.0 (TID 142) in 4132 ms on 10.0.0.133 (executor driver) (52/208) +26/04/01 06:32:29 INFO Executor: Running task 55.0 in stage 14.0 (TID 146) +26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:33 INFO Executor: Finished task 52.0 in stage 14.0 (TID 143). 6563 bytes result sent to driver +26/04/01 06:32:33 INFO TaskSetManager: Starting task 56.0 in stage 14.0 (TID 147) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:33 INFO TaskSetManager: Finished task 52.0 in stage 14.0 (TID 143) in 4148 ms on 10.0.0.133 (executor driver) (53/208) +26/04/01 06:32:33 INFO Executor: Running task 56.0 in stage 14.0 (TID 147) +26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:33 INFO Executor: Finished task 53.0 in stage 14.0 (TID 144). 6563 bytes result sent to driver +26/04/01 06:32:33 INFO TaskSetManager: Starting task 57.0 in stage 14.0 (TID 148) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:33 INFO TaskSetManager: Finished task 53.0 in stage 14.0 (TID 144) in 4129 ms on 10.0.0.133 (executor driver) (54/208) +26/04/01 06:32:33 INFO Executor: Running task 57.0 in stage 14.0 (TID 148) +26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:33 INFO Executor: Finished task 54.0 in stage 14.0 (TID 145). 6563 bytes result sent to driver +26/04/01 06:32:33 INFO TaskSetManager: Starting task 58.0 in stage 14.0 (TID 149) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:33 INFO Executor: Running task 58.0 in stage 14.0 (TID 149) +26/04/01 06:32:33 INFO TaskSetManager: Finished task 54.0 in stage 14.0 (TID 145) in 4130 ms on 10.0.0.133 (executor driver) (55/208) +26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:33 INFO Executor: Finished task 55.0 in stage 14.0 (TID 146). 6563 bytes result sent to driver +26/04/01 06:32:33 INFO TaskSetManager: Starting task 59.0 in stage 14.0 (TID 150) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:33 INFO TaskSetManager: Finished task 55.0 in stage 14.0 (TID 146) in 4129 ms on 10.0.0.133 (executor driver) (56/208) +26/04/01 06:32:33 INFO Executor: Running task 59.0 in stage 14.0 (TID 150) +26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:37 INFO Executor: Finished task 56.0 in stage 14.0 (TID 147). 6563 bytes result sent to driver +26/04/01 06:32:37 INFO TaskSetManager: Starting task 60.0 in stage 14.0 (TID 151) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:32:37 INFO TaskSetManager: Finished task 56.0 in stage 14.0 (TID 147) in 4129 ms on 10.0.0.133 (executor driver) (57/208) +26/04/01 06:32:37 INFO Executor: Running task 60.0 in stage 14.0 (TID 151) +26/04/01 06:32:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:38 INFO Executor: Finished task 58.0 in stage 14.0 (TID 149). 6563 bytes result sent to driver +26/04/01 06:32:38 INFO Executor: Finished task 57.0 in stage 14.0 (TID 148). 6563 bytes result sent to driver +26/04/01 06:32:38 INFO TaskSetManager: Starting task 61.0 in stage 14.0 (TID 152) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:38 INFO Executor: Running task 61.0 in stage 14.0 (TID 152) +26/04/01 06:32:38 INFO TaskSetManager: Starting task 62.0 in stage 14.0 (TID 153) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:38 INFO TaskSetManager: Finished task 58.0 in stage 14.0 (TID 149) in 4135 ms on 10.0.0.133 (executor driver) (58/208) +26/04/01 06:32:38 INFO Executor: Running task 62.0 in stage 14.0 (TID 153) +26/04/01 06:32:38 INFO TaskSetManager: Finished task 57.0 in stage 14.0 (TID 148) in 4137 ms on 10.0.0.133 (executor driver) (59/208) +26/04/01 06:32:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:38 INFO Executor: Finished task 59.0 in stage 14.0 (TID 150). 6563 bytes result sent to driver +26/04/01 06:32:38 INFO TaskSetManager: Starting task 63.0 in stage 14.0 (TID 154) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:38 INFO TaskSetManager: Finished task 59.0 in stage 14.0 (TID 150) in 4134 ms on 10.0.0.133 (executor driver) (60/208) +26/04/01 06:32:38 INFO Executor: Running task 63.0 in stage 14.0 (TID 154) +26/04/01 06:32:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:42 INFO Executor: Finished task 60.0 in stage 14.0 (TID 151). 6563 bytes result sent to driver +26/04/01 06:32:42 INFO TaskSetManager: Starting task 64.0 in stage 14.0 (TID 155) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:42 INFO TaskSetManager: Finished task 60.0 in stage 14.0 (TID 151) in 4139 ms on 10.0.0.133 (executor driver) (61/208) +26/04/01 06:32:42 INFO Executor: Running task 64.0 in stage 14.0 (TID 155) +26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:42 INFO Executor: Finished task 62.0 in stage 14.0 (TID 153). 6563 bytes result sent to driver +26/04/01 06:32:42 INFO TaskSetManager: Starting task 65.0 in stage 14.0 (TID 156) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:42 INFO TaskSetManager: Finished task 62.0 in stage 14.0 (TID 153) in 4141 ms on 10.0.0.133 (executor driver) (62/208) +26/04/01 06:32:42 INFO Executor: Running task 65.0 in stage 14.0 (TID 156) +26/04/01 06:32:42 INFO Executor: Finished task 61.0 in stage 14.0 (TID 152). 6563 bytes result sent to driver +26/04/01 06:32:42 INFO TaskSetManager: Starting task 66.0 in stage 14.0 (TID 157) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:32:42 INFO TaskSetManager: Finished task 61.0 in stage 14.0 (TID 152) in 4142 ms on 10.0.0.133 (executor driver) (63/208) +26/04/01 06:32:42 INFO Executor: Running task 66.0 in stage 14.0 (TID 157) +26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:42 INFO Executor: Finished task 63.0 in stage 14.0 (TID 154). 6563 bytes result sent to driver +26/04/01 06:32:42 INFO TaskSetManager: Starting task 67.0 in stage 14.0 (TID 158) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:32:42 INFO TaskSetManager: Finished task 63.0 in stage 14.0 (TID 154) in 4143 ms on 10.0.0.133 (executor driver) (64/208) +26/04/01 06:32:42 INFO Executor: Running task 67.0 in stage 14.0 (TID 158) +26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:46 INFO Executor: Finished task 64.0 in stage 14.0 (TID 155). 6606 bytes result sent to driver +26/04/01 06:32:46 INFO TaskSetManager: Starting task 68.0 in stage 14.0 (TID 159) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:46 INFO TaskSetManager: Finished task 64.0 in stage 14.0 (TID 155) in 4137 ms on 10.0.0.133 (executor driver) (65/208) +26/04/01 06:32:46 INFO Executor: Running task 68.0 in stage 14.0 (TID 159) +26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:46 INFO Executor: Finished task 66.0 in stage 14.0 (TID 157). 6606 bytes result sent to driver +26/04/01 06:32:46 INFO TaskSetManager: Starting task 69.0 in stage 14.0 (TID 160) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:46 INFO TaskSetManager: Finished task 66.0 in stage 14.0 (TID 157) in 4075 ms on 10.0.0.133 (executor driver) (66/208) +26/04/01 06:32:46 INFO Executor: Running task 69.0 in stage 14.0 (TID 160) +26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:46 INFO Executor: Finished task 65.0 in stage 14.0 (TID 156). 6606 bytes result sent to driver +26/04/01 06:32:46 INFO TaskSetManager: Starting task 70.0 in stage 14.0 (TID 161) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:46 INFO TaskSetManager: Finished task 65.0 in stage 14.0 (TID 156) in 4142 ms on 10.0.0.133 (executor driver) (67/208) +26/04/01 06:32:46 INFO Executor: Running task 70.0 in stage 14.0 (TID 161) +26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:46 INFO Executor: Finished task 67.0 in stage 14.0 (TID 158). 6606 bytes result sent to driver +26/04/01 06:32:46 INFO TaskSetManager: Starting task 71.0 in stage 14.0 (TID 162) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:46 INFO TaskSetManager: Finished task 67.0 in stage 14.0 (TID 158) in 4086 ms on 10.0.0.133 (executor driver) (68/208) +26/04/01 06:32:46 INFO Executor: Running task 71.0 in stage 14.0 (TID 162) +26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:50 INFO Executor: Finished task 68.0 in stage 14.0 (TID 159). 6563 bytes result sent to driver +26/04/01 06:32:50 INFO TaskSetManager: Starting task 72.0 in stage 14.0 (TID 163) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:32:50 INFO TaskSetManager: Finished task 68.0 in stage 14.0 (TID 159) in 4093 ms on 10.0.0.133 (executor driver) (69/208) +26/04/01 06:32:50 INFO Executor: Running task 72.0 in stage 14.0 (TID 163) +26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:50 INFO Executor: Finished task 69.0 in stage 14.0 (TID 160). 6563 bytes result sent to driver +26/04/01 06:32:50 INFO TaskSetManager: Starting task 73.0 in stage 14.0 (TID 164) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:32:50 INFO TaskSetManager: Finished task 69.0 in stage 14.0 (TID 160) in 4078 ms on 10.0.0.133 (executor driver) (70/208) +26/04/01 06:32:50 INFO Executor: Running task 73.0 in stage 14.0 (TID 164) +26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:50 INFO Executor: Finished task 70.0 in stage 14.0 (TID 161). 6563 bytes result sent to driver +26/04/01 06:32:50 INFO TaskSetManager: Starting task 74.0 in stage 14.0 (TID 165) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:50 INFO Executor: Running task 74.0 in stage 14.0 (TID 165) +26/04/01 06:32:50 INFO TaskSetManager: Finished task 70.0 in stage 14.0 (TID 161) in 4087 ms on 10.0.0.133 (executor driver) (71/208) +26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:50 INFO Executor: Finished task 71.0 in stage 14.0 (TID 162). 6563 bytes result sent to driver +26/04/01 06:32:50 INFO TaskSetManager: Starting task 75.0 in stage 14.0 (TID 166) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:50 INFO TaskSetManager: Finished task 71.0 in stage 14.0 (TID 162) in 4080 ms on 10.0.0.133 (executor driver) (72/208) +26/04/01 06:32:50 INFO Executor: Running task 75.0 in stage 14.0 (TID 166) +26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:54 INFO Executor: Finished task 72.0 in stage 14.0 (TID 163). 6563 bytes result sent to driver +26/04/01 06:32:54 INFO TaskSetManager: Starting task 76.0 in stage 14.0 (TID 167) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:54 INFO TaskSetManager: Finished task 72.0 in stage 14.0 (TID 163) in 4111 ms on 10.0.0.133 (executor driver) (73/208) +26/04/01 06:32:54 INFO Executor: Running task 76.0 in stage 14.0 (TID 167) +26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:54 INFO Executor: Finished task 73.0 in stage 14.0 (TID 164). 6563 bytes result sent to driver +26/04/01 06:32:54 INFO TaskSetManager: Starting task 77.0 in stage 14.0 (TID 168) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:54 INFO TaskSetManager: Finished task 73.0 in stage 14.0 (TID 164) in 4123 ms on 10.0.0.133 (executor driver) (74/208) +26/04/01 06:32:54 INFO Executor: Running task 77.0 in stage 14.0 (TID 168) +26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:54 INFO Executor: Finished task 74.0 in stage 14.0 (TID 165). 6563 bytes result sent to driver +26/04/01 06:32:54 INFO TaskSetManager: Starting task 78.0 in stage 14.0 (TID 169) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:32:54 INFO TaskSetManager: Finished task 74.0 in stage 14.0 (TID 165) in 4123 ms on 10.0.0.133 (executor driver) (75/208) +26/04/01 06:32:54 INFO Executor: Running task 78.0 in stage 14.0 (TID 169) +26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:54 INFO Executor: Finished task 75.0 in stage 14.0 (TID 166). 6563 bytes result sent to driver +26/04/01 06:32:54 INFO TaskSetManager: Starting task 79.0 in stage 14.0 (TID 170) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:32:54 INFO TaskSetManager: Finished task 75.0 in stage 14.0 (TID 166) in 4121 ms on 10.0.0.133 (executor driver) (76/208) +26/04/01 06:32:54 INFO Executor: Running task 79.0 in stage 14.0 (TID 170) +26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:58 INFO Executor: Finished task 76.0 in stage 14.0 (TID 167). 6563 bytes result sent to driver +26/04/01 06:32:58 INFO TaskSetManager: Starting task 80.0 in stage 14.0 (TID 171) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:58 INFO TaskSetManager: Finished task 76.0 in stage 14.0 (TID 167) in 4134 ms on 10.0.0.133 (executor driver) (77/208) +26/04/01 06:32:58 INFO Executor: Running task 80.0 in stage 14.0 (TID 171) +26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:58 INFO Executor: Finished task 77.0 in stage 14.0 (TID 168). 6563 bytes result sent to driver +26/04/01 06:32:58 INFO TaskSetManager: Starting task 81.0 in stage 14.0 (TID 172) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:58 INFO TaskSetManager: Finished task 77.0 in stage 14.0 (TID 168) in 4120 ms on 10.0.0.133 (executor driver) (78/208) +26/04/01 06:32:58 INFO Executor: Running task 81.0 in stage 14.0 (TID 172) +26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:58 INFO Executor: Finished task 78.0 in stage 14.0 (TID 169). 6563 bytes result sent to driver +26/04/01 06:32:58 INFO TaskSetManager: Starting task 82.0 in stage 14.0 (TID 173) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:58 INFO TaskSetManager: Finished task 78.0 in stage 14.0 (TID 169) in 4125 ms on 10.0.0.133 (executor driver) (79/208) +26/04/01 06:32:58 INFO Executor: Running task 82.0 in stage 14.0 (TID 173) +26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:58 INFO Executor: Finished task 79.0 in stage 14.0 (TID 170). 6563 bytes result sent to driver +26/04/01 06:32:58 INFO TaskSetManager: Starting task 83.0 in stage 14.0 (TID 174) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:32:58 INFO TaskSetManager: Finished task 79.0 in stage 14.0 (TID 170) in 4123 ms on 10.0.0.133 (executor driver) (80/208) +26/04/01 06:32:58 INFO Executor: Running task 83.0 in stage 14.0 (TID 174) +26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:02 INFO Executor: Finished task 80.0 in stage 14.0 (TID 171). 6563 bytes result sent to driver +26/04/01 06:33:02 INFO TaskSetManager: Starting task 84.0 in stage 14.0 (TID 175) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:33:02 INFO TaskSetManager: Finished task 80.0 in stage 14.0 (TID 171) in 4134 ms on 10.0.0.133 (executor driver) (81/208) +26/04/01 06:33:02 INFO Executor: Running task 84.0 in stage 14.0 (TID 175) +26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:02 INFO Executor: Finished task 81.0 in stage 14.0 (TID 172). 6563 bytes result sent to driver +26/04/01 06:33:02 INFO TaskSetManager: Starting task 85.0 in stage 14.0 (TID 176) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:02 INFO TaskSetManager: Finished task 81.0 in stage 14.0 (TID 172) in 4123 ms on 10.0.0.133 (executor driver) (82/208) +26/04/01 06:33:02 INFO Executor: Running task 85.0 in stage 14.0 (TID 176) +26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:02 INFO Executor: Finished task 82.0 in stage 14.0 (TID 173). 6563 bytes result sent to driver +26/04/01 06:33:02 INFO TaskSetManager: Starting task 86.0 in stage 14.0 (TID 177) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:02 INFO TaskSetManager: Finished task 82.0 in stage 14.0 (TID 173) in 4131 ms on 10.0.0.133 (executor driver) (83/208) +26/04/01 06:33:02 INFO Executor: Running task 86.0 in stage 14.0 (TID 177) +26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:02 INFO Executor: Finished task 83.0 in stage 14.0 (TID 174). 6563 bytes result sent to driver +26/04/01 06:33:02 INFO TaskSetManager: Starting task 87.0 in stage 14.0 (TID 178) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:02 INFO Executor: Running task 87.0 in stage 14.0 (TID 178) +26/04/01 06:33:02 INFO TaskSetManager: Finished task 83.0 in stage 14.0 (TID 174) in 4133 ms on 10.0.0.133 (executor driver) (84/208) +26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:06 INFO Executor: Finished task 84.0 in stage 14.0 (TID 175). 6563 bytes result sent to driver +26/04/01 06:33:06 INFO TaskSetManager: Starting task 88.0 in stage 14.0 (TID 179) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:06 INFO TaskSetManager: Finished task 84.0 in stage 14.0 (TID 175) in 4133 ms on 10.0.0.133 (executor driver) (85/208) +26/04/01 06:33:06 INFO Executor: Running task 88.0 in stage 14.0 (TID 179) +26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:06 INFO Executor: Finished task 85.0 in stage 14.0 (TID 176). 6563 bytes result sent to driver +26/04/01 06:33:06 INFO TaskSetManager: Starting task 89.0 in stage 14.0 (TID 180) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:06 INFO Executor: Running task 89.0 in stage 14.0 (TID 180) +26/04/01 06:33:06 INFO TaskSetManager: Finished task 85.0 in stage 14.0 (TID 176) in 4141 ms on 10.0.0.133 (executor driver) (86/208) +26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:06 INFO Executor: Finished task 86.0 in stage 14.0 (TID 177). 6563 bytes result sent to driver +26/04/01 06:33:06 INFO TaskSetManager: Starting task 90.0 in stage 14.0 (TID 181) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:33:06 INFO TaskSetManager: Finished task 86.0 in stage 14.0 (TID 177) in 4141 ms on 10.0.0.133 (executor driver) (87/208) +26/04/01 06:33:06 INFO Executor: Running task 90.0 in stage 14.0 (TID 181) +26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:06 INFO Executor: Finished task 87.0 in stage 14.0 (TID 178). 6563 bytes result sent to driver +26/04/01 06:33:06 INFO TaskSetManager: Starting task 91.0 in stage 14.0 (TID 182) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:06 INFO TaskSetManager: Finished task 87.0 in stage 14.0 (TID 178) in 4133 ms on 10.0.0.133 (executor driver) (88/208) +26/04/01 06:33:06 INFO Executor: Running task 91.0 in stage 14.0 (TID 182) +26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:10 INFO Executor: Finished task 91.0 in stage 14.0 (TID 182). 6563 bytes result sent to driver +26/04/01 06:33:10 INFO TaskSetManager: Starting task 92.0 in stage 14.0 (TID 183) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:10 INFO TaskSetManager: Finished task 91.0 in stage 14.0 (TID 182) in 3982 ms on 10.0.0.133 (executor driver) (89/208) +26/04/01 06:33:10 INFO Executor: Running task 92.0 in stage 14.0 (TID 183) +26/04/01 06:33:10 INFO Executor: Finished task 88.0 in stage 14.0 (TID 179). 6563 bytes result sent to driver +26/04/01 06:33:10 INFO TaskSetManager: Starting task 93.0 in stage 14.0 (TID 184) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:10 INFO TaskSetManager: Finished task 88.0 in stage 14.0 (TID 179) in 4146 ms on 10.0.0.133 (executor driver) (90/208) +26/04/01 06:33:10 INFO Executor: Running task 93.0 in stage 14.0 (TID 184) +26/04/01 06:33:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:10 INFO Executor: Finished task 89.0 in stage 14.0 (TID 180). 6563 bytes result sent to driver +26/04/01 06:33:10 INFO TaskSetManager: Starting task 94.0 in stage 14.0 (TID 185) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:10 INFO TaskSetManager: Finished task 89.0 in stage 14.0 (TID 180) in 4138 ms on 10.0.0.133 (executor driver) (91/208) +26/04/01 06:33:10 INFO Executor: Running task 94.0 in stage 14.0 (TID 185) +26/04/01 06:33:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:11 INFO Executor: Finished task 90.0 in stage 14.0 (TID 181). 6563 bytes result sent to driver +26/04/01 06:33:11 INFO TaskSetManager: Starting task 95.0 in stage 14.0 (TID 186) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:11 INFO TaskSetManager: Finished task 90.0 in stage 14.0 (TID 181) in 4288 ms on 10.0.0.133 (executor driver) (92/208) +26/04/01 06:33:11 INFO Executor: Running task 95.0 in stage 14.0 (TID 186) +26/04/01 06:33:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:15 INFO Executor: Finished task 93.0 in stage 14.0 (TID 184). 6563 bytes result sent to driver +26/04/01 06:33:15 INFO Executor: Finished task 92.0 in stage 14.0 (TID 183). 6563 bytes result sent to driver +26/04/01 06:33:15 INFO TaskSetManager: Starting task 96.0 in stage 14.0 (TID 187) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:33:15 INFO TaskSetManager: Finished task 93.0 in stage 14.0 (TID 184) in 4144 ms on 10.0.0.133 (executor driver) (93/208) +26/04/01 06:33:15 INFO Executor: Running task 96.0 in stage 14.0 (TID 187) +26/04/01 06:33:15 INFO TaskSetManager: Starting task 97.0 in stage 14.0 (TID 188) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:33:15 INFO TaskSetManager: Finished task 92.0 in stage 14.0 (TID 183) in 4145 ms on 10.0.0.133 (executor driver) (94/208) +26/04/01 06:33:15 INFO Executor: Running task 97.0 in stage 14.0 (TID 188) +26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:15 INFO Executor: Finished task 94.0 in stage 14.0 (TID 185). 6563 bytes result sent to driver +26/04/01 06:33:15 INFO TaskSetManager: Starting task 98.0 in stage 14.0 (TID 189) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:15 INFO TaskSetManager: Finished task 94.0 in stage 14.0 (TID 185) in 4139 ms on 10.0.0.133 (executor driver) (95/208) +26/04/01 06:33:15 INFO Executor: Running task 98.0 in stage 14.0 (TID 189) +26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:15 INFO Executor: Finished task 95.0 in stage 14.0 (TID 186). 6563 bytes result sent to driver +26/04/01 06:33:15 INFO TaskSetManager: Starting task 99.0 in stage 14.0 (TID 190) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:15 INFO Executor: Running task 99.0 in stage 14.0 (TID 190) +26/04/01 06:33:15 INFO TaskSetManager: Finished task 95.0 in stage 14.0 (TID 186) in 4143 ms on 10.0.0.133 (executor driver) (96/208) +26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:19 INFO Executor: Finished task 97.0 in stage 14.0 (TID 188). 6563 bytes result sent to driver +26/04/01 06:33:19 INFO TaskSetManager: Starting task 100.0 in stage 14.0 (TID 191) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:19 INFO TaskSetManager: Finished task 97.0 in stage 14.0 (TID 188) in 4131 ms on 10.0.0.133 (executor driver) (97/208) +26/04/01 06:33:19 INFO Executor: Running task 100.0 in stage 14.0 (TID 191) +26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:19 INFO Executor: Finished task 96.0 in stage 14.0 (TID 187). 6563 bytes result sent to driver +26/04/01 06:33:19 INFO TaskSetManager: Starting task 101.0 in stage 14.0 (TID 192) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:19 INFO TaskSetManager: Finished task 96.0 in stage 14.0 (TID 187) in 4140 ms on 10.0.0.133 (executor driver) (98/208) +26/04/01 06:33:19 INFO Executor: Running task 101.0 in stage 14.0 (TID 192) +26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:19 INFO Executor: Finished task 98.0 in stage 14.0 (TID 189). 6563 bytes result sent to driver +26/04/01 06:33:19 INFO TaskSetManager: Starting task 102.0 in stage 14.0 (TID 193) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:33:19 INFO Executor: Running task 102.0 in stage 14.0 (TID 193) +26/04/01 06:33:19 INFO TaskSetManager: Finished task 98.0 in stage 14.0 (TID 189) in 4140 ms on 10.0.0.133 (executor driver) (99/208) +26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:19 INFO Executor: Finished task 99.0 in stage 14.0 (TID 190). 6563 bytes result sent to driver +26/04/01 06:33:19 INFO TaskSetManager: Starting task 103.0 in stage 14.0 (TID 194) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:19 INFO TaskSetManager: Finished task 99.0 in stage 14.0 (TID 190) in 4118 ms on 10.0.0.133 (executor driver) (100/208) +26/04/01 06:33:19 INFO Executor: Running task 103.0 in stage 14.0 (TID 194) +26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:23 INFO Executor: Finished task 100.0 in stage 14.0 (TID 191). 6563 bytes result sent to driver +26/04/01 06:33:23 INFO Executor: Finished task 101.0 in stage 14.0 (TID 192). 6563 bytes result sent to driver +26/04/01 06:33:23 INFO TaskSetManager: Starting task 104.0 in stage 14.0 (TID 195) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:23 INFO TaskSetManager: Finished task 100.0 in stage 14.0 (TID 191) in 4147 ms on 10.0.0.133 (executor driver) (101/208) +26/04/01 06:33:23 INFO Executor: Running task 104.0 in stage 14.0 (TID 195) +26/04/01 06:33:23 INFO TaskSetManager: Starting task 105.0 in stage 14.0 (TID 196) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:23 INFO TaskSetManager: Finished task 101.0 in stage 14.0 (TID 192) in 4137 ms on 10.0.0.133 (executor driver) (102/208) +26/04/01 06:33:23 INFO Executor: Running task 105.0 in stage 14.0 (TID 196) +26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:23 INFO Executor: Finished task 102.0 in stage 14.0 (TID 193). 6563 bytes result sent to driver +26/04/01 06:33:23 INFO TaskSetManager: Starting task 106.0 in stage 14.0 (TID 197) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:23 INFO TaskSetManager: Finished task 102.0 in stage 14.0 (TID 193) in 4137 ms on 10.0.0.133 (executor driver) (103/208) +26/04/01 06:33:23 INFO Executor: Running task 106.0 in stage 14.0 (TID 197) +26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:23 INFO Executor: Finished task 103.0 in stage 14.0 (TID 194). 6563 bytes result sent to driver +26/04/01 06:33:23 INFO TaskSetManager: Starting task 107.0 in stage 14.0 (TID 198) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:23 INFO TaskSetManager: Finished task 103.0 in stage 14.0 (TID 194) in 4145 ms on 10.0.0.133 (executor driver) (104/208) +26/04/01 06:33:23 INFO Executor: Running task 107.0 in stage 14.0 (TID 198) +26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:27 INFO Executor: Finished task 105.0 in stage 14.0 (TID 196). 6606 bytes result sent to driver +26/04/01 06:33:27 INFO TaskSetManager: Starting task 108.0 in stage 14.0 (TID 199) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:33:27 INFO TaskSetManager: Finished task 105.0 in stage 14.0 (TID 196) in 4139 ms on 10.0.0.133 (executor driver) (105/208) +26/04/01 06:33:27 INFO Executor: Running task 108.0 in stage 14.0 (TID 199) +26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:27 INFO Executor: Finished task 104.0 in stage 14.0 (TID 195). 6606 bytes result sent to driver +26/04/01 06:33:27 INFO TaskSetManager: Starting task 109.0 in stage 14.0 (TID 200) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:33:27 INFO Executor: Running task 109.0 in stage 14.0 (TID 200) +26/04/01 06:33:27 INFO TaskSetManager: Finished task 104.0 in stage 14.0 (TID 195) in 4146 ms on 10.0.0.133 (executor driver) (106/208) +26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:27 INFO Executor: Finished task 106.0 in stage 14.0 (TID 197). 6606 bytes result sent to driver +26/04/01 06:33:27 INFO TaskSetManager: Starting task 110.0 in stage 14.0 (TID 201) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:27 INFO TaskSetManager: Finished task 106.0 in stage 14.0 (TID 197) in 4135 ms on 10.0.0.133 (executor driver) (107/208) +26/04/01 06:33:27 INFO Executor: Running task 110.0 in stage 14.0 (TID 201) +26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:27 INFO Executor: Finished task 107.0 in stage 14.0 (TID 198). 6606 bytes result sent to driver +26/04/01 06:33:27 INFO TaskSetManager: Starting task 111.0 in stage 14.0 (TID 202) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:27 INFO Executor: Running task 111.0 in stage 14.0 (TID 202) +26/04/01 06:33:27 INFO TaskSetManager: Finished task 107.0 in stage 14.0 (TID 198) in 4138 ms on 10.0.0.133 (executor driver) (108/208) +26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:31 INFO Executor: Finished task 109.0 in stage 14.0 (TID 200). 6563 bytes result sent to driver +26/04/01 06:33:31 INFO TaskSetManager: Starting task 112.0 in stage 14.0 (TID 203) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:31 INFO Executor: Running task 112.0 in stage 14.0 (TID 203) +26/04/01 06:33:31 INFO TaskSetManager: Finished task 109.0 in stage 14.0 (TID 200) in 4132 ms on 10.0.0.133 (executor driver) (109/208) +26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:31 INFO Executor: Finished task 108.0 in stage 14.0 (TID 199). 6563 bytes result sent to driver +26/04/01 06:33:31 INFO TaskSetManager: Starting task 113.0 in stage 14.0 (TID 204) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:31 INFO Executor: Running task 113.0 in stage 14.0 (TID 204) +26/04/01 06:33:31 INFO TaskSetManager: Finished task 108.0 in stage 14.0 (TID 199) in 4144 ms on 10.0.0.133 (executor driver) (110/208) +26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:31 INFO Executor: Finished task 110.0 in stage 14.0 (TID 201). 6563 bytes result sent to driver +26/04/01 06:33:31 INFO TaskSetManager: Starting task 114.0 in stage 14.0 (TID 205) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:33:31 INFO TaskSetManager: Finished task 110.0 in stage 14.0 (TID 201) in 4141 ms on 10.0.0.133 (executor driver) (111/208) +26/04/01 06:33:31 INFO Executor: Running task 114.0 in stage 14.0 (TID 205) +26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:31 INFO Executor: Finished task 111.0 in stage 14.0 (TID 202). 6563 bytes result sent to driver +26/04/01 06:33:31 INFO TaskSetManager: Starting task 115.0 in stage 14.0 (TID 206) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:31 INFO TaskSetManager: Finished task 111.0 in stage 14.0 (TID 202) in 4139 ms on 10.0.0.133 (executor driver) (112/208) +26/04/01 06:33:31 INFO Executor: Running task 115.0 in stage 14.0 (TID 206) +26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:35 INFO Executor: Finished task 112.0 in stage 14.0 (TID 203). 6563 bytes result sent to driver +26/04/01 06:33:35 INFO TaskSetManager: Starting task 116.0 in stage 14.0 (TID 207) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:35 INFO TaskSetManager: Finished task 112.0 in stage 14.0 (TID 203) in 4142 ms on 10.0.0.133 (executor driver) (113/208) +26/04/01 06:33:35 INFO Executor: Running task 116.0 in stage 14.0 (TID 207) +26/04/01 06:33:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:35 INFO Executor: Finished task 113.0 in stage 14.0 (TID 204). 6563 bytes result sent to driver +26/04/01 06:33:35 INFO TaskSetManager: Starting task 117.0 in stage 14.0 (TID 208) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:35 INFO Executor: Running task 117.0 in stage 14.0 (TID 208) +26/04/01 06:33:35 INFO TaskSetManager: Finished task 113.0 in stage 14.0 (TID 204) in 4136 ms on 10.0.0.133 (executor driver) (114/208) +26/04/01 06:33:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:35 INFO Executor: Finished task 114.0 in stage 14.0 (TID 205). 6563 bytes result sent to driver +26/04/01 06:33:35 INFO TaskSetManager: Starting task 118.0 in stage 14.0 (TID 209) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:35 INFO TaskSetManager: Finished task 114.0 in stage 14.0 (TID 205) in 4129 ms on 10.0.0.133 (executor driver) (115/208) +26/04/01 06:33:35 INFO Executor: Running task 118.0 in stage 14.0 (TID 209) +26/04/01 06:33:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:36 INFO Executor: Finished task 115.0 in stage 14.0 (TID 206). 6563 bytes result sent to driver +26/04/01 06:33:36 INFO TaskSetManager: Starting task 119.0 in stage 14.0 (TID 210) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:36 INFO TaskSetManager: Finished task 115.0 in stage 14.0 (TID 206) in 4130 ms on 10.0.0.133 (executor driver) (116/208) +26/04/01 06:33:36 INFO Executor: Running task 119.0 in stage 14.0 (TID 210) +26/04/01 06:33:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:39 INFO Executor: Finished task 117.0 in stage 14.0 (TID 208). 6563 bytes result sent to driver +26/04/01 06:33:39 INFO Executor: Finished task 116.0 in stage 14.0 (TID 207). 6563 bytes result sent to driver +26/04/01 06:33:39 INFO TaskSetManager: Starting task 120.0 in stage 14.0 (TID 211) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:33:39 INFO TaskSetManager: Starting task 121.0 in stage 14.0 (TID 212) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:39 INFO Executor: Running task 120.0 in stage 14.0 (TID 211) +26/04/01 06:33:39 INFO TaskSetManager: Finished task 117.0 in stage 14.0 (TID 208) in 4130 ms on 10.0.0.133 (executor driver) (117/208) +26/04/01 06:33:39 INFO Executor: Running task 121.0 in stage 14.0 (TID 212) +26/04/01 06:33:39 INFO TaskSetManager: Finished task 116.0 in stage 14.0 (TID 207) in 4132 ms on 10.0.0.133 (executor driver) (118/208) +26/04/01 06:33:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:39 INFO Executor: Finished task 118.0 in stage 14.0 (TID 209). 6563 bytes result sent to driver +26/04/01 06:33:39 INFO TaskSetManager: Starting task 122.0 in stage 14.0 (TID 213) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:39 INFO TaskSetManager: Finished task 118.0 in stage 14.0 (TID 209) in 4129 ms on 10.0.0.133 (executor driver) (119/208) +26/04/01 06:33:39 INFO Executor: Running task 122.0 in stage 14.0 (TID 213) +26/04/01 06:33:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:40 INFO Executor: Finished task 119.0 in stage 14.0 (TID 210). 6563 bytes result sent to driver +26/04/01 06:33:40 INFO TaskSetManager: Starting task 123.0 in stage 14.0 (TID 214) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:40 INFO TaskSetManager: Finished task 119.0 in stage 14.0 (TID 210) in 4132 ms on 10.0.0.133 (executor driver) (120/208) +26/04/01 06:33:40 INFO Executor: Running task 123.0 in stage 14.0 (TID 214) +26/04/01 06:33:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:44 INFO Executor: Finished task 120.0 in stage 14.0 (TID 211). 6563 bytes result sent to driver +26/04/01 06:33:44 INFO TaskSetManager: Starting task 124.0 in stage 14.0 (TID 215) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:44 INFO TaskSetManager: Finished task 120.0 in stage 14.0 (TID 211) in 4115 ms on 10.0.0.133 (executor driver) (121/208) +26/04/01 06:33:44 INFO Executor: Running task 124.0 in stage 14.0 (TID 215) +26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:44 INFO Executor: Finished task 121.0 in stage 14.0 (TID 212). 6563 bytes result sent to driver +26/04/01 06:33:44 INFO TaskSetManager: Starting task 125.0 in stage 14.0 (TID 216) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:44 INFO TaskSetManager: Finished task 121.0 in stage 14.0 (TID 212) in 4132 ms on 10.0.0.133 (executor driver) (122/208) +26/04/01 06:33:44 INFO Executor: Running task 125.0 in stage 14.0 (TID 216) +26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:44 INFO Executor: Finished task 122.0 in stage 14.0 (TID 213). 6563 bytes result sent to driver +26/04/01 06:33:44 INFO TaskSetManager: Starting task 126.0 in stage 14.0 (TID 217) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:33:44 INFO TaskSetManager: Finished task 122.0 in stage 14.0 (TID 213) in 4132 ms on 10.0.0.133 (executor driver) (123/208) +26/04/01 06:33:44 INFO Executor: Running task 126.0 in stage 14.0 (TID 217) +26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:44 INFO Executor: Finished task 123.0 in stage 14.0 (TID 214). 6563 bytes result sent to driver +26/04/01 06:33:44 INFO TaskSetManager: Starting task 127.0 in stage 14.0 (TID 218) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:44 INFO TaskSetManager: Finished task 123.0 in stage 14.0 (TID 214) in 4129 ms on 10.0.0.133 (executor driver) (124/208) +26/04/01 06:33:44 INFO Executor: Running task 127.0 in stage 14.0 (TID 218) +26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:48 INFO Executor: Finished task 124.0 in stage 14.0 (TID 215). 6563 bytes result sent to driver +26/04/01 06:33:48 INFO TaskSetManager: Starting task 128.0 in stage 14.0 (TID 219) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:48 INFO TaskSetManager: Finished task 124.0 in stage 14.0 (TID 215) in 4118 ms on 10.0.0.133 (executor driver) (125/208) +26/04/01 06:33:48 INFO Executor: Running task 128.0 in stage 14.0 (TID 219) +26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:48 INFO Executor: Finished task 125.0 in stage 14.0 (TID 216). 6563 bytes result sent to driver +26/04/01 06:33:48 INFO TaskSetManager: Starting task 129.0 in stage 14.0 (TID 220) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:48 INFO TaskSetManager: Finished task 125.0 in stage 14.0 (TID 216) in 4128 ms on 10.0.0.133 (executor driver) (126/208) +26/04/01 06:33:48 INFO Executor: Running task 129.0 in stage 14.0 (TID 220) +26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:48 INFO Executor: Finished task 126.0 in stage 14.0 (TID 217). 6563 bytes result sent to driver +26/04/01 06:33:48 INFO TaskSetManager: Starting task 130.0 in stage 14.0 (TID 221) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:48 INFO TaskSetManager: Finished task 126.0 in stage 14.0 (TID 217) in 4120 ms on 10.0.0.133 (executor driver) (127/208) +26/04/01 06:33:48 INFO Executor: Running task 130.0 in stage 14.0 (TID 221) +26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:48 INFO Executor: Finished task 127.0 in stage 14.0 (TID 218). 6563 bytes result sent to driver +26/04/01 06:33:48 INFO TaskSetManager: Starting task 131.0 in stage 14.0 (TID 222) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:48 INFO TaskSetManager: Finished task 127.0 in stage 14.0 (TID 218) in 4117 ms on 10.0.0.133 (executor driver) (128/208) +26/04/01 06:33:48 INFO Executor: Running task 131.0 in stage 14.0 (TID 222) +26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:52 INFO Executor: Finished task 128.0 in stage 14.0 (TID 219). 6563 bytes result sent to driver +26/04/01 06:33:52 INFO TaskSetManager: Starting task 132.0 in stage 14.0 (TID 223) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:33:52 INFO TaskSetManager: Finished task 128.0 in stage 14.0 (TID 219) in 4131 ms on 10.0.0.133 (executor driver) (129/208) +26/04/01 06:33:52 INFO Executor: Running task 132.0 in stage 14.0 (TID 223) +26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:52 INFO Executor: Finished task 129.0 in stage 14.0 (TID 220). 6563 bytes result sent to driver +26/04/01 06:33:52 INFO TaskSetManager: Starting task 133.0 in stage 14.0 (TID 224) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:52 INFO TaskSetManager: Finished task 129.0 in stage 14.0 (TID 220) in 4136 ms on 10.0.0.133 (executor driver) (130/208) +26/04/01 06:33:52 INFO Executor: Running task 133.0 in stage 14.0 (TID 224) +26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:52 INFO Executor: Finished task 130.0 in stage 14.0 (TID 221). 6563 bytes result sent to driver +26/04/01 06:33:52 INFO TaskSetManager: Starting task 134.0 in stage 14.0 (TID 225) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:52 INFO TaskSetManager: Finished task 130.0 in stage 14.0 (TID 221) in 4133 ms on 10.0.0.133 (executor driver) (131/208) +26/04/01 06:33:52 INFO Executor: Running task 134.0 in stage 14.0 (TID 225) +26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:52 INFO Executor: Finished task 131.0 in stage 14.0 (TID 222). 6563 bytes result sent to driver +26/04/01 06:33:52 INFO TaskSetManager: Starting task 135.0 in stage 14.0 (TID 226) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:52 INFO TaskSetManager: Finished task 131.0 in stage 14.0 (TID 222) in 4140 ms on 10.0.0.133 (executor driver) (132/208) +26/04/01 06:33:52 INFO Executor: Running task 135.0 in stage 14.0 (TID 226) +26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:56 INFO Executor: Finished task 132.0 in stage 14.0 (TID 223). 6563 bytes result sent to driver +26/04/01 06:33:56 INFO TaskSetManager: Starting task 136.0 in stage 14.0 (TID 227) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:56 INFO TaskSetManager: Finished task 132.0 in stage 14.0 (TID 223) in 4083 ms on 10.0.0.133 (executor driver) (133/208) +26/04/01 06:33:56 INFO Executor: Running task 136.0 in stage 14.0 (TID 227) +26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:56 INFO Executor: Finished task 133.0 in stage 14.0 (TID 224). 6563 bytes result sent to driver +26/04/01 06:33:56 INFO TaskSetManager: Starting task 137.0 in stage 14.0 (TID 228) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:33:56 INFO TaskSetManager: Finished task 133.0 in stage 14.0 (TID 224) in 4076 ms on 10.0.0.133 (executor driver) (134/208) +26/04/01 06:33:56 INFO Executor: Running task 137.0 in stage 14.0 (TID 228) +26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:56 INFO Executor: Finished task 134.0 in stage 14.0 (TID 225). 6563 bytes result sent to driver +26/04/01 06:33:56 INFO TaskSetManager: Starting task 138.0 in stage 14.0 (TID 229) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:33:56 INFO TaskSetManager: Finished task 134.0 in stage 14.0 (TID 225) in 4084 ms on 10.0.0.133 (executor driver) (135/208) +26/04/01 06:33:56 INFO Executor: Running task 138.0 in stage 14.0 (TID 229) +26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:56 INFO Executor: Finished task 135.0 in stage 14.0 (TID 226). 6563 bytes result sent to driver +26/04/01 06:33:56 INFO TaskSetManager: Starting task 139.0 in stage 14.0 (TID 230) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:33:56 INFO Executor: Running task 139.0 in stage 14.0 (TID 230) +26/04/01 06:33:56 INFO TaskSetManager: Finished task 135.0 in stage 14.0 (TID 226) in 4079 ms on 10.0.0.133 (executor driver) (136/208) +26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:00 INFO Executor: Finished task 136.0 in stage 14.0 (TID 227). 6563 bytes result sent to driver +26/04/01 06:34:00 INFO TaskSetManager: Starting task 140.0 in stage 14.0 (TID 231) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:00 INFO TaskSetManager: Finished task 136.0 in stage 14.0 (TID 227) in 4091 ms on 10.0.0.133 (executor driver) (137/208) +26/04/01 06:34:00 INFO Executor: Running task 140.0 in stage 14.0 (TID 231) +26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:00 INFO Executor: Finished task 137.0 in stage 14.0 (TID 228). 6563 bytes result sent to driver +26/04/01 06:34:00 INFO TaskSetManager: Starting task 141.0 in stage 14.0 (TID 232) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:00 INFO TaskSetManager: Finished task 137.0 in stage 14.0 (TID 228) in 4080 ms on 10.0.0.133 (executor driver) (138/208) +26/04/01 06:34:00 INFO Executor: Running task 141.0 in stage 14.0 (TID 232) +26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:00 INFO Executor: Finished task 138.0 in stage 14.0 (TID 229). 6563 bytes result sent to driver +26/04/01 06:34:00 INFO TaskSetManager: Starting task 142.0 in stage 14.0 (TID 233) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:00 INFO TaskSetManager: Finished task 138.0 in stage 14.0 (TID 229) in 4083 ms on 10.0.0.133 (executor driver) (139/208) +26/04/01 06:34:00 INFO Executor: Running task 142.0 in stage 14.0 (TID 233) +26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:00 INFO Executor: Finished task 139.0 in stage 14.0 (TID 230). 6563 bytes result sent to driver +26/04/01 06:34:00 INFO TaskSetManager: Starting task 143.0 in stage 14.0 (TID 234) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:00 INFO TaskSetManager: Finished task 139.0 in stage 14.0 (TID 230) in 4081 ms on 10.0.0.133 (executor driver) (140/208) +26/04/01 06:34:00 INFO Executor: Running task 143.0 in stage 14.0 (TID 234) +26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:04 INFO Executor: Finished task 140.0 in stage 14.0 (TID 231). 6563 bytes result sent to driver +26/04/01 06:34:04 INFO TaskSetManager: Starting task 144.0 in stage 14.0 (TID 235) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:34:04 INFO TaskSetManager: Finished task 140.0 in stage 14.0 (TID 231) in 4079 ms on 10.0.0.133 (executor driver) (141/208) +26/04/01 06:34:04 INFO Executor: Running task 144.0 in stage 14.0 (TID 235) +26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:04 INFO Executor: Finished task 141.0 in stage 14.0 (TID 232). 6563 bytes result sent to driver +26/04/01 06:34:04 INFO TaskSetManager: Starting task 145.0 in stage 14.0 (TID 236) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:34:04 INFO Executor: Running task 145.0 in stage 14.0 (TID 236) +26/04/01 06:34:04 INFO TaskSetManager: Finished task 141.0 in stage 14.0 (TID 232) in 4078 ms on 10.0.0.133 (executor driver) (142/208) +26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:04 INFO Executor: Finished task 142.0 in stage 14.0 (TID 233). 6563 bytes result sent to driver +26/04/01 06:34:04 INFO TaskSetManager: Starting task 146.0 in stage 14.0 (TID 237) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:04 INFO Executor: Running task 146.0 in stage 14.0 (TID 237) +26/04/01 06:34:04 INFO TaskSetManager: Finished task 142.0 in stage 14.0 (TID 233) in 4077 ms on 10.0.0.133 (executor driver) (143/208) +26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:04 INFO Executor: Finished task 143.0 in stage 14.0 (TID 234). 6563 bytes result sent to driver +26/04/01 06:34:04 INFO TaskSetManager: Starting task 147.0 in stage 14.0 (TID 238) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:04 INFO TaskSetManager: Finished task 143.0 in stage 14.0 (TID 234) in 4081 ms on 10.0.0.133 (executor driver) (144/208) +26/04/01 06:34:04 INFO Executor: Running task 147.0 in stage 14.0 (TID 238) +26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:08 INFO Executor: Finished task 144.0 in stage 14.0 (TID 235). 6606 bytes result sent to driver +26/04/01 06:34:08 INFO TaskSetManager: Starting task 148.0 in stage 14.0 (TID 239) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:08 INFO Executor: Running task 148.0 in stage 14.0 (TID 239) +26/04/01 06:34:08 INFO TaskSetManager: Finished task 144.0 in stage 14.0 (TID 235) in 4139 ms on 10.0.0.133 (executor driver) (145/208) +26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:08 INFO Executor: Finished task 145.0 in stage 14.0 (TID 236). 6606 bytes result sent to driver +26/04/01 06:34:08 INFO TaskSetManager: Starting task 149.0 in stage 14.0 (TID 240) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:08 INFO Executor: Running task 149.0 in stage 14.0 (TID 240) +26/04/01 06:34:08 INFO TaskSetManager: Finished task 145.0 in stage 14.0 (TID 236) in 4129 ms on 10.0.0.133 (executor driver) (146/208) +26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:08 INFO Executor: Finished task 146.0 in stage 14.0 (TID 237). 6606 bytes result sent to driver +26/04/01 06:34:08 INFO TaskSetManager: Starting task 150.0 in stage 14.0 (TID 241) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:34:08 INFO TaskSetManager: Finished task 146.0 in stage 14.0 (TID 237) in 4120 ms on 10.0.0.133 (executor driver) (147/208) +26/04/01 06:34:08 INFO Executor: Running task 150.0 in stage 14.0 (TID 241) +26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:08 INFO Executor: Finished task 147.0 in stage 14.0 (TID 238). 6606 bytes result sent to driver +26/04/01 06:34:08 INFO TaskSetManager: Starting task 151.0 in stage 14.0 (TID 242) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:08 INFO Executor: Running task 151.0 in stage 14.0 (TID 242) +26/04/01 06:34:08 INFO TaskSetManager: Finished task 147.0 in stage 14.0 (TID 238) in 4133 ms on 10.0.0.133 (executor driver) (148/208) +26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:12 INFO Executor: Finished task 148.0 in stage 14.0 (TID 239). 6563 bytes result sent to driver +26/04/01 06:34:12 INFO Executor: Finished task 149.0 in stage 14.0 (TID 240). 6563 bytes result sent to driver +26/04/01 06:34:12 INFO TaskSetManager: Starting task 152.0 in stage 14.0 (TID 243) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:12 INFO TaskSetManager: Starting task 153.0 in stage 14.0 (TID 244) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:12 INFO Executor: Running task 152.0 in stage 14.0 (TID 243) +26/04/01 06:34:12 INFO TaskSetManager: Finished task 148.0 in stage 14.0 (TID 239) in 4158 ms on 10.0.0.133 (executor driver) (149/208) +26/04/01 06:34:12 INFO Executor: Running task 153.0 in stage 14.0 (TID 244) +26/04/01 06:34:12 INFO TaskSetManager: Finished task 149.0 in stage 14.0 (TID 240) in 4157 ms on 10.0.0.133 (executor driver) (150/208) +26/04/01 06:34:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:12 INFO Executor: Finished task 150.0 in stage 14.0 (TID 241). 6563 bytes result sent to driver +26/04/01 06:34:12 INFO TaskSetManager: Starting task 154.0 in stage 14.0 (TID 245) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:12 INFO TaskSetManager: Finished task 150.0 in stage 14.0 (TID 241) in 4151 ms on 10.0.0.133 (executor driver) (151/208) +26/04/01 06:34:12 INFO Executor: Running task 154.0 in stage 14.0 (TID 245) +26/04/01 06:34:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:13 INFO Executor: Finished task 151.0 in stage 14.0 (TID 242). 6563 bytes result sent to driver +26/04/01 06:34:13 INFO TaskSetManager: Starting task 155.0 in stage 14.0 (TID 246) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:13 INFO TaskSetManager: Finished task 151.0 in stage 14.0 (TID 242) in 4146 ms on 10.0.0.133 (executor driver) (152/208) +26/04/01 06:34:13 INFO Executor: Running task 155.0 in stage 14.0 (TID 246) +26/04/01 06:34:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:16 INFO Executor: Finished task 152.0 in stage 14.0 (TID 243). 6563 bytes result sent to driver +26/04/01 06:34:16 INFO TaskSetManager: Starting task 156.0 in stage 14.0 (TID 247) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:34:16 INFO TaskSetManager: Finished task 152.0 in stage 14.0 (TID 243) in 4134 ms on 10.0.0.133 (executor driver) (153/208) +26/04/01 06:34:16 INFO Executor: Running task 156.0 in stage 14.0 (TID 247) +26/04/01 06:34:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:16 INFO Executor: Finished task 153.0 in stage 14.0 (TID 244). 6563 bytes result sent to driver +26/04/01 06:34:16 INFO TaskSetManager: Starting task 157.0 in stage 14.0 (TID 248) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:16 INFO TaskSetManager: Finished task 153.0 in stage 14.0 (TID 244) in 4150 ms on 10.0.0.133 (executor driver) (154/208) +26/04/01 06:34:16 INFO Executor: Running task 157.0 in stage 14.0 (TID 248) +26/04/01 06:34:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:16 INFO Executor: Finished task 154.0 in stage 14.0 (TID 245). 6563 bytes result sent to driver +26/04/01 06:34:16 INFO TaskSetManager: Starting task 158.0 in stage 14.0 (TID 249) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:16 INFO TaskSetManager: Finished task 154.0 in stage 14.0 (TID 245) in 4141 ms on 10.0.0.133 (executor driver) (155/208) +26/04/01 06:34:16 INFO Executor: Running task 158.0 in stage 14.0 (TID 249) +26/04/01 06:34:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:17 INFO Executor: Finished task 155.0 in stage 14.0 (TID 246). 6563 bytes result sent to driver +26/04/01 06:34:17 INFO TaskSetManager: Starting task 159.0 in stage 14.0 (TID 250) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:17 INFO TaskSetManager: Finished task 155.0 in stage 14.0 (TID 246) in 4134 ms on 10.0.0.133 (executor driver) (156/208) +26/04/01 06:34:17 INFO Executor: Running task 159.0 in stage 14.0 (TID 250) +26/04/01 06:34:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:21 INFO Executor: Finished task 156.0 in stage 14.0 (TID 247). 6563 bytes result sent to driver +26/04/01 06:34:21 INFO TaskSetManager: Starting task 160.0 in stage 14.0 (TID 251) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:21 INFO TaskSetManager: Finished task 156.0 in stage 14.0 (TID 247) in 4081 ms on 10.0.0.133 (executor driver) (157/208) +26/04/01 06:34:21 INFO Executor: Running task 160.0 in stage 14.0 (TID 251) +26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:21 INFO Executor: Finished task 157.0 in stage 14.0 (TID 248). 6563 bytes result sent to driver +26/04/01 06:34:21 INFO Executor: Finished task 158.0 in stage 14.0 (TID 249). 6563 bytes result sent to driver +26/04/01 06:34:21 INFO TaskSetManager: Starting task 161.0 in stage 14.0 (TID 252) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:21 INFO Executor: Running task 161.0 in stage 14.0 (TID 252) +26/04/01 06:34:21 INFO TaskSetManager: Starting task 162.0 in stage 14.0 (TID 253) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:34:21 INFO Executor: Running task 162.0 in stage 14.0 (TID 253) +26/04/01 06:34:21 INFO TaskSetManager: Finished task 157.0 in stage 14.0 (TID 248) in 4085 ms on 10.0.0.133 (executor driver) (158/208) +26/04/01 06:34:21 INFO TaskSetManager: Finished task 158.0 in stage 14.0 (TID 249) in 4080 ms on 10.0.0.133 (executor driver) (159/208) +26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:21 INFO Executor: Finished task 159.0 in stage 14.0 (TID 250). 6563 bytes result sent to driver +26/04/01 06:34:21 INFO TaskSetManager: Starting task 163.0 in stage 14.0 (TID 254) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:34:21 INFO TaskSetManager: Finished task 159.0 in stage 14.0 (TID 250) in 4082 ms on 10.0.0.133 (executor driver) (160/208) +26/04/01 06:34:21 INFO Executor: Running task 163.0 in stage 14.0 (TID 254) +26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:25 INFO Executor: Finished task 160.0 in stage 14.0 (TID 251). 6563 bytes result sent to driver +26/04/01 06:34:25 INFO TaskSetManager: Starting task 164.0 in stage 14.0 (TID 255) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:25 INFO TaskSetManager: Finished task 160.0 in stage 14.0 (TID 251) in 4089 ms on 10.0.0.133 (executor driver) (161/208) +26/04/01 06:34:25 INFO Executor: Running task 164.0 in stage 14.0 (TID 255) +26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:25 INFO Executor: Finished task 161.0 in stage 14.0 (TID 252). 6563 bytes result sent to driver +26/04/01 06:34:25 INFO TaskSetManager: Starting task 165.0 in stage 14.0 (TID 256) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:25 INFO TaskSetManager: Finished task 161.0 in stage 14.0 (TID 252) in 4078 ms on 10.0.0.133 (executor driver) (162/208) +26/04/01 06:34:25 INFO Executor: Running task 165.0 in stage 14.0 (TID 256) +26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:25 INFO Executor: Finished task 162.0 in stage 14.0 (TID 253). 6563 bytes result sent to driver +26/04/01 06:34:25 INFO TaskSetManager: Starting task 166.0 in stage 14.0 (TID 257) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:25 INFO Executor: Running task 166.0 in stage 14.0 (TID 257) +26/04/01 06:34:25 INFO TaskSetManager: Finished task 162.0 in stage 14.0 (TID 253) in 4129 ms on 10.0.0.133 (executor driver) (163/208) +26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:25 INFO Executor: Finished task 163.0 in stage 14.0 (TID 254). 6563 bytes result sent to driver +26/04/01 06:34:25 INFO TaskSetManager: Starting task 167.0 in stage 14.0 (TID 258) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:25 INFO TaskSetManager: Finished task 163.0 in stage 14.0 (TID 254) in 4147 ms on 10.0.0.133 (executor driver) (164/208) +26/04/01 06:34:25 INFO Executor: Running task 167.0 in stage 14.0 (TID 258) +26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:29 INFO Executor: Finished task 164.0 in stage 14.0 (TID 255). 6563 bytes result sent to driver +26/04/01 06:34:29 INFO TaskSetManager: Starting task 168.0 in stage 14.0 (TID 259) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:34:29 INFO Executor: Running task 168.0 in stage 14.0 (TID 259) +26/04/01 06:34:29 INFO TaskSetManager: Finished task 164.0 in stage 14.0 (TID 255) in 4135 ms on 10.0.0.133 (executor driver) (165/208) +26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:29 INFO Executor: Finished task 165.0 in stage 14.0 (TID 256). 6563 bytes result sent to driver +26/04/01 06:34:29 INFO TaskSetManager: Starting task 169.0 in stage 14.0 (TID 260) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:29 INFO TaskSetManager: Finished task 165.0 in stage 14.0 (TID 256) in 4130 ms on 10.0.0.133 (executor driver) (166/208) +26/04/01 06:34:29 INFO Executor: Running task 169.0 in stage 14.0 (TID 260) +26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:29 INFO Executor: Finished task 166.0 in stage 14.0 (TID 257). 6563 bytes result sent to driver +26/04/01 06:34:29 INFO TaskSetManager: Starting task 170.0 in stage 14.0 (TID 261) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:29 INFO TaskSetManager: Finished task 166.0 in stage 14.0 (TID 257) in 4117 ms on 10.0.0.133 (executor driver) (167/208) +26/04/01 06:34:29 INFO Executor: Running task 170.0 in stage 14.0 (TID 261) +26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:29 INFO Executor: Finished task 167.0 in stage 14.0 (TID 258). 6563 bytes result sent to driver +26/04/01 06:34:29 INFO TaskSetManager: Starting task 171.0 in stage 14.0 (TID 262) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:29 INFO Executor: Running task 171.0 in stage 14.0 (TID 262) +26/04/01 06:34:29 INFO TaskSetManager: Finished task 167.0 in stage 14.0 (TID 258) in 4130 ms on 10.0.0.133 (executor driver) (168/208) +26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:33 INFO Executor: Finished task 169.0 in stage 14.0 (TID 260). 6563 bytes result sent to driver +26/04/01 06:34:33 INFO TaskSetManager: Starting task 172.0 in stage 14.0 (TID 263) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:33 INFO Executor: Running task 172.0 in stage 14.0 (TID 263) +26/04/01 06:34:33 INFO TaskSetManager: Finished task 169.0 in stage 14.0 (TID 260) in 4086 ms on 10.0.0.133 (executor driver) (169/208) +26/04/01 06:34:33 INFO Executor: Finished task 168.0 in stage 14.0 (TID 259). 6563 bytes result sent to driver +26/04/01 06:34:33 INFO TaskSetManager: Starting task 173.0 in stage 14.0 (TID 264) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:33 INFO Executor: Running task 173.0 in stage 14.0 (TID 264) +26/04/01 06:34:33 INFO TaskSetManager: Finished task 168.0 in stage 14.0 (TID 259) in 4090 ms on 10.0.0.133 (executor driver) (170/208) +26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:33 INFO Executor: Finished task 170.0 in stage 14.0 (TID 261). 6563 bytes result sent to driver +26/04/01 06:34:33 INFO TaskSetManager: Starting task 174.0 in stage 14.0 (TID 265) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9268 bytes) +26/04/01 06:34:33 INFO TaskSetManager: Finished task 170.0 in stage 14.0 (TID 261) in 4080 ms on 10.0.0.133 (executor driver) (171/208) +26/04/01 06:34:33 INFO Executor: Running task 174.0 in stage 14.0 (TID 265) +26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:33 INFO Executor: Finished task 171.0 in stage 14.0 (TID 262). 6563 bytes result sent to driver +26/04/01 06:34:33 INFO TaskSetManager: Starting task 175.0 in stage 14.0 (TID 266) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9273 bytes) +26/04/01 06:34:33 INFO Executor: Running task 175.0 in stage 14.0 (TID 266) +26/04/01 06:34:33 INFO TaskSetManager: Finished task 171.0 in stage 14.0 (TID 262) in 4081 ms on 10.0.0.133 (executor driver) (172/208) +26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:37 INFO Executor: Finished task 172.0 in stage 14.0 (TID 263). 6563 bytes result sent to driver +26/04/01 06:34:37 INFO TaskSetManager: Starting task 176.0 in stage 14.0 (TID 267) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:37 INFO TaskSetManager: Finished task 172.0 in stage 14.0 (TID 263) in 4071 ms on 10.0.0.133 (executor driver) (173/208) +26/04/01 06:34:37 INFO Executor: Running task 176.0 in stage 14.0 (TID 267) +26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:37 INFO Executor: Finished task 173.0 in stage 14.0 (TID 264). 6563 bytes result sent to driver +26/04/01 06:34:37 INFO TaskSetManager: Starting task 177.0 in stage 14.0 (TID 268) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:37 INFO Executor: Running task 177.0 in stage 14.0 (TID 268) +26/04/01 06:34:37 INFO TaskSetManager: Finished task 173.0 in stage 14.0 (TID 264) in 4079 ms on 10.0.0.133 (executor driver) (174/208) +26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:37 INFO Executor: Finished task 174.0 in stage 14.0 (TID 265). 6563 bytes result sent to driver +26/04/01 06:34:37 INFO TaskSetManager: Starting task 178.0 in stage 14.0 (TID 269) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:37 INFO Executor: Running task 178.0 in stage 14.0 (TID 269) +26/04/01 06:34:37 INFO TaskSetManager: Finished task 174.0 in stage 14.0 (TID 265) in 4135 ms on 10.0.0.133 (executor driver) (175/208) +26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:37 INFO Executor: Finished task 175.0 in stage 14.0 (TID 266). 6563 bytes result sent to driver +26/04/01 06:34:37 INFO TaskSetManager: Starting task 179.0 in stage 14.0 (TID 270) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:37 INFO Executor: Running task 179.0 in stage 14.0 (TID 270) +26/04/01 06:34:37 INFO TaskSetManager: Finished task 175.0 in stage 14.0 (TID 266) in 4134 ms on 10.0.0.133 (executor driver) (176/208) +26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:41 INFO Executor: Finished task 176.0 in stage 14.0 (TID 267). 6563 bytes result sent to driver +26/04/01 06:34:41 INFO Executor: Finished task 177.0 in stage 14.0 (TID 268). 6563 bytes result sent to driver +26/04/01 06:34:41 INFO TaskSetManager: Starting task 180.0 in stage 14.0 (TID 271) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:34:41 INFO Executor: Running task 180.0 in stage 14.0 (TID 271) +26/04/01 06:34:41 INFO TaskSetManager: Starting task 181.0 in stage 14.0 (TID 272) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:41 INFO TaskSetManager: Finished task 176.0 in stage 14.0 (TID 267) in 4146 ms on 10.0.0.133 (executor driver) (177/208) +26/04/01 06:34:41 INFO Executor: Running task 181.0 in stage 14.0 (TID 272) +26/04/01 06:34:41 INFO TaskSetManager: Finished task 177.0 in stage 14.0 (TID 268) in 4137 ms on 10.0.0.133 (executor driver) (178/208) +26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:41 INFO Executor: Finished task 178.0 in stage 14.0 (TID 269). 6563 bytes result sent to driver +26/04/01 06:34:41 INFO TaskSetManager: Starting task 182.0 in stage 14.0 (TID 273) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:41 INFO TaskSetManager: Finished task 178.0 in stage 14.0 (TID 269) in 4151 ms on 10.0.0.133 (executor driver) (179/208) +26/04/01 06:34:41 INFO Executor: Running task 182.0 in stage 14.0 (TID 273) +26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:41 INFO Executor: Finished task 179.0 in stage 14.0 (TID 270). 6563 bytes result sent to driver +26/04/01 06:34:41 INFO TaskSetManager: Starting task 183.0 in stage 14.0 (TID 274) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:41 INFO TaskSetManager: Finished task 179.0 in stage 14.0 (TID 270) in 4136 ms on 10.0.0.133 (executor driver) (180/208) +26/04/01 06:34:41 INFO Executor: Running task 183.0 in stage 14.0 (TID 274) +26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:45 INFO Executor: Finished task 180.0 in stage 14.0 (TID 271). 6563 bytes result sent to driver +26/04/01 06:34:45 INFO TaskSetManager: Starting task 184.0 in stage 14.0 (TID 275) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:45 INFO TaskSetManager: Finished task 180.0 in stage 14.0 (TID 271) in 4130 ms on 10.0.0.133 (executor driver) (181/208) +26/04/01 06:34:45 INFO Executor: Running task 184.0 in stage 14.0 (TID 275) +26/04/01 06:34:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:45 INFO Executor: Finished task 181.0 in stage 14.0 (TID 272). 6563 bytes result sent to driver +26/04/01 06:34:45 INFO TaskSetManager: Starting task 185.0 in stage 14.0 (TID 276) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:45 INFO TaskSetManager: Finished task 181.0 in stage 14.0 (TID 272) in 4140 ms on 10.0.0.133 (executor driver) (182/208) +26/04/01 06:34:45 INFO Executor: Running task 185.0 in stage 14.0 (TID 276) +26/04/01 06:34:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:45 INFO Executor: Finished task 182.0 in stage 14.0 (TID 273). 6563 bytes result sent to driver +26/04/01 06:34:45 INFO TaskSetManager: Starting task 186.0 in stage 14.0 (TID 277) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:34:45 INFO TaskSetManager: Finished task 182.0 in stage 14.0 (TID 273) in 4145 ms on 10.0.0.133 (executor driver) (183/208) +26/04/01 06:34:45 INFO Executor: Running task 186.0 in stage 14.0 (TID 277) +26/04/01 06:34:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:46 INFO Executor: Finished task 183.0 in stage 14.0 (TID 274). 6563 bytes result sent to driver +26/04/01 06:34:46 INFO TaskSetManager: Starting task 187.0 in stage 14.0 (TID 278) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9274 bytes) +26/04/01 06:34:46 INFO TaskSetManager: Finished task 183.0 in stage 14.0 (TID 274) in 4128 ms on 10.0.0.133 (executor driver) (184/208) +26/04/01 06:34:46 INFO Executor: Running task 187.0 in stage 14.0 (TID 278) +26/04/01 06:34:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:49 INFO Executor: Finished task 184.0 in stage 14.0 (TID 275). 6606 bytes result sent to driver +26/04/01 06:34:49 INFO TaskSetManager: Starting task 188.0 in stage 14.0 (TID 279) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:49 INFO TaskSetManager: Finished task 184.0 in stage 14.0 (TID 275) in 4139 ms on 10.0.0.133 (executor driver) (185/208) +26/04/01 06:34:49 INFO Executor: Finished task 185.0 in stage 14.0 (TID 276). 6606 bytes result sent to driver +26/04/01 06:34:49 INFO Executor: Running task 188.0 in stage 14.0 (TID 279) +26/04/01 06:34:49 INFO TaskSetManager: Starting task 189.0 in stage 14.0 (TID 280) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:49 INFO TaskSetManager: Finished task 185.0 in stage 14.0 (TID 276) in 4128 ms on 10.0.0.133 (executor driver) (186/208) +26/04/01 06:34:49 INFO Executor: Running task 189.0 in stage 14.0 (TID 280) +26/04/01 06:34:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:49 INFO Executor: Finished task 186.0 in stage 14.0 (TID 277). 6606 bytes result sent to driver +26/04/01 06:34:49 INFO TaskSetManager: Starting task 190.0 in stage 14.0 (TID 281) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:49 INFO Executor: Running task 190.0 in stage 14.0 (TID 281) +26/04/01 06:34:49 INFO TaskSetManager: Finished task 186.0 in stage 14.0 (TID 277) in 4120 ms on 10.0.0.133 (executor driver) (187/208) +26/04/01 06:34:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:50 INFO Executor: Finished task 187.0 in stage 14.0 (TID 278). 6606 bytes result sent to driver +26/04/01 06:34:50 INFO TaskSetManager: Starting task 191.0 in stage 14.0 (TID 282) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9275 bytes) +26/04/01 06:34:50 INFO TaskSetManager: Finished task 187.0 in stage 14.0 (TID 278) in 4148 ms on 10.0.0.133 (executor driver) (188/208) +26/04/01 06:34:50 INFO Executor: Running task 191.0 in stage 14.0 (TID 282) +26/04/01 06:34:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:53 INFO Executor: Finished task 189.0 in stage 14.0 (TID 280). 6563 bytes result sent to driver +26/04/01 06:34:53 INFO TaskSetManager: Starting task 192.0 in stage 14.0 (TID 283) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:34:53 INFO TaskSetManager: Finished task 189.0 in stage 14.0 (TID 280) in 4124 ms on 10.0.0.133 (executor driver) (189/208) +26/04/01 06:34:53 INFO Executor: Running task 192.0 in stage 14.0 (TID 283) +26/04/01 06:34:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:53 INFO Executor: Finished task 188.0 in stage 14.0 (TID 279). 6563 bytes result sent to driver +26/04/01 06:34:53 INFO TaskSetManager: Starting task 193.0 in stage 14.0 (TID 284) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:34:53 INFO Executor: Running task 193.0 in stage 14.0 (TID 284) +26/04/01 06:34:53 INFO TaskSetManager: Finished task 188.0 in stage 14.0 (TID 279) in 4128 ms on 10.0.0.133 (executor driver) (190/208) +26/04/01 06:34:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:54 INFO Executor: Finished task 190.0 in stage 14.0 (TID 281). 6563 bytes result sent to driver +26/04/01 06:34:54 INFO TaskSetManager: Starting task 194.0 in stage 14.0 (TID 285) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:34:54 INFO TaskSetManager: Finished task 190.0 in stage 14.0 (TID 281) in 4131 ms on 10.0.0.133 (executor driver) (191/208) +26/04/01 06:34:54 INFO Executor: Running task 194.0 in stage 14.0 (TID 285) +26/04/01 06:34:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:54 INFO Executor: Finished task 191.0 in stage 14.0 (TID 282). 6563 bytes result sent to driver +26/04/01 06:34:54 INFO TaskSetManager: Starting task 195.0 in stage 14.0 (TID 286) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:34:54 INFO Executor: Running task 195.0 in stage 14.0 (TID 286) +26/04/01 06:34:54 INFO TaskSetManager: Finished task 191.0 in stage 14.0 (TID 282) in 4129 ms on 10.0.0.133 (executor driver) (192/208) +26/04/01 06:34:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:56 INFO Executor: Finished task 193.0 in stage 14.0 (TID 284). 6563 bytes result sent to driver +26/04/01 06:34:56 INFO TaskSetManager: Starting task 196.0 in stage 14.0 (TID 287) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:34:56 INFO TaskSetManager: Finished task 193.0 in stage 14.0 (TID 284) in 2583 ms on 10.0.0.133 (executor driver) (193/208) +26/04/01 06:34:56 INFO Executor: Running task 196.0 in stage 14.0 (TID 287) +26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:56 INFO Executor: Finished task 192.0 in stage 14.0 (TID 283). 6563 bytes result sent to driver +26/04/01 06:34:56 INFO TaskSetManager: Starting task 197.0 in stage 14.0 (TID 288) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:34:56 INFO Executor: Running task 197.0 in stage 14.0 (TID 288) +26/04/01 06:34:56 INFO TaskSetManager: Finished task 192.0 in stage 14.0 (TID 283) in 2597 ms on 10.0.0.133 (executor driver) (194/208) +26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:56 INFO Executor: Finished task 194.0 in stage 14.0 (TID 285). 6563 bytes result sent to driver +26/04/01 06:34:56 INFO TaskSetManager: Starting task 198.0 in stage 14.0 (TID 289) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:34:56 INFO TaskSetManager: Finished task 194.0 in stage 14.0 (TID 285) in 2581 ms on 10.0.0.133 (executor driver) (195/208) +26/04/01 06:34:56 INFO Executor: Running task 198.0 in stage 14.0 (TID 289) +26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:56 INFO Executor: Finished task 195.0 in stage 14.0 (TID 286). 6563 bytes result sent to driver +26/04/01 06:34:56 INFO TaskSetManager: Starting task 199.0 in stage 14.0 (TID 290) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:34:56 INFO TaskSetManager: Finished task 195.0 in stage 14.0 (TID 286) in 2590 ms on 10.0.0.133 (executor driver) (196/208) +26/04/01 06:34:56 INFO Executor: Running task 199.0 in stage 14.0 (TID 290) +26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:59 INFO Executor: Finished task 196.0 in stage 14.0 (TID 287). 6563 bytes result sent to driver +26/04/01 06:34:59 INFO TaskSetManager: Starting task 200.0 in stage 14.0 (TID 291) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9346 bytes) +26/04/01 06:34:59 INFO TaskSetManager: Finished task 196.0 in stage 14.0 (TID 287) in 2570 ms on 10.0.0.133 (executor driver) (197/208) +26/04/01 06:34:59 INFO Executor: Running task 200.0 in stage 14.0 (TID 291) +26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:59 INFO Executor: Finished task 197.0 in stage 14.0 (TID 288). 6563 bytes result sent to driver +26/04/01 06:34:59 INFO TaskSetManager: Starting task 201.0 in stage 14.0 (TID 292) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:34:59 INFO Executor: Running task 201.0 in stage 14.0 (TID 292) +26/04/01 06:34:59 INFO TaskSetManager: Finished task 197.0 in stage 14.0 (TID 288) in 2567 ms on 10.0.0.133 (executor driver) (198/208) +26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:59 INFO Executor: Finished task 198.0 in stage 14.0 (TID 289). 6563 bytes result sent to driver +26/04/01 06:34:59 INFO TaskSetManager: Starting task 202.0 in stage 14.0 (TID 293) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:34:59 INFO TaskSetManager: Finished task 198.0 in stage 14.0 (TID 289) in 2577 ms on 10.0.0.133 (executor driver) (199/208) +26/04/01 06:34:59 INFO Executor: Running task 202.0 in stage 14.0 (TID 293) +26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:59 INFO Executor: Finished task 199.0 in stage 14.0 (TID 290). 6563 bytes result sent to driver +26/04/01 06:34:59 INFO TaskSetManager: Starting task 203.0 in stage 14.0 (TID 294) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:34:59 INFO TaskSetManager: Finished task 199.0 in stage 14.0 (TID 290) in 2567 ms on 10.0.0.133 (executor driver) (200/208) +26/04/01 06:34:59 INFO Executor: Running task 203.0 in stage 14.0 (TID 294) +26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:01 INFO Executor: Finished task 200.0 in stage 14.0 (TID 291). 6563 bytes result sent to driver +26/04/01 06:35:01 INFO TaskSetManager: Starting task 204.0 in stage 14.0 (TID 295) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:35:01 INFO TaskSetManager: Finished task 200.0 in stage 14.0 (TID 291) in 2572 ms on 10.0.0.133 (executor driver) (201/208) +26/04/01 06:35:01 INFO Executor: Running task 204.0 in stage 14.0 (TID 295) +26/04/01 06:35:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:01 INFO Executor: Finished task 201.0 in stage 14.0 (TID 292). 6563 bytes result sent to driver +26/04/01 06:35:01 INFO TaskSetManager: Starting task 205.0 in stage 14.0 (TID 296) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9348 bytes) +26/04/01 06:35:01 INFO TaskSetManager: Finished task 201.0 in stage 14.0 (TID 292) in 2578 ms on 10.0.0.133 (executor driver) (202/208) +26/04/01 06:35:01 INFO Executor: Running task 205.0 in stage 14.0 (TID 296) +26/04/01 06:35:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:01 INFO Executor: Finished task 202.0 in stage 14.0 (TID 293). 6563 bytes result sent to driver +26/04/01 06:35:01 INFO TaskSetManager: Starting task 206.0 in stage 14.0 (TID 297) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:35:01 INFO TaskSetManager: Finished task 202.0 in stage 14.0 (TID 293) in 2579 ms on 10.0.0.133 (executor driver) (203/208) +26/04/01 06:35:01 INFO Executor: Running task 206.0 in stage 14.0 (TID 297) +26/04/01 06:35:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:02 INFO Executor: Finished task 203.0 in stage 14.0 (TID 294). 6563 bytes result sent to driver +26/04/01 06:35:02 INFO TaskSetManager: Starting task 207.0 in stage 14.0 (TID 298) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9347 bytes) +26/04/01 06:35:02 INFO TaskSetManager: Finished task 203.0 in stage 14.0 (TID 294) in 2571 ms on 10.0.0.133 (executor driver) (204/208) +26/04/01 06:35:02 INFO Executor: Running task 207.0 in stage 14.0 (TID 298) +26/04/01 06:35:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO Executor: Finished task 204.0 in stage 14.0 (TID 295). 6563 bytes result sent to driver +26/04/01 06:35:04 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 299) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) +26/04/01 06:35:04 INFO TaskSetManager: Finished task 204.0 in stage 14.0 (TID 295) in 2552 ms on 10.0.0.133 (executor driver) (205/208) +26/04/01 06:35:04 INFO Executor: Running task 0.0 in stage 15.0 (TID 299) +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO Executor: Finished task 0.0 in stage 15.0 (TID 299). 11159 bytes result sent to driver +26/04/01 06:35:04 INFO TaskSetManager: Starting task 0.0 in stage 16.0 (TID 300) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9722 bytes) +26/04/01 06:35:04 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 299) in 15 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:35:04 INFO Executor: Running task 0.0 in stage 16.0 (TID 300) +26/04/01 06:35:04 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool +26/04/01 06:35:04 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 228.291 s +26/04/01 06:35:04 INFO DAGScheduler: Job 14 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:35:04 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished +26/04/01 06:35:04 INFO DAGScheduler: Job 14 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 228.290733 s +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO Utils: Coalesced 1 broadcast batches into 1 (5 rows) +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO MemoryStore: Block broadcast_25 stored as values in memory (estimated size 608.0 B, free 12.3 GiB) +26/04/01 06:35:04 INFO MemoryStore: Block broadcast_25_piece0 stored as bytes in memory (estimated size 563.0 B, free 12.3 GiB) +26/04/01 06:35:04 INFO BlockManagerInfo: Added broadcast_25_piece0 in memory on 10.0.0.133:58442 (size: 563.0 B, free: 12.6 GiB) +26/04/01 06:35:04 INFO SparkContext: Created broadcast 25 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:35:04 INFO Executor: Finished task 205.0 in stage 14.0 (TID 296). 6563 bytes result sent to driver +26/04/01 06:35:04 INFO TaskSetManager: Starting task 1.0 in stage 16.0 (TID 301) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9724 bytes) +26/04/01 06:35:04 INFO Executor: Running task 1.0 in stage 16.0 (TID 301) +26/04/01 06:35:04 INFO TaskSetManager: Finished task 205.0 in stage 14.0 (TID 296) in 2558 ms on 10.0.0.133 (executor driver) (206/208) +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO Executor: Finished task 206.0 in stage 14.0 (TID 297). 6563 bytes result sent to driver +26/04/01 06:35:04 INFO TaskSetManager: Starting task 2.0 in stage 16.0 (TID 302) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9724 bytes) +26/04/01 06:35:04 INFO Executor: Running task 2.0 in stage 16.0 (TID 302) +26/04/01 06:35:04 INFO TaskSetManager: Finished task 206.0 in stage 14.0 (TID 297) in 2552 ms on 10.0.0.133 (executor driver) (207/208) +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO Executor: Finished task 0.0 in stage 16.0 (TID 300). 6563 bytes result sent to driver +26/04/01 06:35:04 INFO TaskSetManager: Starting task 3.0 in stage 16.0 (TID 303) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9721 bytes) +26/04/01 06:35:04 INFO Executor: Running task 3.0 in stage 16.0 (TID 303) +26/04/01 06:35:04 INFO TaskSetManager: Finished task 0.0 in stage 16.0 (TID 300) in 207 ms on 10.0.0.133 (executor driver) (1/4) +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO Executor: Finished task 1.0 in stage 16.0 (TID 301). 6563 bytes result sent to driver +26/04/01 06:35:04 INFO TaskSetManager: Starting task 0.0 in stage 19.0 (TID 304) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:04 INFO Executor: Running task 0.0 in stage 19.0 (TID 304) +26/04/01 06:35:04 INFO TaskSetManager: Finished task 1.0 in stage 16.0 (TID 301) in 211 ms on 10.0.0.133 (executor driver) (2/4) +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO Executor: Finished task 2.0 in stage 16.0 (TID 302). 6563 bytes result sent to driver +26/04/01 06:35:04 INFO TaskSetManager: Starting task 1.0 in stage 19.0 (TID 305) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:04 INFO TaskSetManager: Finished task 2.0 in stage 16.0 (TID 302) in 210 ms on 10.0.0.133 (executor driver) (3/4) +26/04/01 06:35:04 INFO Executor: Running task 1.0 in stage 19.0 (TID 305) +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO Executor: Finished task 207.0 in stage 14.0 (TID 298). 6563 bytes result sent to driver +26/04/01 06:35:04 INFO TaskSetManager: Starting task 2.0 in stage 19.0 (TID 306) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:04 INFO TaskSetManager: Finished task 207.0 in stage 14.0 (TID 298) in 2553 ms on 10.0.0.133 (executor driver) (208/208) +26/04/01 06:35:04 INFO Executor: Running task 2.0 in stage 19.0 (TID 306) +26/04/01 06:35:04 INFO TaskSchedulerImpl: Removed TaskSet 14.0, whose tasks have all completed, from pool +26/04/01 06:35:04 INFO DAGScheduler: ShuffleMapStage 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 228.648 s +26/04/01 06:35:04 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:35:04 INFO DAGScheduler: running: Set(ShuffleMapStage 19, ShuffleMapStage 16) +26/04/01 06:35:04 INFO DAGScheduler: waiting: Set() +26/04/01 06:35:04 INFO DAGScheduler: failed: Set() +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO Executor: Finished task 3.0 in stage 16.0 (TID 303). 6563 bytes result sent to driver +26/04/01 06:35:04 INFO TaskSetManager: Starting task 3.0 in stage 19.0 (TID 307) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:04 INFO TaskSetManager: Finished task 3.0 in stage 16.0 (TID 303) in 208 ms on 10.0.0.133 (executor driver) (4/4) +26/04/01 06:35:04 INFO Executor: Running task 3.0 in stage 19.0 (TID 307) +26/04/01 06:35:04 INFO TaskSchedulerImpl: Removed TaskSet 16.0, whose tasks have all completed, from pool +26/04/01 06:35:04 INFO DAGScheduler: ShuffleMapStage 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 228.699 s +26/04/01 06:35:04 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:35:04 INFO DAGScheduler: running: Set(ShuffleMapStage 19) +26/04/01 06:35:04 INFO DAGScheduler: waiting: Set() +26/04/01 06:35:04 INFO DAGScheduler: failed: Set() +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:04 INFO BlockManagerInfo: Removed broadcast_20_piece0 on 10.0.0.133:58442 in memory (size: 8.0 KiB, free: 12.6 GiB) +26/04/01 06:35:04 INFO BlockManagerInfo: Removed broadcast_22_piece0 on 10.0.0.133:58442 in memory (size: 7.8 KiB, free: 12.6 GiB) +26/04/01 06:35:04 INFO BlockManagerInfo: Removed broadcast_21_piece0 on 10.0.0.133:58442 in memory (size: 30.7 KiB, free: 12.6 GiB) +26/04/01 06:35:15 INFO Executor: Finished task 0.0 in stage 19.0 (TID 304). 9594 bytes result sent to driver +26/04/01 06:35:15 INFO TaskSetManager: Starting task 4.0 in stage 19.0 (TID 308) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:15 INFO Executor: Running task 4.0 in stage 19.0 (TID 308) +26/04/01 06:35:15 INFO TaskSetManager: Finished task 0.0 in stage 19.0 (TID 304) in 11227 ms on 10.0.0.133 (executor driver) (1/6) +26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:16 INFO Executor: Finished task 1.0 in stage 19.0 (TID 305). 9594 bytes result sent to driver +26/04/01 06:35:16 INFO TaskSetManager: Starting task 5.0 in stage 19.0 (TID 309) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:16 INFO TaskSetManager: Finished task 1.0 in stage 19.0 (TID 305) in 11541 ms on 10.0.0.133 (executor driver) (2/6) +26/04/01 06:35:16 INFO Executor: Running task 5.0 in stage 19.0 (TID 309) +26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Getting 16 (11.1 MiB) non-empty blocks including 16 (11.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Getting 64 (23.7 MiB) non-empty blocks including 64 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Getting 16 (11.1 MiB) non-empty blocks including 16 (11.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Getting 64 (23.7 MiB) non-empty blocks including 64 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:16 INFO Executor: Finished task 3.0 in stage 19.0 (TID 307). 9594 bytes result sent to driver +26/04/01 06:35:16 INFO TaskSetManager: Finished task 3.0 in stage 19.0 (TID 307) in 11966 ms on 10.0.0.133 (executor driver) (3/6) +26/04/01 06:35:16 INFO Executor: Finished task 2.0 in stage 19.0 (TID 306). 9594 bytes result sent to driver +26/04/01 06:35:16 INFO TaskSetManager: Finished task 2.0 in stage 19.0 (TID 306) in 12038 ms on 10.0.0.133 (executor driver) (4/6) +26/04/01 06:35:21 INFO Executor: Finished task 5.0 in stage 19.0 (TID 309). 8863 bytes result sent to driver +26/04/01 06:35:21 INFO TaskSetManager: Finished task 5.0 in stage 19.0 (TID 309) in 5567 ms on 10.0.0.133 (executor driver) (5/6) +26/04/01 06:35:25 INFO Executor: Finished task 4.0 in stage 19.0 (TID 308). 9594 bytes result sent to driver +26/04/01 06:35:25 INFO TaskSetManager: Finished task 4.0 in stage 19.0 (TID 308) in 10294 ms on 10.0.0.133 (executor driver) (6/6) +26/04/01 06:35:25 INFO TaskSchedulerImpl: Removed TaskSet 19.0, whose tasks have all completed, from pool +26/04/01 06:35:25 INFO DAGScheduler: ShuffleMapStage 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 229.659 s +26/04/01 06:35:25 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:35:25 INFO DAGScheduler: running: Set() +26/04/01 06:35:25 INFO DAGScheduler: waiting: Set() +26/04/01 06:35:25 INFO DAGScheduler: failed: Set() +26/04/01 06:35:25 INFO ShufflePartitionsUtil: For shuffle(5, 3), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 06:35:25 INFO DAGScheduler: Registering RDD 48 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 6 +26/04/01 06:35:25 INFO DAGScheduler: Got map stage job 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions +26/04/01 06:35:25 INFO DAGScheduler: Final stage: ShuffleMapStage 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:35:25 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 22, ShuffleMapStage 23) +26/04/01 06:35:25 INFO DAGScheduler: Missing parents: List() +26/04/01 06:35:25 INFO DAGScheduler: Submitting ShuffleMapStage 24 (MapPartitionsRDD[48] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:35:26 INFO MemoryStore: Block broadcast_26 stored as values in memory (estimated size 29.4 KiB, free 12.6 GiB) +26/04/01 06:35:26 INFO MemoryStore: Block broadcast_26_piece0 stored as bytes in memory (estimated size 11.7 KiB, free 12.6 GiB) +26/04/01 06:35:26 INFO BlockManagerInfo: Added broadcast_26_piece0 in memory on 10.0.0.133:58442 (size: 11.7 KiB, free: 12.6 GiB) +26/04/01 06:35:26 INFO SparkContext: Created broadcast 26 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:35:26 INFO DAGScheduler: Submitting 200 missing tasks from ShuffleMapStage 24 (MapPartitionsRDD[48] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 06:35:26 INFO TaskSchedulerImpl: Adding task set 24.0 with 200 tasks resource profile 0 +26/04/01 06:35:26 INFO TaskSetManager: Starting task 0.0 in stage 24.0 (TID 310) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:26 INFO TaskSetManager: Starting task 1.0 in stage 24.0 (TID 311) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:26 INFO TaskSetManager: Starting task 2.0 in stage 24.0 (TID 312) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:26 INFO TaskSetManager: Starting task 3.0 in stage 24.0 (TID 313) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:26 INFO Executor: Running task 1.0 in stage 24.0 (TID 311) +26/04/01 06:35:26 INFO Executor: Running task 0.0 in stage 24.0 (TID 310) +26/04/01 06:35:26 INFO Executor: Running task 3.0 in stage 24.0 (TID 313) +26/04/01 06:35:26 INFO Executor: Running task 2.0 in stage 24.0 (TID 312) +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:29 INFO Executor: Finished task 1.0 in stage 24.0 (TID 311). 8470 bytes result sent to driver +26/04/01 06:35:29 INFO TaskSetManager: Starting task 4.0 in stage 24.0 (TID 314) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:29 INFO TaskSetManager: Finished task 1.0 in stage 24.0 (TID 311) in 3517 ms on 10.0.0.133 (executor driver) (1/200) +26/04/01 06:35:29 INFO Executor: Running task 4.0 in stage 24.0 (TID 314) +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:29 INFO Executor: Finished task 2.0 in stage 24.0 (TID 312). 8470 bytes result sent to driver +26/04/01 06:35:29 INFO TaskSetManager: Starting task 5.0 in stage 24.0 (TID 315) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:29 INFO TaskSetManager: Finished task 2.0 in stage 24.0 (TID 312) in 3576 ms on 10.0.0.133 (executor driver) (2/200) +26/04/01 06:35:29 INFO Executor: Running task 5.0 in stage 24.0 (TID 315) +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO Executor: Finished task 0.0 in stage 24.0 (TID 310). 8470 bytes result sent to driver +26/04/01 06:35:29 INFO TaskSetManager: Starting task 6.0 in stage 24.0 (TID 316) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:29 INFO TaskSetManager: Finished task 0.0 in stage 24.0 (TID 310) in 3582 ms on 10.0.0.133 (executor driver) (3/200) +26/04/01 06:35:29 INFO Executor: Running task 6.0 in stage 24.0 (TID 316) +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:29 INFO Executor: Finished task 3.0 in stage 24.0 (TID 313). 8470 bytes result sent to driver +26/04/01 06:35:29 INFO TaskSetManager: Starting task 7.0 in stage 24.0 (TID 317) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:29 INFO TaskSetManager: Finished task 3.0 in stage 24.0 (TID 313) in 3591 ms on 10.0.0.133 (executor driver) (4/200) +26/04/01 06:35:29 INFO Executor: Running task 7.0 in stage 24.0 (TID 317) +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:32 INFO Executor: Finished task 4.0 in stage 24.0 (TID 314). 8470 bytes result sent to driver +26/04/01 06:35:32 INFO TaskSetManager: Starting task 8.0 in stage 24.0 (TID 318) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:32 INFO TaskSetManager: Finished task 4.0 in stage 24.0 (TID 314) in 3395 ms on 10.0.0.133 (executor driver) (5/200) +26/04/01 06:35:32 INFO Executor: Running task 8.0 in stage 24.0 (TID 318) +26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:33 INFO Executor: Finished task 5.0 in stage 24.0 (TID 315). 8470 bytes result sent to driver +26/04/01 06:35:33 INFO TaskSetManager: Starting task 9.0 in stage 24.0 (TID 319) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:33 INFO TaskSetManager: Finished task 5.0 in stage 24.0 (TID 315) in 3449 ms on 10.0.0.133 (executor driver) (6/200) +26/04/01 06:35:33 INFO Executor: Running task 9.0 in stage 24.0 (TID 319) +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:33 INFO Executor: Finished task 6.0 in stage 24.0 (TID 316). 8470 bytes result sent to driver +26/04/01 06:35:33 INFO TaskSetManager: Starting task 10.0 in stage 24.0 (TID 320) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:33 INFO Executor: Running task 10.0 in stage 24.0 (TID 320) +26/04/01 06:35:33 INFO TaskSetManager: Finished task 6.0 in stage 24.0 (TID 316) in 3491 ms on 10.0.0.133 (executor driver) (7/200) +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:33 INFO Executor: Finished task 7.0 in stage 24.0 (TID 317). 8470 bytes result sent to driver +26/04/01 06:35:33 INFO TaskSetManager: Starting task 11.0 in stage 24.0 (TID 321) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:33 INFO Executor: Running task 11.0 in stage 24.0 (TID 321) +26/04/01 06:35:33 INFO TaskSetManager: Finished task 7.0 in stage 24.0 (TID 317) in 3486 ms on 10.0.0.133 (executor driver) (8/200) +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:36 INFO Executor: Finished task 8.0 in stage 24.0 (TID 318). 9201 bytes result sent to driver +26/04/01 06:35:36 INFO TaskSetManager: Starting task 12.0 in stage 24.0 (TID 322) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:36 INFO TaskSetManager: Finished task 8.0 in stage 24.0 (TID 318) in 3404 ms on 10.0.0.133 (executor driver) (9/200) +26/04/01 06:35:36 INFO Executor: Running task 12.0 in stage 24.0 (TID 322) +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:36 INFO Executor: Finished task 9.0 in stage 24.0 (TID 319). 9201 bytes result sent to driver +26/04/01 06:35:36 INFO TaskSetManager: Starting task 13.0 in stage 24.0 (TID 323) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:36 INFO Executor: Running task 13.0 in stage 24.0 (TID 323) +26/04/01 06:35:36 INFO TaskSetManager: Finished task 9.0 in stage 24.0 (TID 319) in 3400 ms on 10.0.0.133 (executor driver) (10/200) +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:36 INFO Executor: Finished task 10.0 in stage 24.0 (TID 320). 9201 bytes result sent to driver +26/04/01 06:35:36 INFO TaskSetManager: Starting task 14.0 in stage 24.0 (TID 324) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:36 INFO TaskSetManager: Finished task 10.0 in stage 24.0 (TID 320) in 3449 ms on 10.0.0.133 (executor driver) (11/200) +26/04/01 06:35:36 INFO Executor: Running task 14.0 in stage 24.0 (TID 324) +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1027.0 KiB) non-empty blocks including 6 (1027.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO Executor: Finished task 11.0 in stage 24.0 (TID 321). 9201 bytes result sent to driver +26/04/01 06:35:36 INFO TaskSetManager: Starting task 15.0 in stage 24.0 (TID 325) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:36 INFO Executor: Running task 15.0 in stage 24.0 (TID 325) +26/04/01 06:35:36 INFO TaskSetManager: Finished task 11.0 in stage 24.0 (TID 321) in 3450 ms on 10.0.0.133 (executor driver) (12/200) +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1027.0 KiB) non-empty blocks including 6 (1027.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:39 INFO Executor: Finished task 12.0 in stage 24.0 (TID 322). 8470 bytes result sent to driver +26/04/01 06:35:39 INFO TaskSetManager: Starting task 16.0 in stage 24.0 (TID 326) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:39 INFO TaskSetManager: Finished task 12.0 in stage 24.0 (TID 322) in 3387 ms on 10.0.0.133 (executor driver) (13/200) +26/04/01 06:35:39 INFO Executor: Running task 16.0 in stage 24.0 (TID 326) +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:39 INFO Executor: Finished task 13.0 in stage 24.0 (TID 323). 8470 bytes result sent to driver +26/04/01 06:35:39 INFO TaskSetManager: Starting task 17.0 in stage 24.0 (TID 327) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:39 INFO Executor: Running task 17.0 in stage 24.0 (TID 327) +26/04/01 06:35:39 INFO TaskSetManager: Finished task 13.0 in stage 24.0 (TID 323) in 3387 ms on 10.0.0.133 (executor driver) (14/200) +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:39 INFO Executor: Finished task 14.0 in stage 24.0 (TID 324). 8470 bytes result sent to driver +26/04/01 06:35:39 INFO TaskSetManager: Starting task 18.0 in stage 24.0 (TID 328) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:39 INFO TaskSetManager: Finished task 14.0 in stage 24.0 (TID 324) in 3441 ms on 10.0.0.133 (executor driver) (15/200) +26/04/01 06:35:39 INFO Executor: Running task 18.0 in stage 24.0 (TID 328) +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO Executor: Finished task 15.0 in stage 24.0 (TID 325). 8470 bytes result sent to driver +26/04/01 06:35:39 INFO TaskSetManager: Starting task 19.0 in stage 24.0 (TID 329) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:39 INFO TaskSetManager: Finished task 15.0 in stage 24.0 (TID 325) in 3441 ms on 10.0.0.133 (executor driver) (16/200) +26/04/01 06:35:39 INFO Executor: Running task 19.0 in stage 24.0 (TID 329) +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:43 INFO Executor: Finished task 16.0 in stage 24.0 (TID 326). 8470 bytes result sent to driver +26/04/01 06:35:43 INFO TaskSetManager: Starting task 20.0 in stage 24.0 (TID 330) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:43 INFO Executor: Running task 20.0 in stage 24.0 (TID 330) +26/04/01 06:35:43 INFO TaskSetManager: Finished task 16.0 in stage 24.0 (TID 326) in 3376 ms on 10.0.0.133 (executor driver) (17/200) +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:43 INFO Executor: Finished task 17.0 in stage 24.0 (TID 327). 8470 bytes result sent to driver +26/04/01 06:35:43 INFO TaskSetManager: Starting task 21.0 in stage 24.0 (TID 331) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:43 INFO TaskSetManager: Finished task 17.0 in stage 24.0 (TID 327) in 3375 ms on 10.0.0.133 (executor driver) (18/200) +26/04/01 06:35:43 INFO Executor: Running task 21.0 in stage 24.0 (TID 331) +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:43 INFO Executor: Finished task 19.0 in stage 24.0 (TID 329). 8470 bytes result sent to driver +26/04/01 06:35:43 INFO TaskSetManager: Starting task 22.0 in stage 24.0 (TID 332) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:43 INFO TaskSetManager: Finished task 19.0 in stage 24.0 (TID 329) in 3440 ms on 10.0.0.133 (executor driver) (19/200) +26/04/01 06:35:43 INFO Executor: Running task 22.0 in stage 24.0 (TID 332) +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO Executor: Finished task 18.0 in stage 24.0 (TID 328). 8470 bytes result sent to driver +26/04/01 06:35:43 INFO TaskSetManager: Starting task 23.0 in stage 24.0 (TID 333) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:43 INFO TaskSetManager: Finished task 18.0 in stage 24.0 (TID 328) in 3454 ms on 10.0.0.133 (executor driver) (20/200) +26/04/01 06:35:43 INFO Executor: Running task 23.0 in stage 24.0 (TID 333) +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:46 INFO Executor: Finished task 20.0 in stage 24.0 (TID 330). 9201 bytes result sent to driver +26/04/01 06:35:46 INFO TaskSetManager: Starting task 24.0 in stage 24.0 (TID 334) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:46 INFO TaskSetManager: Finished task 20.0 in stage 24.0 (TID 330) in 3384 ms on 10.0.0.133 (executor driver) (21/200) +26/04/01 06:35:46 INFO Executor: Running task 24.0 in stage 24.0 (TID 334) +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:46 INFO Executor: Finished task 21.0 in stage 24.0 (TID 331). 9201 bytes result sent to driver +26/04/01 06:35:46 INFO TaskSetManager: Starting task 25.0 in stage 24.0 (TID 335) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:46 INFO TaskSetManager: Finished task 21.0 in stage 24.0 (TID 331) in 3389 ms on 10.0.0.133 (executor driver) (22/200) +26/04/01 06:35:46 INFO Executor: Running task 25.0 in stage 24.0 (TID 335) +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:46 INFO Executor: Finished task 22.0 in stage 24.0 (TID 332). 9201 bytes result sent to driver +26/04/01 06:35:46 INFO TaskSetManager: Starting task 26.0 in stage 24.0 (TID 336) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:46 INFO TaskSetManager: Finished task 22.0 in stage 24.0 (TID 332) in 3437 ms on 10.0.0.133 (executor driver) (23/200) +26/04/01 06:35:46 INFO Executor: Running task 26.0 in stage 24.0 (TID 336) +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO Executor: Finished task 23.0 in stage 24.0 (TID 333). 9201 bytes result sent to driver +26/04/01 06:35:46 INFO TaskSetManager: Starting task 27.0 in stage 24.0 (TID 337) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:46 INFO TaskSetManager: Finished task 23.0 in stage 24.0 (TID 333) in 3433 ms on 10.0.0.133 (executor driver) (24/200) +26/04/01 06:35:46 INFO Executor: Running task 27.0 in stage 24.0 (TID 337) +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:49 INFO Executor: Finished task 24.0 in stage 24.0 (TID 334). 8470 bytes result sent to driver +26/04/01 06:35:49 INFO TaskSetManager: Starting task 28.0 in stage 24.0 (TID 338) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:49 INFO TaskSetManager: Finished task 24.0 in stage 24.0 (TID 334) in 3393 ms on 10.0.0.133 (executor driver) (25/200) +26/04/01 06:35:49 INFO Executor: Running task 28.0 in stage 24.0 (TID 338) +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:49 INFO Executor: Finished task 25.0 in stage 24.0 (TID 335). 8470 bytes result sent to driver +26/04/01 06:35:49 INFO TaskSetManager: Starting task 29.0 in stage 24.0 (TID 339) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:49 INFO Executor: Running task 29.0 in stage 24.0 (TID 339) +26/04/01 06:35:49 INFO TaskSetManager: Finished task 25.0 in stage 24.0 (TID 335) in 3394 ms on 10.0.0.133 (executor driver) (26/200) +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:50 INFO Executor: Finished task 26.0 in stage 24.0 (TID 336). 8470 bytes result sent to driver +26/04/01 06:35:50 INFO TaskSetManager: Starting task 30.0 in stage 24.0 (TID 340) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:50 INFO Executor: Running task 30.0 in stage 24.0 (TID 340) +26/04/01 06:35:50 INFO TaskSetManager: Finished task 26.0 in stage 24.0 (TID 336) in 3474 ms on 10.0.0.133 (executor driver) (27/200) +26/04/01 06:35:50 INFO Executor: Finished task 27.0 in stage 24.0 (TID 337). 8470 bytes result sent to driver +26/04/01 06:35:50 INFO TaskSetManager: Starting task 31.0 in stage 24.0 (TID 341) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:50 INFO TaskSetManager: Finished task 27.0 in stage 24.0 (TID 337) in 3470 ms on 10.0.0.133 (executor driver) (28/200) +26/04/01 06:35:50 INFO Executor: Running task 31.0 in stage 24.0 (TID 341) +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:53 INFO Executor: Finished task 28.0 in stage 24.0 (TID 338). 8470 bytes result sent to driver +26/04/01 06:35:53 INFO TaskSetManager: Starting task 32.0 in stage 24.0 (TID 342) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:53 INFO TaskSetManager: Finished task 28.0 in stage 24.0 (TID 338) in 3377 ms on 10.0.0.133 (executor driver) (29/200) +26/04/01 06:35:53 INFO Executor: Running task 32.0 in stage 24.0 (TID 342) +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:53 INFO Executor: Finished task 29.0 in stage 24.0 (TID 339). 8470 bytes result sent to driver +26/04/01 06:35:53 INFO TaskSetManager: Starting task 33.0 in stage 24.0 (TID 343) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:53 INFO TaskSetManager: Finished task 29.0 in stage 24.0 (TID 339) in 3377 ms on 10.0.0.133 (executor driver) (30/200) +26/04/01 06:35:53 INFO Executor: Running task 33.0 in stage 24.0 (TID 343) +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:53 INFO Executor: Finished task 30.0 in stage 24.0 (TID 340). 8470 bytes result sent to driver +26/04/01 06:35:53 INFO TaskSetManager: Starting task 34.0 in stage 24.0 (TID 344) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:53 INFO Executor: Running task 34.0 in stage 24.0 (TID 344) +26/04/01 06:35:53 INFO TaskSetManager: Finished task 30.0 in stage 24.0 (TID 340) in 3422 ms on 10.0.0.133 (executor driver) (31/200) +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO Executor: Finished task 31.0 in stage 24.0 (TID 341). 8470 bytes result sent to driver +26/04/01 06:35:53 INFO TaskSetManager: Starting task 35.0 in stage 24.0 (TID 345) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:53 INFO TaskSetManager: Finished task 31.0 in stage 24.0 (TID 341) in 3429 ms on 10.0.0.133 (executor driver) (32/200) +26/04/01 06:35:53 INFO Executor: Running task 35.0 in stage 24.0 (TID 345) +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:56 INFO Executor: Finished task 32.0 in stage 24.0 (TID 342). 9201 bytes result sent to driver +26/04/01 06:35:56 INFO TaskSetManager: Starting task 36.0 in stage 24.0 (TID 346) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:56 INFO Executor: Running task 36.0 in stage 24.0 (TID 346) +26/04/01 06:35:56 INFO TaskSetManager: Finished task 32.0 in stage 24.0 (TID 342) in 3359 ms on 10.0.0.133 (executor driver) (33/200) +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:56 INFO Executor: Finished task 33.0 in stage 24.0 (TID 343). 9201 bytes result sent to driver +26/04/01 06:35:56 INFO TaskSetManager: Starting task 37.0 in stage 24.0 (TID 347) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:56 INFO Executor: Running task 37.0 in stage 24.0 (TID 347) +26/04/01 06:35:56 INFO TaskSetManager: Finished task 33.0 in stage 24.0 (TID 343) in 3358 ms on 10.0.0.133 (executor driver) (34/200) +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:57 INFO Executor: Finished task 34.0 in stage 24.0 (TID 344). 9201 bytes result sent to driver +26/04/01 06:35:57 INFO Executor: Finished task 35.0 in stage 24.0 (TID 345). 9201 bytes result sent to driver +26/04/01 06:35:57 INFO TaskSetManager: Starting task 38.0 in stage 24.0 (TID 348) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:57 INFO Executor: Running task 38.0 in stage 24.0 (TID 348) +26/04/01 06:35:57 INFO TaskSetManager: Starting task 39.0 in stage 24.0 (TID 349) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:57 INFO Executor: Running task 39.0 in stage 24.0 (TID 349) +26/04/01 06:35:57 INFO TaskSetManager: Finished task 34.0 in stage 24.0 (TID 344) in 3434 ms on 10.0.0.133 (executor driver) (35/200) +26/04/01 06:35:57 INFO TaskSetManager: Finished task 35.0 in stage 24.0 (TID 345) in 3428 ms on 10.0.0.133 (executor driver) (36/200) +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:59 INFO Executor: Finished task 36.0 in stage 24.0 (TID 346). 8470 bytes result sent to driver +26/04/01 06:35:59 INFO TaskSetManager: Starting task 40.0 in stage 24.0 (TID 350) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9428 bytes) +26/04/01 06:35:59 INFO Executor: Running task 40.0 in stage 24.0 (TID 350) +26/04/01 06:35:59 INFO TaskSetManager: Finished task 36.0 in stage 24.0 (TID 346) in 3370 ms on 10.0.0.133 (executor driver) (37/200) +26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:35:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:35:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:00 INFO Executor: Finished task 37.0 in stage 24.0 (TID 347). 8470 bytes result sent to driver +26/04/01 06:36:00 INFO TaskSetManager: Starting task 41.0 in stage 24.0 (TID 351) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:00 INFO Executor: Running task 41.0 in stage 24.0 (TID 351) +26/04/01 06:36:00 INFO TaskSetManager: Finished task 37.0 in stage 24.0 (TID 347) in 3373 ms on 10.0.0.133 (executor driver) (38/200) +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:00 INFO Executor: Finished task 38.0 in stage 24.0 (TID 348). 8470 bytes result sent to driver +26/04/01 06:36:00 INFO TaskSetManager: Starting task 42.0 in stage 24.0 (TID 352) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:00 INFO TaskSetManager: Finished task 38.0 in stage 24.0 (TID 348) in 3442 ms on 10.0.0.133 (executor driver) (39/200) +26/04/01 06:36:00 INFO Executor: Running task 42.0 in stage 24.0 (TID 352) +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:00 INFO Executor: Finished task 39.0 in stage 24.0 (TID 349). 8470 bytes result sent to driver +26/04/01 06:36:00 INFO TaskSetManager: Starting task 43.0 in stage 24.0 (TID 353) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:00 INFO TaskSetManager: Finished task 39.0 in stage 24.0 (TID 349) in 3449 ms on 10.0.0.133 (executor driver) (40/200) +26/04/01 06:36:00 INFO Executor: Running task 43.0 in stage 24.0 (TID 353) +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:03 INFO Executor: Finished task 40.0 in stage 24.0 (TID 350). 8470 bytes result sent to driver +26/04/01 06:36:03 INFO TaskSetManager: Starting task 44.0 in stage 24.0 (TID 354) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:03 INFO TaskSetManager: Finished task 40.0 in stage 24.0 (TID 350) in 3358 ms on 10.0.0.133 (executor driver) (41/200) +26/04/01 06:36:03 INFO Executor: Running task 44.0 in stage 24.0 (TID 354) +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:03 INFO Executor: Finished task 41.0 in stage 24.0 (TID 351). 8470 bytes result sent to driver +26/04/01 06:36:03 INFO TaskSetManager: Starting task 45.0 in stage 24.0 (TID 355) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:03 INFO TaskSetManager: Finished task 41.0 in stage 24.0 (TID 351) in 3353 ms on 10.0.0.133 (executor driver) (42/200) +26/04/01 06:36:03 INFO Executor: Running task 45.0 in stage 24.0 (TID 355) +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:04 INFO Executor: Finished task 43.0 in stage 24.0 (TID 353). 8470 bytes result sent to driver +26/04/01 06:36:04 INFO TaskSetManager: Starting task 46.0 in stage 24.0 (TID 356) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:04 INFO TaskSetManager: Finished task 43.0 in stage 24.0 (TID 353) in 3417 ms on 10.0.0.133 (executor driver) (43/200) +26/04/01 06:36:04 INFO Executor: Running task 46.0 in stage 24.0 (TID 356) +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:04 INFO Executor: Finished task 42.0 in stage 24.0 (TID 352). 8470 bytes result sent to driver +26/04/01 06:36:04 INFO TaskSetManager: Starting task 47.0 in stage 24.0 (TID 357) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:04 INFO Executor: Running task 47.0 in stage 24.0 (TID 357) +26/04/01 06:36:04 INFO TaskSetManager: Finished task 42.0 in stage 24.0 (TID 352) in 3429 ms on 10.0.0.133 (executor driver) (44/200) +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:06 INFO Executor: Finished task 44.0 in stage 24.0 (TID 354). 9201 bytes result sent to driver +26/04/01 06:36:06 INFO TaskSetManager: Starting task 48.0 in stage 24.0 (TID 358) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:06 INFO TaskSetManager: Finished task 44.0 in stage 24.0 (TID 354) in 3364 ms on 10.0.0.133 (executor driver) (45/200) +26/04/01 06:36:06 INFO Executor: Running task 48.0 in stage 24.0 (TID 358) +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:06 INFO Executor: Finished task 45.0 in stage 24.0 (TID 355). 9201 bytes result sent to driver +26/04/01 06:36:06 INFO TaskSetManager: Starting task 49.0 in stage 24.0 (TID 359) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:06 INFO Executor: Running task 49.0 in stage 24.0 (TID 359) +26/04/01 06:36:06 INFO TaskSetManager: Finished task 45.0 in stage 24.0 (TID 355) in 3366 ms on 10.0.0.133 (executor driver) (46/200) +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:07 INFO Executor: Finished task 47.0 in stage 24.0 (TID 357). 9201 bytes result sent to driver +26/04/01 06:36:07 INFO Executor: Finished task 46.0 in stage 24.0 (TID 356). 9201 bytes result sent to driver +26/04/01 06:36:07 INFO TaskSetManager: Starting task 50.0 in stage 24.0 (TID 360) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:07 INFO Executor: Running task 50.0 in stage 24.0 (TID 360) +26/04/01 06:36:07 INFO TaskSetManager: Starting task 51.0 in stage 24.0 (TID 361) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:07 INFO Executor: Running task 51.0 in stage 24.0 (TID 361) +26/04/01 06:36:07 INFO TaskSetManager: Finished task 47.0 in stage 24.0 (TID 357) in 3430 ms on 10.0.0.133 (executor driver) (47/200) +26/04/01 06:36:07 INFO TaskSetManager: Finished task 46.0 in stage 24.0 (TID 356) in 3435 ms on 10.0.0.133 (executor driver) (48/200) +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:10 INFO Executor: Finished task 48.0 in stage 24.0 (TID 358). 8470 bytes result sent to driver +26/04/01 06:36:10 INFO TaskSetManager: Starting task 52.0 in stage 24.0 (TID 362) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:10 INFO TaskSetManager: Finished task 48.0 in stage 24.0 (TID 358) in 3391 ms on 10.0.0.133 (executor driver) (49/200) +26/04/01 06:36:10 INFO Executor: Running task 52.0 in stage 24.0 (TID 362) +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:10 INFO Executor: Finished task 49.0 in stage 24.0 (TID 359). 8470 bytes result sent to driver +26/04/01 06:36:10 INFO TaskSetManager: Starting task 53.0 in stage 24.0 (TID 363) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:10 INFO TaskSetManager: Finished task 49.0 in stage 24.0 (TID 359) in 3384 ms on 10.0.0.133 (executor driver) (50/200) +26/04/01 06:36:10 INFO Executor: Running task 53.0 in stage 24.0 (TID 363) +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:10 INFO Executor: Finished task 51.0 in stage 24.0 (TID 361). 8470 bytes result sent to driver +26/04/01 06:36:10 INFO TaskSetManager: Starting task 54.0 in stage 24.0 (TID 364) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:10 INFO Executor: Running task 54.0 in stage 24.0 (TID 364) +26/04/01 06:36:10 INFO TaskSetManager: Finished task 51.0 in stage 24.0 (TID 361) in 3455 ms on 10.0.0.133 (executor driver) (51/200) +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:10 INFO Executor: Finished task 50.0 in stage 24.0 (TID 360). 8470 bytes result sent to driver +26/04/01 06:36:10 INFO TaskSetManager: Starting task 55.0 in stage 24.0 (TID 365) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:10 INFO TaskSetManager: Finished task 50.0 in stage 24.0 (TID 360) in 3473 ms on 10.0.0.133 (executor driver) (52/200) +26/04/01 06:36:10 INFO Executor: Running task 55.0 in stage 24.0 (TID 365) +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:13 INFO Executor: Finished task 52.0 in stage 24.0 (TID 362). 8470 bytes result sent to driver +26/04/01 06:36:13 INFO TaskSetManager: Starting task 56.0 in stage 24.0 (TID 366) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:13 INFO TaskSetManager: Finished task 52.0 in stage 24.0 (TID 362) in 3351 ms on 10.0.0.133 (executor driver) (53/200) +26/04/01 06:36:13 INFO Executor: Running task 56.0 in stage 24.0 (TID 366) +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:13 INFO Executor: Finished task 53.0 in stage 24.0 (TID 363). 8470 bytes result sent to driver +26/04/01 06:36:13 INFO TaskSetManager: Starting task 57.0 in stage 24.0 (TID 367) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:13 INFO TaskSetManager: Finished task 53.0 in stage 24.0 (TID 363) in 3376 ms on 10.0.0.133 (executor driver) (54/200) +26/04/01 06:36:13 INFO Executor: Running task 57.0 in stage 24.0 (TID 367) +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:14 INFO Executor: Finished task 54.0 in stage 24.0 (TID 364). 8470 bytes result sent to driver +26/04/01 06:36:14 INFO TaskSetManager: Starting task 58.0 in stage 24.0 (TID 368) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:14 INFO Executor: Running task 58.0 in stage 24.0 (TID 368) +26/04/01 06:36:14 INFO TaskSetManager: Finished task 54.0 in stage 24.0 (TID 364) in 3403 ms on 10.0.0.133 (executor driver) (55/200) +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:14 INFO Executor: Finished task 55.0 in stage 24.0 (TID 365). 8470 bytes result sent to driver +26/04/01 06:36:14 INFO TaskSetManager: Starting task 59.0 in stage 24.0 (TID 369) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:14 INFO TaskSetManager: Finished task 55.0 in stage 24.0 (TID 365) in 3403 ms on 10.0.0.133 (executor driver) (56/200) +26/04/01 06:36:14 INFO Executor: Running task 59.0 in stage 24.0 (TID 369) +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:16 INFO Executor: Finished task 56.0 in stage 24.0 (TID 366). 9201 bytes result sent to driver +26/04/01 06:36:16 INFO TaskSetManager: Starting task 60.0 in stage 24.0 (TID 370) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:16 INFO Executor: Running task 60.0 in stage 24.0 (TID 370) +26/04/01 06:36:16 INFO TaskSetManager: Finished task 56.0 in stage 24.0 (TID 366) in 3376 ms on 10.0.0.133 (executor driver) (57/200) +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:16 INFO Executor: Finished task 57.0 in stage 24.0 (TID 367). 9201 bytes result sent to driver +26/04/01 06:36:16 INFO TaskSetManager: Starting task 61.0 in stage 24.0 (TID 371) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:16 INFO TaskSetManager: Finished task 57.0 in stage 24.0 (TID 367) in 3376 ms on 10.0.0.133 (executor driver) (58/200) +26/04/01 06:36:16 INFO Executor: Running task 61.0 in stage 24.0 (TID 371) +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:17 INFO Executor: Finished task 59.0 in stage 24.0 (TID 369). 9201 bytes result sent to driver +26/04/01 06:36:17 INFO TaskSetManager: Starting task 62.0 in stage 24.0 (TID 372) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:17 INFO Executor: Running task 62.0 in stage 24.0 (TID 372) +26/04/01 06:36:17 INFO TaskSetManager: Finished task 59.0 in stage 24.0 (TID 369) in 3440 ms on 10.0.0.133 (executor driver) (59/200) +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:17 INFO Executor: Finished task 58.0 in stage 24.0 (TID 368). 9201 bytes result sent to driver +26/04/01 06:36:17 INFO TaskSetManager: Starting task 63.0 in stage 24.0 (TID 373) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:17 INFO Executor: Running task 63.0 in stage 24.0 (TID 373) +26/04/01 06:36:17 INFO TaskSetManager: Finished task 58.0 in stage 24.0 (TID 368) in 3463 ms on 10.0.0.133 (executor driver) (60/200) +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 6 (999.4 KiB) non-empty blocks including 6 (999.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 6 (999.4 KiB) non-empty blocks including 6 (999.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:20 INFO Executor: Finished task 60.0 in stage 24.0 (TID 370). 8470 bytes result sent to driver +26/04/01 06:36:20 INFO TaskSetManager: Starting task 64.0 in stage 24.0 (TID 374) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:20 INFO TaskSetManager: Finished task 60.0 in stage 24.0 (TID 370) in 3374 ms on 10.0.0.133 (executor driver) (61/200) +26/04/01 06:36:20 INFO Executor: Running task 64.0 in stage 24.0 (TID 374) +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.3 MiB) non-empty blocks including 208 (45.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.3 MiB) non-empty blocks including 208 (45.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:20 INFO Executor: Finished task 61.0 in stage 24.0 (TID 371). 8470 bytes result sent to driver +26/04/01 06:36:20 INFO TaskSetManager: Starting task 65.0 in stage 24.0 (TID 375) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:20 INFO TaskSetManager: Finished task 61.0 in stage 24.0 (TID 371) in 3373 ms on 10.0.0.133 (executor driver) (62/200) +26/04/01 06:36:20 INFO Executor: Running task 65.0 in stage 24.0 (TID 375) +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:21 INFO Executor: Finished task 63.0 in stage 24.0 (TID 373). 8470 bytes result sent to driver +26/04/01 06:36:21 INFO TaskSetManager: Starting task 66.0 in stage 24.0 (TID 376) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:21 INFO TaskSetManager: Finished task 63.0 in stage 24.0 (TID 373) in 3426 ms on 10.0.0.133 (executor driver) (63/200) +26/04/01 06:36:21 INFO Executor: Running task 66.0 in stage 24.0 (TID 376) +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:21 INFO Executor: Finished task 62.0 in stage 24.0 (TID 372). 8470 bytes result sent to driver +26/04/01 06:36:21 INFO TaskSetManager: Starting task 67.0 in stage 24.0 (TID 377) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:21 INFO TaskSetManager: Finished task 62.0 in stage 24.0 (TID 372) in 3442 ms on 10.0.0.133 (executor driver) (64/200) +26/04/01 06:36:21 INFO Executor: Running task 67.0 in stage 24.0 (TID 377) +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:23 INFO Executor: Finished task 64.0 in stage 24.0 (TID 374). 8470 bytes result sent to driver +26/04/01 06:36:23 INFO TaskSetManager: Starting task 68.0 in stage 24.0 (TID 378) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:23 INFO Executor: Running task 68.0 in stage 24.0 (TID 378) +26/04/01 06:36:23 INFO TaskSetManager: Finished task 64.0 in stage 24.0 (TID 374) in 3364 ms on 10.0.0.133 (executor driver) (65/200) +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:23 INFO Executor: Finished task 65.0 in stage 24.0 (TID 375). 8470 bytes result sent to driver +26/04/01 06:36:23 INFO TaskSetManager: Starting task 69.0 in stage 24.0 (TID 379) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:23 INFO TaskSetManager: Finished task 65.0 in stage 24.0 (TID 375) in 3368 ms on 10.0.0.133 (executor driver) (66/200) +26/04/01 06:36:23 INFO Executor: Running task 69.0 in stage 24.0 (TID 379) +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:24 INFO Executor: Finished task 67.0 in stage 24.0 (TID 377). 8470 bytes result sent to driver +26/04/01 06:36:24 INFO TaskSetManager: Starting task 70.0 in stage 24.0 (TID 380) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:24 INFO Executor: Running task 70.0 in stage 24.0 (TID 380) +26/04/01 06:36:24 INFO TaskSetManager: Finished task 67.0 in stage 24.0 (TID 377) in 3417 ms on 10.0.0.133 (executor driver) (67/200) +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:24 INFO Executor: Finished task 66.0 in stage 24.0 (TID 376). 8470 bytes result sent to driver +26/04/01 06:36:24 INFO TaskSetManager: Starting task 71.0 in stage 24.0 (TID 381) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:24 INFO Executor: Running task 71.0 in stage 24.0 (TID 381) +26/04/01 06:36:24 INFO TaskSetManager: Finished task 66.0 in stage 24.0 (TID 376) in 3439 ms on 10.0.0.133 (executor driver) (68/200) +26/04/01 06:36:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:26 INFO Executor: Finished task 68.0 in stage 24.0 (TID 378). 9201 bytes result sent to driver +26/04/01 06:36:26 INFO TaskSetManager: Starting task 72.0 in stage 24.0 (TID 382) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:26 INFO TaskSetManager: Finished task 68.0 in stage 24.0 (TID 378) in 3367 ms on 10.0.0.133 (executor driver) (69/200) +26/04/01 06:36:26 INFO Executor: Running task 72.0 in stage 24.0 (TID 382) +26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:27 INFO Executor: Finished task 69.0 in stage 24.0 (TID 379). 9201 bytes result sent to driver +26/04/01 06:36:27 INFO TaskSetManager: Starting task 73.0 in stage 24.0 (TID 383) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:27 INFO Executor: Running task 73.0 in stage 24.0 (TID 383) +26/04/01 06:36:27 INFO TaskSetManager: Finished task 69.0 in stage 24.0 (TID 379) in 3359 ms on 10.0.0.133 (executor driver) (70/200) +26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:28 INFO Executor: Finished task 70.0 in stage 24.0 (TID 380). 9201 bytes result sent to driver +26/04/01 06:36:28 INFO TaskSetManager: Starting task 74.0 in stage 24.0 (TID 384) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:28 INFO TaskSetManager: Finished task 70.0 in stage 24.0 (TID 380) in 3419 ms on 10.0.0.133 (executor driver) (71/200) +26/04/01 06:36:28 INFO Executor: Running task 74.0 in stage 24.0 (TID 384) +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:28 INFO Executor: Finished task 71.0 in stage 24.0 (TID 381). 9201 bytes result sent to driver +26/04/01 06:36:28 INFO TaskSetManager: Starting task 75.0 in stage 24.0 (TID 385) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:28 INFO Executor: Running task 75.0 in stage 24.0 (TID 385) +26/04/01 06:36:28 INFO TaskSetManager: Finished task 71.0 in stage 24.0 (TID 381) in 3415 ms on 10.0.0.133 (executor driver) (72/200) +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:30 INFO Executor: Finished task 72.0 in stage 24.0 (TID 382). 8470 bytes result sent to driver +26/04/01 06:36:30 INFO TaskSetManager: Starting task 76.0 in stage 24.0 (TID 386) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:30 INFO Executor: Running task 76.0 in stage 24.0 (TID 386) +26/04/01 06:36:30 INFO TaskSetManager: Finished task 72.0 in stage 24.0 (TID 382) in 3375 ms on 10.0.0.133 (executor driver) (73/200) +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:30 INFO Executor: Finished task 73.0 in stage 24.0 (TID 383). 8470 bytes result sent to driver +26/04/01 06:36:30 INFO TaskSetManager: Starting task 77.0 in stage 24.0 (TID 387) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:30 INFO TaskSetManager: Finished task 73.0 in stage 24.0 (TID 383) in 3370 ms on 10.0.0.133 (executor driver) (74/200) +26/04/01 06:36:30 INFO Executor: Running task 77.0 in stage 24.0 (TID 387) +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:31 INFO Executor: Finished task 74.0 in stage 24.0 (TID 384). 8470 bytes result sent to driver +26/04/01 06:36:31 INFO TaskSetManager: Starting task 78.0 in stage 24.0 (TID 388) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:31 INFO Executor: Running task 78.0 in stage 24.0 (TID 388) +26/04/01 06:36:31 INFO TaskSetManager: Finished task 74.0 in stage 24.0 (TID 384) in 3444 ms on 10.0.0.133 (executor driver) (75/200) +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:31 INFO Executor: Finished task 75.0 in stage 24.0 (TID 385). 8470 bytes result sent to driver +26/04/01 06:36:31 INFO TaskSetManager: Starting task 79.0 in stage 24.0 (TID 389) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:31 INFO Executor: Running task 79.0 in stage 24.0 (TID 389) +26/04/01 06:36:31 INFO TaskSetManager: Finished task 75.0 in stage 24.0 (TID 385) in 3450 ms on 10.0.0.133 (executor driver) (76/200) +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:33 INFO Executor: Finished task 76.0 in stage 24.0 (TID 386). 8470 bytes result sent to driver +26/04/01 06:36:33 INFO TaskSetManager: Starting task 80.0 in stage 24.0 (TID 390) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:33 INFO Executor: Running task 80.0 in stage 24.0 (TID 390) +26/04/01 06:36:33 INFO TaskSetManager: Finished task 76.0 in stage 24.0 (TID 386) in 3354 ms on 10.0.0.133 (executor driver) (77/200) +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:33 INFO Executor: Finished task 77.0 in stage 24.0 (TID 387). 8470 bytes result sent to driver +26/04/01 06:36:33 INFO TaskSetManager: Starting task 81.0 in stage 24.0 (TID 391) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:33 INFO TaskSetManager: Finished task 77.0 in stage 24.0 (TID 387) in 3356 ms on 10.0.0.133 (executor driver) (78/200) +26/04/01 06:36:33 INFO Executor: Running task 81.0 in stage 24.0 (TID 391) +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:34 INFO Executor: Finished task 79.0 in stage 24.0 (TID 389). 9201 bytes result sent to driver +26/04/01 06:36:34 INFO TaskSetManager: Starting task 82.0 in stage 24.0 (TID 392) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:34 INFO Executor: Running task 82.0 in stage 24.0 (TID 392) +26/04/01 06:36:34 INFO TaskSetManager: Finished task 79.0 in stage 24.0 (TID 389) in 3408 ms on 10.0.0.133 (executor driver) (79/200) +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:34 INFO Executor: Finished task 78.0 in stage 24.0 (TID 388). 9201 bytes result sent to driver +26/04/01 06:36:34 INFO TaskSetManager: Starting task 83.0 in stage 24.0 (TID 393) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:34 INFO Executor: Running task 83.0 in stage 24.0 (TID 393) +26/04/01 06:36:34 INFO TaskSetManager: Finished task 78.0 in stage 24.0 (TID 388) in 3443 ms on 10.0.0.133 (executor driver) (80/200) +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:37 INFO Executor: Finished task 80.0 in stage 24.0 (TID 390). 9201 bytes result sent to driver +26/04/01 06:36:37 INFO TaskSetManager: Starting task 84.0 in stage 24.0 (TID 394) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:37 INFO Executor: Running task 84.0 in stage 24.0 (TID 394) +26/04/01 06:36:37 INFO TaskSetManager: Finished task 80.0 in stage 24.0 (TID 390) in 3382 ms on 10.0.0.133 (executor driver) (81/200) +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:37 INFO Executor: Finished task 81.0 in stage 24.0 (TID 391). 9201 bytes result sent to driver +26/04/01 06:36:37 INFO TaskSetManager: Starting task 85.0 in stage 24.0 (TID 395) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:37 INFO Executor: Running task 85.0 in stage 24.0 (TID 395) +26/04/01 06:36:37 INFO TaskSetManager: Finished task 81.0 in stage 24.0 (TID 391) in 3370 ms on 10.0.0.133 (executor driver) (82/200) +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:38 INFO Executor: Finished task 82.0 in stage 24.0 (TID 392). 8470 bytes result sent to driver +26/04/01 06:36:38 INFO TaskSetManager: Starting task 86.0 in stage 24.0 (TID 396) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:38 INFO Executor: Running task 86.0 in stage 24.0 (TID 396) +26/04/01 06:36:38 INFO TaskSetManager: Finished task 82.0 in stage 24.0 (TID 392) in 3423 ms on 10.0.0.133 (executor driver) (83/200) +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:38 INFO Executor: Finished task 83.0 in stage 24.0 (TID 393). 8470 bytes result sent to driver +26/04/01 06:36:38 INFO TaskSetManager: Starting task 87.0 in stage 24.0 (TID 397) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:38 INFO Executor: Running task 87.0 in stage 24.0 (TID 397) +26/04/01 06:36:38 INFO TaskSetManager: Finished task 83.0 in stage 24.0 (TID 393) in 3414 ms on 10.0.0.133 (executor driver) (84/200) +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:40 INFO Executor: Finished task 84.0 in stage 24.0 (TID 394). 8470 bytes result sent to driver +26/04/01 06:36:40 INFO TaskSetManager: Starting task 88.0 in stage 24.0 (TID 398) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:40 INFO Executor: Running task 88.0 in stage 24.0 (TID 398) +26/04/01 06:36:40 INFO TaskSetManager: Finished task 84.0 in stage 24.0 (TID 394) in 3382 ms on 10.0.0.133 (executor driver) (85/200) +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:40 INFO Executor: Finished task 85.0 in stage 24.0 (TID 395). 8470 bytes result sent to driver +26/04/01 06:36:40 INFO TaskSetManager: Starting task 89.0 in stage 24.0 (TID 399) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:40 INFO Executor: Running task 89.0 in stage 24.0 (TID 399) +26/04/01 06:36:40 INFO TaskSetManager: Finished task 85.0 in stage 24.0 (TID 395) in 3381 ms on 10.0.0.133 (executor driver) (86/200) +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:41 INFO Executor: Finished task 87.0 in stage 24.0 (TID 397). 8470 bytes result sent to driver +26/04/01 06:36:41 INFO TaskSetManager: Starting task 90.0 in stage 24.0 (TID 400) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:41 INFO Executor: Running task 90.0 in stage 24.0 (TID 400) +26/04/01 06:36:41 INFO TaskSetManager: Finished task 87.0 in stage 24.0 (TID 397) in 3422 ms on 10.0.0.133 (executor driver) (87/200) +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:41 INFO Executor: Finished task 86.0 in stage 24.0 (TID 396). 8470 bytes result sent to driver +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:41 INFO TaskSetManager: Starting task 91.0 in stage 24.0 (TID 401) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:41 INFO Executor: Running task 91.0 in stage 24.0 (TID 401) +26/04/01 06:36:41 INFO TaskSetManager: Finished task 86.0 in stage 24.0 (TID 396) in 3441 ms on 10.0.0.133 (executor driver) (88/200) +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:43 INFO Executor: Finished task 88.0 in stage 24.0 (TID 398). 8470 bytes result sent to driver +26/04/01 06:36:43 INFO TaskSetManager: Starting task 92.0 in stage 24.0 (TID 402) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:43 INFO Executor: Running task 92.0 in stage 24.0 (TID 402) +26/04/01 06:36:43 INFO TaskSetManager: Finished task 88.0 in stage 24.0 (TID 398) in 3376 ms on 10.0.0.133 (executor driver) (89/200) +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:43 INFO Executor: Finished task 89.0 in stage 24.0 (TID 399). 8470 bytes result sent to driver +26/04/01 06:36:43 INFO TaskSetManager: Starting task 93.0 in stage 24.0 (TID 403) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:43 INFO Executor: Running task 93.0 in stage 24.0 (TID 403) +26/04/01 06:36:43 INFO TaskSetManager: Finished task 89.0 in stage 24.0 (TID 399) in 3384 ms on 10.0.0.133 (executor driver) (90/200) +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:45 INFO Executor: Finished task 91.0 in stage 24.0 (TID 401). 9201 bytes result sent to driver +26/04/01 06:36:45 INFO TaskSetManager: Starting task 94.0 in stage 24.0 (TID 404) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:45 INFO Executor: Running task 94.0 in stage 24.0 (TID 404) +26/04/01 06:36:45 INFO TaskSetManager: Finished task 91.0 in stage 24.0 (TID 401) in 3424 ms on 10.0.0.133 (executor driver) (91/200) +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:45 INFO Executor: Finished task 90.0 in stage 24.0 (TID 400). 9201 bytes result sent to driver +26/04/01 06:36:45 INFO TaskSetManager: Starting task 95.0 in stage 24.0 (TID 405) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:45 INFO Executor: Running task 95.0 in stage 24.0 (TID 405) +26/04/01 06:36:45 INFO TaskSetManager: Finished task 90.0 in stage 24.0 (TID 400) in 3434 ms on 10.0.0.133 (executor driver) (92/200) +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:47 INFO Executor: Finished task 92.0 in stage 24.0 (TID 402). 9201 bytes result sent to driver +26/04/01 06:36:47 INFO TaskSetManager: Starting task 96.0 in stage 24.0 (TID 406) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:47 INFO Executor: Running task 96.0 in stage 24.0 (TID 406) +26/04/01 06:36:47 INFO TaskSetManager: Finished task 92.0 in stage 24.0 (TID 402) in 3380 ms on 10.0.0.133 (executor driver) (93/200) +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:47 INFO Executor: Finished task 93.0 in stage 24.0 (TID 403). 9201 bytes result sent to driver +26/04/01 06:36:47 INFO TaskSetManager: Starting task 97.0 in stage 24.0 (TID 407) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:47 INFO Executor: Running task 97.0 in stage 24.0 (TID 407) +26/04/01 06:36:47 INFO TaskSetManager: Finished task 93.0 in stage 24.0 (TID 403) in 3390 ms on 10.0.0.133 (executor driver) (94/200) +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:48 INFO Executor: Finished task 94.0 in stage 24.0 (TID 404). 8470 bytes result sent to driver +26/04/01 06:36:48 INFO Executor: Finished task 95.0 in stage 24.0 (TID 405). 8470 bytes result sent to driver +26/04/01 06:36:48 INFO TaskSetManager: Starting task 98.0 in stage 24.0 (TID 408) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:48 INFO Executor: Running task 98.0 in stage 24.0 (TID 408) +26/04/01 06:36:48 INFO TaskSetManager: Starting task 99.0 in stage 24.0 (TID 409) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:48 INFO TaskSetManager: Finished task 94.0 in stage 24.0 (TID 404) in 3442 ms on 10.0.0.133 (executor driver) (95/200) +26/04/01 06:36:48 INFO Executor: Running task 99.0 in stage 24.0 (TID 409) +26/04/01 06:36:48 INFO TaskSetManager: Finished task 95.0 in stage 24.0 (TID 405) in 3440 ms on 10.0.0.133 (executor driver) (96/200) +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:50 INFO Executor: Finished task 96.0 in stage 24.0 (TID 406). 8470 bytes result sent to driver +26/04/01 06:36:50 INFO TaskSetManager: Starting task 100.0 in stage 24.0 (TID 410) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:50 INFO Executor: Running task 100.0 in stage 24.0 (TID 410) +26/04/01 06:36:50 INFO TaskSetManager: Finished task 96.0 in stage 24.0 (TID 406) in 3380 ms on 10.0.0.133 (executor driver) (97/200) +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:50 INFO Executor: Finished task 97.0 in stage 24.0 (TID 407). 8470 bytes result sent to driver +26/04/01 06:36:50 INFO TaskSetManager: Starting task 101.0 in stage 24.0 (TID 411) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:50 INFO Executor: Running task 101.0 in stage 24.0 (TID 411) +26/04/01 06:36:50 INFO TaskSetManager: Finished task 97.0 in stage 24.0 (TID 407) in 3387 ms on 10.0.0.133 (executor driver) (98/200) +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:52 INFO Executor: Finished task 98.0 in stage 24.0 (TID 408). 8470 bytes result sent to driver +26/04/01 06:36:52 INFO TaskSetManager: Starting task 102.0 in stage 24.0 (TID 412) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:52 INFO Executor: Running task 102.0 in stage 24.0 (TID 412) +26/04/01 06:36:52 INFO TaskSetManager: Finished task 98.0 in stage 24.0 (TID 408) in 3441 ms on 10.0.0.133 (executor driver) (99/200) +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:52 INFO Executor: Finished task 99.0 in stage 24.0 (TID 409). 8470 bytes result sent to driver +26/04/01 06:36:52 INFO TaskSetManager: Starting task 103.0 in stage 24.0 (TID 413) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:52 INFO TaskSetManager: Finished task 99.0 in stage 24.0 (TID 409) in 3446 ms on 10.0.0.133 (executor driver) (100/200) +26/04/01 06:36:52 INFO Executor: Running task 103.0 in stage 24.0 (TID 413) +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:53 INFO Executor: Finished task 100.0 in stage 24.0 (TID 410). 8470 bytes result sent to driver +26/04/01 06:36:53 INFO TaskSetManager: Starting task 104.0 in stage 24.0 (TID 414) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:53 INFO Executor: Running task 104.0 in stage 24.0 (TID 414) +26/04/01 06:36:53 INFO TaskSetManager: Finished task 100.0 in stage 24.0 (TID 410) in 3382 ms on 10.0.0.133 (executor driver) (101/200) +26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:54 INFO Executor: Finished task 101.0 in stage 24.0 (TID 411). 8470 bytes result sent to driver +26/04/01 06:36:54 INFO TaskSetManager: Starting task 105.0 in stage 24.0 (TID 415) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:54 INFO Executor: Running task 105.0 in stage 24.0 (TID 415) +26/04/01 06:36:54 INFO TaskSetManager: Finished task 101.0 in stage 24.0 (TID 411) in 3356 ms on 10.0.0.133 (executor driver) (102/200) +26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:55 INFO Executor: Finished task 103.0 in stage 24.0 (TID 413). 9201 bytes result sent to driver +26/04/01 06:36:55 INFO TaskSetManager: Starting task 106.0 in stage 24.0 (TID 416) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:55 INFO Executor: Running task 106.0 in stage 24.0 (TID 416) +26/04/01 06:36:55 INFO TaskSetManager: Finished task 103.0 in stage 24.0 (TID 413) in 3428 ms on 10.0.0.133 (executor driver) (103/200) +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:55 INFO Executor: Finished task 102.0 in stage 24.0 (TID 412). 9201 bytes result sent to driver +26/04/01 06:36:55 INFO TaskSetManager: Starting task 107.0 in stage 24.0 (TID 417) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:55 INFO TaskSetManager: Finished task 102.0 in stage 24.0 (TID 412) in 3443 ms on 10.0.0.133 (executor driver) (104/200) +26/04/01 06:36:55 INFO Executor: Running task 107.0 in stage 24.0 (TID 417) +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:57 INFO Executor: Finished task 104.0 in stage 24.0 (TID 414). 9201 bytes result sent to driver +26/04/01 06:36:57 INFO TaskSetManager: Starting task 108.0 in stage 24.0 (TID 418) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:57 INFO Executor: Running task 108.0 in stage 24.0 (TID 418) +26/04/01 06:36:57 INFO TaskSetManager: Finished task 104.0 in stage 24.0 (TID 414) in 3360 ms on 10.0.0.133 (executor driver) (105/200) +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:57 INFO Executor: Finished task 105.0 in stage 24.0 (TID 415). 9201 bytes result sent to driver +26/04/01 06:36:57 INFO TaskSetManager: Starting task 109.0 in stage 24.0 (TID 419) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:57 INFO Executor: Running task 109.0 in stage 24.0 (TID 419) +26/04/01 06:36:57 INFO TaskSetManager: Finished task 105.0 in stage 24.0 (TID 415) in 3374 ms on 10.0.0.133 (executor driver) (106/200) +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:58 INFO Executor: Finished task 106.0 in stage 24.0 (TID 416). 8470 bytes result sent to driver +26/04/01 06:36:58 INFO TaskSetManager: Starting task 110.0 in stage 24.0 (TID 420) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:58 INFO Executor: Running task 110.0 in stage 24.0 (TID 420) +26/04/01 06:36:58 INFO TaskSetManager: Finished task 106.0 in stage 24.0 (TID 416) in 3441 ms on 10.0.0.133 (executor driver) (107/200) +26/04/01 06:36:58 INFO Executor: Finished task 107.0 in stage 24.0 (TID 417). 8470 bytes result sent to driver +26/04/01 06:36:58 INFO TaskSetManager: Starting task 111.0 in stage 24.0 (TID 421) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9428 bytes) +26/04/01 06:36:58 INFO Executor: Running task 111.0 in stage 24.0 (TID 421) +26/04/01 06:36:58 INFO TaskSetManager: Finished task 107.0 in stage 24.0 (TID 417) in 3433 ms on 10.0.0.133 (executor driver) (108/200) +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:36:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:36:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:00 INFO Executor: Finished task 108.0 in stage 24.0 (TID 418). 8470 bytes result sent to driver +26/04/01 06:37:00 INFO TaskSetManager: Starting task 112.0 in stage 24.0 (TID 422) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:00 INFO Executor: Running task 112.0 in stage 24.0 (TID 422) +26/04/01 06:37:00 INFO TaskSetManager: Finished task 108.0 in stage 24.0 (TID 418) in 3393 ms on 10.0.0.133 (executor driver) (109/200) +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:00 INFO Executor: Finished task 109.0 in stage 24.0 (TID 419). 8470 bytes result sent to driver +26/04/01 06:37:00 INFO TaskSetManager: Starting task 113.0 in stage 24.0 (TID 423) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:00 INFO Executor: Running task 113.0 in stage 24.0 (TID 423) +26/04/01 06:37:00 INFO TaskSetManager: Finished task 109.0 in stage 24.0 (TID 419) in 3384 ms on 10.0.0.133 (executor driver) (110/200) +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:02 INFO Executor: Finished task 111.0 in stage 24.0 (TID 421). 8470 bytes result sent to driver +26/04/01 06:37:02 INFO TaskSetManager: Starting task 114.0 in stage 24.0 (TID 424) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:02 INFO Executor: Running task 114.0 in stage 24.0 (TID 424) +26/04/01 06:37:02 INFO TaskSetManager: Finished task 111.0 in stage 24.0 (TID 421) in 3434 ms on 10.0.0.133 (executor driver) (111/200) +26/04/01 06:37:02 INFO Executor: Finished task 110.0 in stage 24.0 (TID 420). 8470 bytes result sent to driver +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:02 INFO TaskSetManager: Starting task 115.0 in stage 24.0 (TID 425) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:02 INFO Executor: Running task 115.0 in stage 24.0 (TID 425) +26/04/01 06:37:02 INFO TaskSetManager: Finished task 110.0 in stage 24.0 (TID 420) in 3436 ms on 10.0.0.133 (executor driver) (112/200) +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:04 INFO Executor: Finished task 112.0 in stage 24.0 (TID 422). 8470 bytes result sent to driver +26/04/01 06:37:04 INFO TaskSetManager: Starting task 116.0 in stage 24.0 (TID 426) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:04 INFO Executor: Running task 116.0 in stage 24.0 (TID 426) +26/04/01 06:37:04 INFO TaskSetManager: Finished task 112.0 in stage 24.0 (TID 422) in 3383 ms on 10.0.0.133 (executor driver) (113/200) +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:04 INFO Executor: Finished task 113.0 in stage 24.0 (TID 423). 8470 bytes result sent to driver +26/04/01 06:37:04 INFO TaskSetManager: Starting task 117.0 in stage 24.0 (TID 427) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:04 INFO TaskSetManager: Finished task 113.0 in stage 24.0 (TID 423) in 3375 ms on 10.0.0.133 (executor driver) (114/200) +26/04/01 06:37:04 INFO Executor: Running task 117.0 in stage 24.0 (TID 427) +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:05 INFO Executor: Finished task 114.0 in stage 24.0 (TID 424). 9201 bytes result sent to driver +26/04/01 06:37:05 INFO TaskSetManager: Starting task 118.0 in stage 24.0 (TID 428) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:05 INFO Executor: Running task 118.0 in stage 24.0 (TID 428) +26/04/01 06:37:05 INFO TaskSetManager: Finished task 114.0 in stage 24.0 (TID 424) in 3436 ms on 10.0.0.133 (executor driver) (115/200) +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1017.0 KiB) non-empty blocks including 6 (1017.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:05 INFO Executor: Finished task 115.0 in stage 24.0 (TID 425). 9201 bytes result sent to driver +26/04/01 06:37:05 INFO TaskSetManager: Starting task 119.0 in stage 24.0 (TID 429) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:05 INFO TaskSetManager: Finished task 115.0 in stage 24.0 (TID 425) in 3437 ms on 10.0.0.133 (executor driver) (116/200) +26/04/01 06:37:05 INFO Executor: Running task 119.0 in stage 24.0 (TID 429) +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1017.0 KiB) non-empty blocks including 6 (1017.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:07 INFO Executor: Finished task 116.0 in stage 24.0 (TID 426). 9201 bytes result sent to driver +26/04/01 06:37:07 INFO TaskSetManager: Starting task 120.0 in stage 24.0 (TID 430) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:07 INFO TaskSetManager: Finished task 116.0 in stage 24.0 (TID 426) in 3373 ms on 10.0.0.133 (executor driver) (117/200) +26/04/01 06:37:07 INFO Executor: Running task 120.0 in stage 24.0 (TID 430) +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:07 INFO Executor: Finished task 117.0 in stage 24.0 (TID 427). 9201 bytes result sent to driver +26/04/01 06:37:07 INFO TaskSetManager: Starting task 121.0 in stage 24.0 (TID 431) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:07 INFO TaskSetManager: Finished task 117.0 in stage 24.0 (TID 427) in 3379 ms on 10.0.0.133 (executor driver) (118/200) +26/04/01 06:37:07 INFO Executor: Running task 121.0 in stage 24.0 (TID 431) +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:09 INFO Executor: Finished task 119.0 in stage 24.0 (TID 429). 8470 bytes result sent to driver +26/04/01 06:37:09 INFO TaskSetManager: Starting task 122.0 in stage 24.0 (TID 432) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:09 INFO Executor: Running task 122.0 in stage 24.0 (TID 432) +26/04/01 06:37:09 INFO TaskSetManager: Finished task 119.0 in stage 24.0 (TID 429) in 3458 ms on 10.0.0.133 (executor driver) (119/200) +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:09 INFO Executor: Finished task 118.0 in stage 24.0 (TID 428). 8470 bytes result sent to driver +26/04/01 06:37:09 INFO TaskSetManager: Starting task 123.0 in stage 24.0 (TID 433) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:09 INFO Executor: Running task 123.0 in stage 24.0 (TID 433) +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:09 INFO TaskSetManager: Finished task 118.0 in stage 24.0 (TID 428) in 3468 ms on 10.0.0.133 (executor driver) (120/200) +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:10 INFO Executor: Finished task 120.0 in stage 24.0 (TID 430). 8470 bytes result sent to driver +26/04/01 06:37:10 INFO TaskSetManager: Starting task 124.0 in stage 24.0 (TID 434) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:10 INFO Executor: Running task 124.0 in stage 24.0 (TID 434) +26/04/01 06:37:10 INFO TaskSetManager: Finished task 120.0 in stage 24.0 (TID 430) in 3370 ms on 10.0.0.133 (executor driver) (121/200) +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:10 INFO Executor: Finished task 121.0 in stage 24.0 (TID 431). 8470 bytes result sent to driver +26/04/01 06:37:10 INFO TaskSetManager: Starting task 125.0 in stage 24.0 (TID 435) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:10 INFO Executor: Running task 125.0 in stage 24.0 (TID 435) +26/04/01 06:37:10 INFO TaskSetManager: Finished task 121.0 in stage 24.0 (TID 431) in 3367 ms on 10.0.0.133 (executor driver) (122/200) +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:12 INFO Executor: Finished task 122.0 in stage 24.0 (TID 432). 8470 bytes result sent to driver +26/04/01 06:37:12 INFO TaskSetManager: Starting task 126.0 in stage 24.0 (TID 436) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:12 INFO Executor: Running task 126.0 in stage 24.0 (TID 436) +26/04/01 06:37:12 INFO TaskSetManager: Finished task 122.0 in stage 24.0 (TID 432) in 3460 ms on 10.0.0.133 (executor driver) (123/200) +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:12 INFO Executor: Finished task 123.0 in stage 24.0 (TID 433). 8470 bytes result sent to driver +26/04/01 06:37:12 INFO TaskSetManager: Starting task 127.0 in stage 24.0 (TID 437) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:12 INFO Executor: Running task 127.0 in stage 24.0 (TID 437) +26/04/01 06:37:12 INFO TaskSetManager: Finished task 123.0 in stage 24.0 (TID 433) in 3454 ms on 10.0.0.133 (executor driver) (124/200) +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:14 INFO Executor: Finished task 124.0 in stage 24.0 (TID 434). 8470 bytes result sent to driver +26/04/01 06:37:14 INFO TaskSetManager: Starting task 128.0 in stage 24.0 (TID 438) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:14 INFO TaskSetManager: Finished task 124.0 in stage 24.0 (TID 434) in 3421 ms on 10.0.0.133 (executor driver) (125/200) +26/04/01 06:37:14 INFO Executor: Running task 128.0 in stage 24.0 (TID 438) +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:14 INFO Executor: Finished task 125.0 in stage 24.0 (TID 435). 8470 bytes result sent to driver +26/04/01 06:37:14 INFO TaskSetManager: Starting task 129.0 in stage 24.0 (TID 439) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:14 INFO TaskSetManager: Finished task 125.0 in stage 24.0 (TID 435) in 3397 ms on 10.0.0.133 (executor driver) (126/200) +26/04/01 06:37:14 INFO Executor: Running task 129.0 in stage 24.0 (TID 439) +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:16 INFO Executor: Finished task 126.0 in stage 24.0 (TID 436). 9201 bytes result sent to driver +26/04/01 06:37:16 INFO TaskSetManager: Starting task 130.0 in stage 24.0 (TID 440) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:16 INFO Executor: Running task 130.0 in stage 24.0 (TID 440) +26/04/01 06:37:16 INFO TaskSetManager: Finished task 126.0 in stage 24.0 (TID 436) in 3444 ms on 10.0.0.133 (executor driver) (127/200) +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:16 INFO Executor: Finished task 127.0 in stage 24.0 (TID 437). 9201 bytes result sent to driver +26/04/01 06:37:16 INFO TaskSetManager: Starting task 131.0 in stage 24.0 (TID 441) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:16 INFO Executor: Running task 131.0 in stage 24.0 (TID 441) +26/04/01 06:37:16 INFO TaskSetManager: Finished task 127.0 in stage 24.0 (TID 437) in 3446 ms on 10.0.0.133 (executor driver) (128/200) +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:17 INFO Executor: Finished task 128.0 in stage 24.0 (TID 438). 9201 bytes result sent to driver +26/04/01 06:37:17 INFO TaskSetManager: Starting task 132.0 in stage 24.0 (TID 442) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:17 INFO Executor: Running task 132.0 in stage 24.0 (TID 442) +26/04/01 06:37:17 INFO TaskSetManager: Finished task 128.0 in stage 24.0 (TID 438) in 3411 ms on 10.0.0.133 (executor driver) (129/200) +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:17 INFO Executor: Finished task 129.0 in stage 24.0 (TID 439). 9201 bytes result sent to driver +26/04/01 06:37:17 INFO TaskSetManager: Starting task 133.0 in stage 24.0 (TID 443) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:17 INFO TaskSetManager: Finished task 129.0 in stage 24.0 (TID 439) in 3385 ms on 10.0.0.133 (executor driver) (130/200) +26/04/01 06:37:17 INFO Executor: Running task 133.0 in stage 24.0 (TID 443) +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:19 INFO Executor: Finished task 131.0 in stage 24.0 (TID 441). 8470 bytes result sent to driver +26/04/01 06:37:19 INFO TaskSetManager: Starting task 134.0 in stage 24.0 (TID 444) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:19 INFO TaskSetManager: Finished task 131.0 in stage 24.0 (TID 441) in 3441 ms on 10.0.0.133 (executor driver) (131/200) +26/04/01 06:37:19 INFO Executor: Running task 134.0 in stage 24.0 (TID 444) +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:19 INFO Executor: Finished task 130.0 in stage 24.0 (TID 440). 8470 bytes result sent to driver +26/04/01 06:37:19 INFO TaskSetManager: Starting task 135.0 in stage 24.0 (TID 445) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:19 INFO Executor: Running task 135.0 in stage 24.0 (TID 445) +26/04/01 06:37:19 INFO TaskSetManager: Finished task 130.0 in stage 24.0 (TID 440) in 3452 ms on 10.0.0.133 (executor driver) (132/200) +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:21 INFO Executor: Finished task 132.0 in stage 24.0 (TID 442). 8470 bytes result sent to driver +26/04/01 06:37:21 INFO TaskSetManager: Starting task 136.0 in stage 24.0 (TID 446) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:21 INFO Executor: Running task 136.0 in stage 24.0 (TID 446) +26/04/01 06:37:21 INFO TaskSetManager: Finished task 132.0 in stage 24.0 (TID 442) in 3406 ms on 10.0.0.133 (executor driver) (133/200) +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:21 INFO Executor: Finished task 133.0 in stage 24.0 (TID 443). 8470 bytes result sent to driver +26/04/01 06:37:21 INFO TaskSetManager: Starting task 137.0 in stage 24.0 (TID 447) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:21 INFO Executor: Running task 137.0 in stage 24.0 (TID 447) +26/04/01 06:37:21 INFO TaskSetManager: Finished task 133.0 in stage 24.0 (TID 443) in 3394 ms on 10.0.0.133 (executor driver) (134/200) +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:23 INFO Executor: Finished task 135.0 in stage 24.0 (TID 445). 8470 bytes result sent to driver +26/04/01 06:37:23 INFO TaskSetManager: Starting task 138.0 in stage 24.0 (TID 448) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:23 INFO Executor: Running task 138.0 in stage 24.0 (TID 448) +26/04/01 06:37:23 INFO Executor: Finished task 134.0 in stage 24.0 (TID 444). 8470 bytes result sent to driver +26/04/01 06:37:23 INFO TaskSetManager: Finished task 135.0 in stage 24.0 (TID 445) in 3444 ms on 10.0.0.133 (executor driver) (135/200) +26/04/01 06:37:23 INFO TaskSetManager: Starting task 139.0 in stage 24.0 (TID 449) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:23 INFO Executor: Running task 139.0 in stage 24.0 (TID 449) +26/04/01 06:37:23 INFO TaskSetManager: Finished task 134.0 in stage 24.0 (TID 444) in 3450 ms on 10.0.0.133 (executor driver) (136/200) +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:24 INFO Executor: Finished task 136.0 in stage 24.0 (TID 446). 8470 bytes result sent to driver +26/04/01 06:37:24 INFO TaskSetManager: Starting task 140.0 in stage 24.0 (TID 450) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:24 INFO Executor: Running task 140.0 in stage 24.0 (TID 450) +26/04/01 06:37:24 INFO TaskSetManager: Finished task 136.0 in stage 24.0 (TID 446) in 3415 ms on 10.0.0.133 (executor driver) (137/200) +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:24 INFO Executor: Finished task 137.0 in stage 24.0 (TID 447). 8470 bytes result sent to driver +26/04/01 06:37:24 INFO TaskSetManager: Starting task 141.0 in stage 24.0 (TID 451) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:24 INFO Executor: Running task 141.0 in stage 24.0 (TID 451) +26/04/01 06:37:24 INFO TaskSetManager: Finished task 137.0 in stage 24.0 (TID 447) in 3403 ms on 10.0.0.133 (executor driver) (138/200) +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:26 INFO Executor: Finished task 138.0 in stage 24.0 (TID 448). 9201 bytes result sent to driver +26/04/01 06:37:26 INFO TaskSetManager: Starting task 142.0 in stage 24.0 (TID 452) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:26 INFO Executor: Running task 142.0 in stage 24.0 (TID 452) +26/04/01 06:37:26 INFO TaskSetManager: Finished task 138.0 in stage 24.0 (TID 448) in 3436 ms on 10.0.0.133 (executor driver) (139/200) +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:26 INFO Executor: Finished task 139.0 in stage 24.0 (TID 449). 9201 bytes result sent to driver +26/04/01 06:37:26 INFO TaskSetManager: Starting task 143.0 in stage 24.0 (TID 453) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:26 INFO TaskSetManager: Finished task 139.0 in stage 24.0 (TID 449) in 3444 ms on 10.0.0.133 (executor driver) (140/200) +26/04/01 06:37:26 INFO Executor: Running task 143.0 in stage 24.0 (TID 453) +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:27 INFO Executor: Finished task 140.0 in stage 24.0 (TID 450). 9201 bytes result sent to driver +26/04/01 06:37:27 INFO TaskSetManager: Starting task 144.0 in stage 24.0 (TID 454) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:27 INFO Executor: Running task 144.0 in stage 24.0 (TID 454) +26/04/01 06:37:27 INFO TaskSetManager: Finished task 140.0 in stage 24.0 (TID 450) in 3417 ms on 10.0.0.133 (executor driver) (141/200) +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:27 INFO Executor: Finished task 141.0 in stage 24.0 (TID 451). 9201 bytes result sent to driver +26/04/01 06:37:27 INFO TaskSetManager: Starting task 145.0 in stage 24.0 (TID 455) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:27 INFO TaskSetManager: Finished task 141.0 in stage 24.0 (TID 451) in 3410 ms on 10.0.0.133 (executor driver) (142/200) +26/04/01 06:37:27 INFO Executor: Running task 145.0 in stage 24.0 (TID 455) +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:30 INFO Executor: Finished task 142.0 in stage 24.0 (TID 452). 8470 bytes result sent to driver +26/04/01 06:37:30 INFO TaskSetManager: Starting task 146.0 in stage 24.0 (TID 456) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:30 INFO Executor: Running task 146.0 in stage 24.0 (TID 456) +26/04/01 06:37:30 INFO TaskSetManager: Finished task 142.0 in stage 24.0 (TID 452) in 3452 ms on 10.0.0.133 (executor driver) (143/200) +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:30 INFO Executor: Finished task 143.0 in stage 24.0 (TID 453). 8470 bytes result sent to driver +26/04/01 06:37:30 INFO TaskSetManager: Starting task 147.0 in stage 24.0 (TID 457) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:30 INFO Executor: Running task 147.0 in stage 24.0 (TID 457) +26/04/01 06:37:30 INFO TaskSetManager: Finished task 143.0 in stage 24.0 (TID 453) in 3458 ms on 10.0.0.133 (executor driver) (144/200) +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:31 INFO Executor: Finished task 144.0 in stage 24.0 (TID 454). 8470 bytes result sent to driver +26/04/01 06:37:31 INFO TaskSetManager: Starting task 148.0 in stage 24.0 (TID 458) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:31 INFO Executor: Running task 148.0 in stage 24.0 (TID 458) +26/04/01 06:37:31 INFO TaskSetManager: Finished task 144.0 in stage 24.0 (TID 454) in 3431 ms on 10.0.0.133 (executor driver) (145/200) +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:31 INFO Executor: Finished task 145.0 in stage 24.0 (TID 455). 8470 bytes result sent to driver +26/04/01 06:37:31 INFO TaskSetManager: Starting task 149.0 in stage 24.0 (TID 459) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:31 INFO TaskSetManager: Finished task 145.0 in stage 24.0 (TID 455) in 3420 ms on 10.0.0.133 (executor driver) (146/200) +26/04/01 06:37:31 INFO Executor: Running task 149.0 in stage 24.0 (TID 459) +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:33 INFO Executor: Finished task 147.0 in stage 24.0 (TID 457). 8470 bytes result sent to driver +26/04/01 06:37:33 INFO TaskSetManager: Starting task 150.0 in stage 24.0 (TID 460) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:33 INFO Executor: Running task 150.0 in stage 24.0 (TID 460) +26/04/01 06:37:33 INFO TaskSetManager: Finished task 147.0 in stage 24.0 (TID 457) in 3414 ms on 10.0.0.133 (executor driver) (147/200) +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:33 INFO Executor: Finished task 146.0 in stage 24.0 (TID 456). 8470 bytes result sent to driver +26/04/01 06:37:33 INFO TaskSetManager: Starting task 151.0 in stage 24.0 (TID 461) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:33 INFO Executor: Running task 151.0 in stage 24.0 (TID 461) +26/04/01 06:37:33 INFO TaskSetManager: Finished task 146.0 in stage 24.0 (TID 456) in 3433 ms on 10.0.0.133 (executor driver) (148/200) +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:34 INFO Executor: Finished task 149.0 in stage 24.0 (TID 459). 8470 bytes result sent to driver +26/04/01 06:37:34 INFO TaskSetManager: Starting task 152.0 in stage 24.0 (TID 462) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:34 INFO Executor: Running task 152.0 in stage 24.0 (TID 462) +26/04/01 06:37:34 INFO TaskSetManager: Finished task 149.0 in stage 24.0 (TID 459) in 3430 ms on 10.0.0.133 (executor driver) (149/200) +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:34 INFO Executor: Finished task 148.0 in stage 24.0 (TID 458). 8470 bytes result sent to driver +26/04/01 06:37:34 INFO TaskSetManager: Starting task 153.0 in stage 24.0 (TID 463) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:34 INFO Executor: Running task 153.0 in stage 24.0 (TID 463) +26/04/01 06:37:34 INFO TaskSetManager: Finished task 148.0 in stage 24.0 (TID 458) in 3449 ms on 10.0.0.133 (executor driver) (150/200) +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:36 INFO Executor: Finished task 150.0 in stage 24.0 (TID 460). 9201 bytes result sent to driver +26/04/01 06:37:36 INFO TaskSetManager: Starting task 154.0 in stage 24.0 (TID 464) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:36 INFO Executor: Running task 154.0 in stage 24.0 (TID 464) +26/04/01 06:37:36 INFO TaskSetManager: Finished task 150.0 in stage 24.0 (TID 460) in 3438 ms on 10.0.0.133 (executor driver) (151/200) +26/04/01 06:37:36 INFO Executor: Finished task 151.0 in stage 24.0 (TID 461). 9201 bytes result sent to driver +26/04/01 06:37:36 INFO TaskSetManager: Starting task 155.0 in stage 24.0 (TID 465) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:36 INFO Executor: Running task 155.0 in stage 24.0 (TID 465) +26/04/01 06:37:36 INFO TaskSetManager: Finished task 151.0 in stage 24.0 (TID 461) in 3434 ms on 10.0.0.133 (executor driver) (152/200) +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:38 INFO Executor: Finished task 152.0 in stage 24.0 (TID 462). 9201 bytes result sent to driver +26/04/01 06:37:38 INFO TaskSetManager: Starting task 156.0 in stage 24.0 (TID 466) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:38 INFO Executor: Running task 156.0 in stage 24.0 (TID 466) +26/04/01 06:37:38 INFO TaskSetManager: Finished task 152.0 in stage 24.0 (TID 462) in 3436 ms on 10.0.0.133 (executor driver) (153/200) +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:38 INFO Executor: Finished task 153.0 in stage 24.0 (TID 463). 9201 bytes result sent to driver +26/04/01 06:37:38 INFO TaskSetManager: Starting task 157.0 in stage 24.0 (TID 467) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:38 INFO Executor: Running task 157.0 in stage 24.0 (TID 467) +26/04/01 06:37:38 INFO TaskSetManager: Finished task 153.0 in stage 24.0 (TID 463) in 3434 ms on 10.0.0.133 (executor driver) (154/200) +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:40 INFO Executor: Finished task 155.0 in stage 24.0 (TID 465). 8470 bytes result sent to driver +26/04/01 06:37:40 INFO TaskSetManager: Starting task 158.0 in stage 24.0 (TID 468) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:40 INFO Executor: Running task 158.0 in stage 24.0 (TID 468) +26/04/01 06:37:40 INFO TaskSetManager: Finished task 155.0 in stage 24.0 (TID 465) in 3449 ms on 10.0.0.133 (executor driver) (155/200) +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:40 INFO Executor: Finished task 154.0 in stage 24.0 (TID 464). 8470 bytes result sent to driver +26/04/01 06:37:40 INFO TaskSetManager: Starting task 159.0 in stage 24.0 (TID 469) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:40 INFO TaskSetManager: Finished task 154.0 in stage 24.0 (TID 464) in 3453 ms on 10.0.0.133 (executor driver) (156/200) +26/04/01 06:37:40 INFO Executor: Running task 159.0 in stage 24.0 (TID 469) +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:41 INFO Executor: Finished task 157.0 in stage 24.0 (TID 467). 8470 bytes result sent to driver +26/04/01 06:37:41 INFO TaskSetManager: Starting task 160.0 in stage 24.0 (TID 470) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:41 INFO Executor: Running task 160.0 in stage 24.0 (TID 470) +26/04/01 06:37:41 INFO TaskSetManager: Finished task 157.0 in stage 24.0 (TID 467) in 3431 ms on 10.0.0.133 (executor driver) (157/200) +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:41 INFO Executor: Finished task 156.0 in stage 24.0 (TID 466). 8470 bytes result sent to driver +26/04/01 06:37:41 INFO TaskSetManager: Starting task 161.0 in stage 24.0 (TID 471) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:41 INFO Executor: Running task 161.0 in stage 24.0 (TID 471) +26/04/01 06:37:41 INFO TaskSetManager: Finished task 156.0 in stage 24.0 (TID 466) in 3452 ms on 10.0.0.133 (executor driver) (158/200) +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:43 INFO Executor: Finished task 159.0 in stage 24.0 (TID 469). 8470 bytes result sent to driver +26/04/01 06:37:43 INFO TaskSetManager: Starting task 162.0 in stage 24.0 (TID 472) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:43 INFO Executor: Running task 162.0 in stage 24.0 (TID 472) +26/04/01 06:37:43 INFO TaskSetManager: Finished task 159.0 in stage 24.0 (TID 469) in 3442 ms on 10.0.0.133 (executor driver) (159/200) +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:43 INFO Executor: Finished task 158.0 in stage 24.0 (TID 468). 8470 bytes result sent to driver +26/04/01 06:37:43 INFO TaskSetManager: Starting task 163.0 in stage 24.0 (TID 473) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:43 INFO Executor: Running task 163.0 in stage 24.0 (TID 473) +26/04/01 06:37:43 INFO TaskSetManager: Finished task 158.0 in stage 24.0 (TID 468) in 3453 ms on 10.0.0.133 (executor driver) (160/200) +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:45 INFO Executor: Finished task 160.0 in stage 24.0 (TID 470). 9201 bytes result sent to driver +26/04/01 06:37:45 INFO TaskSetManager: Starting task 164.0 in stage 24.0 (TID 474) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:45 INFO Executor: Running task 164.0 in stage 24.0 (TID 474) +26/04/01 06:37:45 INFO TaskSetManager: Finished task 160.0 in stage 24.0 (TID 470) in 3438 ms on 10.0.0.133 (executor driver) (161/200) +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:45 INFO Executor: Finished task 161.0 in stage 24.0 (TID 471). 9201 bytes result sent to driver +26/04/01 06:37:45 INFO TaskSetManager: Starting task 165.0 in stage 24.0 (TID 475) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:45 INFO TaskSetManager: Finished task 161.0 in stage 24.0 (TID 471) in 3431 ms on 10.0.0.133 (executor driver) (162/200) +26/04/01 06:37:45 INFO Executor: Running task 165.0 in stage 24.0 (TID 475) +26/04/01 06:37:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:47 INFO Executor: Finished task 162.0 in stage 24.0 (TID 472). 9201 bytes result sent to driver +26/04/01 06:37:47 INFO TaskSetManager: Starting task 166.0 in stage 24.0 (TID 476) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:47 INFO Executor: Running task 166.0 in stage 24.0 (TID 476) +26/04/01 06:37:47 INFO TaskSetManager: Finished task 162.0 in stage 24.0 (TID 472) in 3448 ms on 10.0.0.133 (executor driver) (163/200) +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:47 INFO Executor: Finished task 163.0 in stage 24.0 (TID 473). 9201 bytes result sent to driver +26/04/01 06:37:47 INFO TaskSetManager: Starting task 167.0 in stage 24.0 (TID 477) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:47 INFO TaskSetManager: Finished task 163.0 in stage 24.0 (TID 473) in 3450 ms on 10.0.0.133 (executor driver) (164/200) +26/04/01 06:37:47 INFO Executor: Running task 167.0 in stage 24.0 (TID 477) +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:48 INFO Executor: Finished task 165.0 in stage 24.0 (TID 475). 8470 bytes result sent to driver +26/04/01 06:37:48 INFO TaskSetManager: Starting task 168.0 in stage 24.0 (TID 478) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:48 INFO Executor: Running task 168.0 in stage 24.0 (TID 478) +26/04/01 06:37:48 INFO TaskSetManager: Finished task 165.0 in stage 24.0 (TID 475) in 3413 ms on 10.0.0.133 (executor driver) (165/200) +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:48 INFO Executor: Finished task 164.0 in stage 24.0 (TID 474). 8470 bytes result sent to driver +26/04/01 06:37:48 INFO TaskSetManager: Starting task 169.0 in stage 24.0 (TID 479) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:48 INFO Executor: Running task 169.0 in stage 24.0 (TID 479) +26/04/01 06:37:48 INFO TaskSetManager: Finished task 164.0 in stage 24.0 (TID 474) in 3441 ms on 10.0.0.133 (executor driver) (166/200) +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:50 INFO Executor: Finished task 167.0 in stage 24.0 (TID 477). 8470 bytes result sent to driver +26/04/01 06:37:50 INFO Executor: Finished task 166.0 in stage 24.0 (TID 476). 8470 bytes result sent to driver +26/04/01 06:37:50 INFO TaskSetManager: Starting task 170.0 in stage 24.0 (TID 480) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:50 INFO Executor: Running task 170.0 in stage 24.0 (TID 480) +26/04/01 06:37:50 INFO TaskSetManager: Starting task 171.0 in stage 24.0 (TID 481) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:50 INFO TaskSetManager: Finished task 167.0 in stage 24.0 (TID 477) in 3418 ms on 10.0.0.133 (executor driver) (167/200) +26/04/01 06:37:50 INFO Executor: Running task 171.0 in stage 24.0 (TID 481) +26/04/01 06:37:50 INFO TaskSetManager: Finished task 166.0 in stage 24.0 (TID 476) in 3427 ms on 10.0.0.133 (executor driver) (168/200) +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:51 INFO Executor: Finished task 168.0 in stage 24.0 (TID 478). 8470 bytes result sent to driver +26/04/01 06:37:51 INFO TaskSetManager: Starting task 172.0 in stage 24.0 (TID 482) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:51 INFO Executor: Finished task 169.0 in stage 24.0 (TID 479). 8470 bytes result sent to driver +26/04/01 06:37:51 INFO Executor: Running task 172.0 in stage 24.0 (TID 482) +26/04/01 06:37:51 INFO TaskSetManager: Starting task 173.0 in stage 24.0 (TID 483) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:51 INFO Executor: Running task 173.0 in stage 24.0 (TID 483) +26/04/01 06:37:51 INFO TaskSetManager: Finished task 168.0 in stage 24.0 (TID 478) in 3437 ms on 10.0.0.133 (executor driver) (169/200) +26/04/01 06:37:51 INFO TaskSetManager: Finished task 169.0 in stage 24.0 (TID 479) in 3419 ms on 10.0.0.133 (executor driver) (170/200) +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:54 INFO Executor: Finished task 170.0 in stage 24.0 (TID 480). 8470 bytes result sent to driver +26/04/01 06:37:54 INFO Executor: Finished task 171.0 in stage 24.0 (TID 481). 8470 bytes result sent to driver +26/04/01 06:37:54 INFO TaskSetManager: Starting task 174.0 in stage 24.0 (TID 484) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:54 INFO Executor: Running task 174.0 in stage 24.0 (TID 484) +26/04/01 06:37:54 INFO TaskSetManager: Starting task 175.0 in stage 24.0 (TID 485) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:54 INFO TaskSetManager: Finished task 170.0 in stage 24.0 (TID 480) in 3443 ms on 10.0.0.133 (executor driver) (171/200) +26/04/01 06:37:54 INFO Executor: Running task 175.0 in stage 24.0 (TID 485) +26/04/01 06:37:54 INFO TaskSetManager: Finished task 171.0 in stage 24.0 (TID 481) in 3443 ms on 10.0.0.133 (executor driver) (172/200) +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:55 INFO Executor: Finished task 172.0 in stage 24.0 (TID 482). 9201 bytes result sent to driver +26/04/01 06:37:55 INFO TaskSetManager: Starting task 176.0 in stage 24.0 (TID 486) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:55 INFO Executor: Running task 176.0 in stage 24.0 (TID 486) +26/04/01 06:37:55 INFO TaskSetManager: Finished task 172.0 in stage 24.0 (TID 482) in 3443 ms on 10.0.0.133 (executor driver) (173/200) +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:55 INFO Executor: Finished task 173.0 in stage 24.0 (TID 483). 9201 bytes result sent to driver +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:55 INFO TaskSetManager: Starting task 177.0 in stage 24.0 (TID 487) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:55 INFO Executor: Running task 177.0 in stage 24.0 (TID 487) +26/04/01 06:37:55 INFO TaskSetManager: Finished task 173.0 in stage 24.0 (TID 483) in 3450 ms on 10.0.0.133 (executor driver) (174/200) +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:57 INFO Executor: Finished task 175.0 in stage 24.0 (TID 485). 9201 bytes result sent to driver +26/04/01 06:37:57 INFO TaskSetManager: Starting task 178.0 in stage 24.0 (TID 488) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:57 INFO Executor: Running task 178.0 in stage 24.0 (TID 488) +26/04/01 06:37:57 INFO TaskSetManager: Finished task 175.0 in stage 24.0 (TID 485) in 3440 ms on 10.0.0.133 (executor driver) (175/200) +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:57 INFO Executor: Finished task 174.0 in stage 24.0 (TID 484). 9201 bytes result sent to driver +26/04/01 06:37:57 INFO TaskSetManager: Starting task 179.0 in stage 24.0 (TID 489) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:57 INFO TaskSetManager: Finished task 174.0 in stage 24.0 (TID 484) in 3462 ms on 10.0.0.133 (executor driver) (176/200) +26/04/01 06:37:57 INFO Executor: Running task 179.0 in stage 24.0 (TID 489) +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:58 INFO Executor: Finished task 176.0 in stage 24.0 (TID 486). 8470 bytes result sent to driver +26/04/01 06:37:58 INFO TaskSetManager: Starting task 180.0 in stage 24.0 (TID 490) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:58 INFO Executor: Running task 180.0 in stage 24.0 (TID 490) +26/04/01 06:37:58 INFO TaskSetManager: Finished task 176.0 in stage 24.0 (TID 486) in 3463 ms on 10.0.0.133 (executor driver) (177/200) +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:58 INFO Executor: Finished task 177.0 in stage 24.0 (TID 487). 8470 bytes result sent to driver +26/04/01 06:37:58 INFO TaskSetManager: Starting task 181.0 in stage 24.0 (TID 491) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9428 bytes) +26/04/01 06:37:58 INFO Executor: Running task 181.0 in stage 24.0 (TID 491) +26/04/01 06:37:58 INFO TaskSetManager: Finished task 177.0 in stage 24.0 (TID 487) in 3459 ms on 10.0.0.133 (executor driver) (178/200) +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:37:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:37:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:00 INFO Executor: Finished task 178.0 in stage 24.0 (TID 488). 8470 bytes result sent to driver +26/04/01 06:38:00 INFO TaskSetManager: Starting task 182.0 in stage 24.0 (TID 492) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:00 INFO Executor: Running task 182.0 in stage 24.0 (TID 492) +26/04/01 06:38:00 INFO TaskSetManager: Finished task 178.0 in stage 24.0 (TID 488) in 3435 ms on 10.0.0.133 (executor driver) (179/200) +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:00 INFO Executor: Finished task 179.0 in stage 24.0 (TID 489). 8470 bytes result sent to driver +26/04/01 06:38:00 INFO TaskSetManager: Starting task 183.0 in stage 24.0 (TID 493) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:00 INFO Executor: Running task 183.0 in stage 24.0 (TID 493) +26/04/01 06:38:00 INFO TaskSetManager: Finished task 179.0 in stage 24.0 (TID 489) in 3418 ms on 10.0.0.133 (executor driver) (180/200) +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:02 INFO Executor: Finished task 180.0 in stage 24.0 (TID 490). 8470 bytes result sent to driver +26/04/01 06:38:02 INFO TaskSetManager: Starting task 184.0 in stage 24.0 (TID 494) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:02 INFO Executor: Running task 184.0 in stage 24.0 (TID 494) +26/04/01 06:38:02 INFO TaskSetManager: Finished task 180.0 in stage 24.0 (TID 490) in 3467 ms on 10.0.0.133 (executor driver) (181/200) +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:02 INFO Executor: Finished task 181.0 in stage 24.0 (TID 491). 8470 bytes result sent to driver +26/04/01 06:38:02 INFO TaskSetManager: Starting task 185.0 in stage 24.0 (TID 495) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:02 INFO Executor: Running task 185.0 in stage 24.0 (TID 495) +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:02 INFO TaskSetManager: Finished task 181.0 in stage 24.0 (TID 491) in 3470 ms on 10.0.0.133 (executor driver) (182/200) +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:04 INFO Executor: Finished task 183.0 in stage 24.0 (TID 493). 8470 bytes result sent to driver +26/04/01 06:38:04 INFO TaskSetManager: Starting task 186.0 in stage 24.0 (TID 496) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:04 INFO Executor: Running task 186.0 in stage 24.0 (TID 496) +26/04/01 06:38:04 INFO TaskSetManager: Finished task 183.0 in stage 24.0 (TID 493) in 3450 ms on 10.0.0.133 (executor driver) (183/200) +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:04 INFO Executor: Finished task 182.0 in stage 24.0 (TID 492). 8470 bytes result sent to driver +26/04/01 06:38:04 INFO TaskSetManager: Starting task 187.0 in stage 24.0 (TID 497) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:04 INFO Executor: Running task 187.0 in stage 24.0 (TID 497) +26/04/01 06:38:04 INFO TaskSetManager: Finished task 182.0 in stage 24.0 (TID 492) in 3459 ms on 10.0.0.133 (executor driver) (184/200) +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:05 INFO Executor: Finished task 184.0 in stage 24.0 (TID 494). 9201 bytes result sent to driver +26/04/01 06:38:05 INFO TaskSetManager: Starting task 188.0 in stage 24.0 (TID 498) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:05 INFO Executor: Running task 188.0 in stage 24.0 (TID 498) +26/04/01 06:38:05 INFO TaskSetManager: Finished task 184.0 in stage 24.0 (TID 494) in 3459 ms on 10.0.0.133 (executor driver) (185/200) +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:05 INFO Executor: Finished task 185.0 in stage 24.0 (TID 495). 9201 bytes result sent to driver +26/04/01 06:38:05 INFO TaskSetManager: Starting task 189.0 in stage 24.0 (TID 499) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:05 INFO Executor: Running task 189.0 in stage 24.0 (TID 499) +26/04/01 06:38:05 INFO TaskSetManager: Finished task 185.0 in stage 24.0 (TID 495) in 3463 ms on 10.0.0.133 (executor driver) (186/200) +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:07 INFO Executor: Finished task 187.0 in stage 24.0 (TID 497). 9201 bytes result sent to driver +26/04/01 06:38:07 INFO TaskSetManager: Starting task 190.0 in stage 24.0 (TID 500) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:07 INFO Executor: Running task 190.0 in stage 24.0 (TID 500) +26/04/01 06:38:07 INFO TaskSetManager: Finished task 187.0 in stage 24.0 (TID 497) in 3438 ms on 10.0.0.133 (executor driver) (187/200) +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:07 INFO Executor: Finished task 186.0 in stage 24.0 (TID 496). 9201 bytes result sent to driver +26/04/01 06:38:07 INFO TaskSetManager: Starting task 191.0 in stage 24.0 (TID 501) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:07 INFO Executor: Running task 191.0 in stage 24.0 (TID 501) +26/04/01 06:38:07 INFO TaskSetManager: Finished task 186.0 in stage 24.0 (TID 496) in 3456 ms on 10.0.0.133 (executor driver) (188/200) +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:09 INFO Executor: Finished task 188.0 in stage 24.0 (TID 498). 8470 bytes result sent to driver +26/04/01 06:38:09 INFO TaskSetManager: Starting task 192.0 in stage 24.0 (TID 502) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:09 INFO Executor: Running task 192.0 in stage 24.0 (TID 502) +26/04/01 06:38:09 INFO TaskSetManager: Finished task 188.0 in stage 24.0 (TID 498) in 3442 ms on 10.0.0.133 (executor driver) (189/200) +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:09 INFO Executor: Finished task 189.0 in stage 24.0 (TID 499). 8470 bytes result sent to driver +26/04/01 06:38:09 INFO TaskSetManager: Starting task 193.0 in stage 24.0 (TID 503) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:09 INFO TaskSetManager: Finished task 189.0 in stage 24.0 (TID 499) in 3439 ms on 10.0.0.133 (executor driver) (190/200) +26/04/01 06:38:09 INFO Executor: Running task 193.0 in stage 24.0 (TID 503) +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:11 INFO Executor: Finished task 191.0 in stage 24.0 (TID 501). 8470 bytes result sent to driver +26/04/01 06:38:11 INFO TaskSetManager: Starting task 194.0 in stage 24.0 (TID 504) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:11 INFO Executor: Running task 194.0 in stage 24.0 (TID 504) +26/04/01 06:38:11 INFO TaskSetManager: Finished task 191.0 in stage 24.0 (TID 501) in 3422 ms on 10.0.0.133 (executor driver) (191/200) +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:11 INFO Executor: Finished task 190.0 in stage 24.0 (TID 500). 8470 bytes result sent to driver +26/04/01 06:38:11 INFO TaskSetManager: Starting task 195.0 in stage 24.0 (TID 505) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:11 INFO TaskSetManager: Finished task 190.0 in stage 24.0 (TID 500) in 3461 ms on 10.0.0.133 (executor driver) (192/200) +26/04/01 06:38:11 INFO Executor: Running task 195.0 in stage 24.0 (TID 505) +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:12 INFO Executor: Finished task 193.0 in stage 24.0 (TID 503). 8470 bytes result sent to driver +26/04/01 06:38:12 INFO TaskSetManager: Starting task 196.0 in stage 24.0 (TID 506) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:12 INFO Executor: Running task 196.0 in stage 24.0 (TID 506) +26/04/01 06:38:12 INFO TaskSetManager: Finished task 193.0 in stage 24.0 (TID 503) in 3438 ms on 10.0.0.133 (executor driver) (193/200) +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:12 INFO Executor: Finished task 192.0 in stage 24.0 (TID 502). 8470 bytes result sent to driver +26/04/01 06:38:12 INFO TaskSetManager: Starting task 197.0 in stage 24.0 (TID 507) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:12 INFO Executor: Running task 197.0 in stage 24.0 (TID 507) +26/04/01 06:38:12 INFO TaskSetManager: Finished task 192.0 in stage 24.0 (TID 502) in 3461 ms on 10.0.0.133 (executor driver) (194/200) +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:14 INFO Executor: Finished task 195.0 in stage 24.0 (TID 505). 8470 bytes result sent to driver +26/04/01 06:38:14 INFO Executor: Finished task 194.0 in stage 24.0 (TID 504). 8470 bytes result sent to driver +26/04/01 06:38:14 INFO TaskSetManager: Starting task 198.0 in stage 24.0 (TID 508) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:14 INFO Executor: Running task 198.0 in stage 24.0 (TID 508) +26/04/01 06:38:14 INFO TaskSetManager: Starting task 199.0 in stage 24.0 (TID 509) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9428 bytes) +26/04/01 06:38:14 INFO TaskSetManager: Finished task 195.0 in stage 24.0 (TID 505) in 3407 ms on 10.0.0.133 (executor driver) (195/200) +26/04/01 06:38:14 INFO Executor: Running task 199.0 in stage 24.0 (TID 509) +26/04/01 06:38:14 INFO TaskSetManager: Finished task 194.0 in stage 24.0 (TID 504) in 3435 ms on 10.0.0.133 (executor driver) (196/200) +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:16 INFO Executor: Finished task 197.0 in stage 24.0 (TID 507). 9201 bytes result sent to driver +26/04/01 06:38:16 INFO TaskSetManager: Finished task 197.0 in stage 24.0 (TID 507) in 3426 ms on 10.0.0.133 (executor driver) (197/200) +26/04/01 06:38:16 INFO Executor: Finished task 196.0 in stage 24.0 (TID 506). 9201 bytes result sent to driver +26/04/01 06:38:16 INFO TaskSetManager: Finished task 196.0 in stage 24.0 (TID 506) in 3445 ms on 10.0.0.133 (executor driver) (198/200) +26/04/01 06:38:18 INFO Executor: Finished task 198.0 in stage 24.0 (TID 508). 9201 bytes result sent to driver +26/04/01 06:38:18 INFO TaskSetManager: Finished task 198.0 in stage 24.0 (TID 508) in 3303 ms on 10.0.0.133 (executor driver) (199/200) +26/04/01 06:38:18 INFO Executor: Finished task 199.0 in stage 24.0 (TID 509). 9201 bytes result sent to driver +26/04/01 06:38:18 INFO TaskSetManager: Finished task 199.0 in stage 24.0 (TID 509) in 3317 ms on 10.0.0.133 (executor driver) (200/200) +26/04/01 06:38:18 INFO TaskSchedulerImpl: Removed TaskSet 24.0, whose tasks have all completed, from pool +26/04/01 06:38:18 INFO DAGScheduler: ShuffleMapStage 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 172.034 s +26/04/01 06:38:18 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:38:18 INFO DAGScheduler: running: Set() +26/04/01 06:38:18 INFO DAGScheduler: waiting: Set() +26/04/01 06:38:18 INFO DAGScheduler: failed: Set() +26/04/01 06:38:18 INFO ShufflePartitionsUtil: For shuffle(6, 4), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 06:38:18 INFO DAGScheduler: Registering RDD 54 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 7 +26/04/01 06:38:18 INFO DAGScheduler: Got map stage job 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 25 output partitions +26/04/01 06:38:18 INFO DAGScheduler: Final stage: ShuffleMapStage 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:38:18 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 30, ShuffleMapStage 29) +26/04/01 06:38:18 INFO DAGScheduler: Missing parents: List() +26/04/01 06:38:18 INFO DAGScheduler: Submitting ShuffleMapStage 31 (MapPartitionsRDD[54] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:38:18 INFO MemoryStore: Block broadcast_27 stored as values in memory (estimated size 39.3 KiB, free 12.6 GiB) +26/04/01 06:38:18 INFO MemoryStore: Block broadcast_27_piece0 stored as bytes in memory (estimated size 12.3 KiB, free 12.6 GiB) +26/04/01 06:38:18 INFO BlockManagerInfo: Added broadcast_27_piece0 in memory on 10.0.0.133:58442 (size: 12.3 KiB, free: 12.6 GiB) +26/04/01 06:38:18 INFO SparkContext: Created broadcast 27 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:38:18 INFO DAGScheduler: Submitting 25 missing tasks from ShuffleMapStage 31 (MapPartitionsRDD[54] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 06:38:18 INFO TaskSchedulerImpl: Adding task set 31.0 with 25 tasks resource profile 0 +26/04/01 06:38:18 INFO TaskSetManager: Starting task 0.0 in stage 31.0 (TID 510) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:18 INFO TaskSetManager: Starting task 1.0 in stage 31.0 (TID 511) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:18 INFO TaskSetManager: Starting task 2.0 in stage 31.0 (TID 512) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:18 INFO TaskSetManager: Starting task 3.0 in stage 31.0 (TID 513) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:18 INFO Executor: Running task 1.0 in stage 31.0 (TID 511) +26/04/01 06:38:18 INFO Executor: Running task 3.0 in stage 31.0 (TID 513) +26/04/01 06:38:18 INFO Executor: Running task 2.0 in stage 31.0 (TID 512) +26/04/01 06:38:18 INFO Executor: Running task 0.0 in stage 31.0 (TID 510) +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (320.9 KiB) non-empty blocks including 4 (320.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (320.9 KiB) non-empty blocks including 4 (320.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:26 INFO Executor: Finished task 3.0 in stage 31.0 (TID 513). 11071 bytes result sent to driver +26/04/01 06:38:26 INFO TaskSetManager: Starting task 4.0 in stage 31.0 (TID 514) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:26 INFO Executor: Running task 4.0 in stage 31.0 (TID 514) +26/04/01 06:38:26 INFO TaskSetManager: Finished task 3.0 in stage 31.0 (TID 513) in 8854 ms on 10.0.0.133 (executor driver) (1/25) +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO Executor: Finished task 0.0 in stage 31.0 (TID 510). 11071 bytes result sent to driver +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO TaskSetManager: Starting task 5.0 in stage 31.0 (TID 515) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:26 INFO Executor: Running task 5.0 in stage 31.0 (TID 515) +26/04/01 06:38:26 INFO TaskSetManager: Finished task 0.0 in stage 31.0 (TID 510) in 8862 ms on 10.0.0.133 (executor driver) (2/25) +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:26 INFO Executor: Finished task 2.0 in stage 31.0 (TID 512). 11071 bytes result sent to driver +26/04/01 06:38:26 INFO TaskSetManager: Starting task 6.0 in stage 31.0 (TID 516) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:26 INFO Executor: Running task 6.0 in stage 31.0 (TID 516) +26/04/01 06:38:26 INFO TaskSetManager: Finished task 2.0 in stage 31.0 (TID 512) in 8872 ms on 10.0.0.133 (executor driver) (3/25) +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO Executor: Finished task 1.0 in stage 31.0 (TID 511). 11071 bytes result sent to driver +26/04/01 06:38:26 INFO TaskSetManager: Starting task 7.0 in stage 31.0 (TID 517) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:26 INFO Executor: Running task 7.0 in stage 31.0 (TID 517) +26/04/01 06:38:26 INFO TaskSetManager: Finished task 1.0 in stage 31.0 (TID 511) in 8874 ms on 10.0.0.133 (executor driver) (4/25) +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:35 INFO Executor: Finished task 5.0 in stage 31.0 (TID 515). 11071 bytes result sent to driver +26/04/01 06:38:35 INFO TaskSetManager: Starting task 8.0 in stage 31.0 (TID 518) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:35 INFO Executor: Running task 8.0 in stage 31.0 (TID 518) +26/04/01 06:38:35 INFO TaskSetManager: Finished task 5.0 in stage 31.0 (TID 515) in 8743 ms on 10.0.0.133 (executor driver) (5/25) +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:35 INFO Executor: Finished task 4.0 in stage 31.0 (TID 514). 11071 bytes result sent to driver +26/04/01 06:38:35 INFO TaskSetManager: Starting task 9.0 in stage 31.0 (TID 519) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:35 INFO TaskSetManager: Finished task 4.0 in stage 31.0 (TID 514) in 8776 ms on 10.0.0.133 (executor driver) (6/25) +26/04/01 06:38:35 INFO Executor: Running task 9.0 in stage 31.0 (TID 519) +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (317.2 KiB) non-empty blocks including 4 (317.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (317.2 KiB) non-empty blocks including 4 (317.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:35 INFO Executor: Finished task 6.0 in stage 31.0 (TID 516). 11071 bytes result sent to driver +26/04/01 06:38:35 INFO TaskSetManager: Starting task 10.0 in stage 31.0 (TID 520) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:35 INFO TaskSetManager: Finished task 6.0 in stage 31.0 (TID 516) in 8805 ms on 10.0.0.133 (executor driver) (7/25) +26/04/01 06:38:35 INFO Executor: Running task 10.0 in stage 31.0 (TID 520) +26/04/01 06:38:35 INFO Executor: Finished task 7.0 in stage 31.0 (TID 517). 11071 bytes result sent to driver +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO TaskSetManager: Starting task 11.0 in stage 31.0 (TID 521) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:35 INFO Executor: Running task 11.0 in stage 31.0 (TID 521) +26/04/01 06:38:35 INFO TaskSetManager: Finished task 7.0 in stage 31.0 (TID 517) in 8805 ms on 10.0.0.133 (executor driver) (8/25) +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:44 INFO Executor: Finished task 8.0 in stage 31.0 (TID 518). 10340 bytes result sent to driver +26/04/01 06:38:44 INFO TaskSetManager: Starting task 12.0 in stage 31.0 (TID 522) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:44 INFO Executor: Running task 12.0 in stage 31.0 (TID 522) +26/04/01 06:38:44 INFO TaskSetManager: Finished task 8.0 in stage 31.0 (TID 518) in 8729 ms on 10.0.0.133 (executor driver) (9/25) +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:44 INFO Executor: Finished task 9.0 in stage 31.0 (TID 519). 10340 bytes result sent to driver +26/04/01 06:38:44 INFO TaskSetManager: Starting task 13.0 in stage 31.0 (TID 523) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:44 INFO TaskSetManager: Finished task 9.0 in stage 31.0 (TID 519) in 8738 ms on 10.0.0.133 (executor driver) (10/25) +26/04/01 06:38:44 INFO Executor: Running task 13.0 in stage 31.0 (TID 523) +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (322.7 KiB) non-empty blocks including 4 (322.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (322.7 KiB) non-empty blocks including 4 (322.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:44 INFO Executor: Finished task 10.0 in stage 31.0 (TID 520). 10340 bytes result sent to driver +26/04/01 06:38:44 INFO TaskSetManager: Starting task 14.0 in stage 31.0 (TID 524) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:44 INFO Executor: Running task 14.0 in stage 31.0 (TID 524) +26/04/01 06:38:44 INFO TaskSetManager: Finished task 10.0 in stage 31.0 (TID 520) in 8775 ms on 10.0.0.133 (executor driver) (11/25) +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO Executor: Finished task 11.0 in stage 31.0 (TID 521). 10340 bytes result sent to driver +26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:44 INFO TaskSetManager: Starting task 15.0 in stage 31.0 (TID 525) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:44 INFO TaskSetManager: Finished task 11.0 in stage 31.0 (TID 521) in 8783 ms on 10.0.0.133 (executor driver) (12/25) +26/04/01 06:38:44 INFO Executor: Running task 15.0 in stage 31.0 (TID 525) +26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:53 INFO Executor: Finished task 12.0 in stage 31.0 (TID 522). 11071 bytes result sent to driver +26/04/01 06:38:53 INFO TaskSetManager: Starting task 16.0 in stage 31.0 (TID 526) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:53 INFO TaskSetManager: Finished task 12.0 in stage 31.0 (TID 522) in 8714 ms on 10.0.0.133 (executor driver) (13/25) +26/04/01 06:38:53 INFO Executor: Running task 16.0 in stage 31.0 (TID 526) +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:53 INFO Executor: Finished task 13.0 in stage 31.0 (TID 523). 11071 bytes result sent to driver +26/04/01 06:38:53 INFO TaskSetManager: Starting task 17.0 in stage 31.0 (TID 527) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:53 INFO Executor: Running task 17.0 in stage 31.0 (TID 527) +26/04/01 06:38:53 INFO TaskSetManager: Finished task 13.0 in stage 31.0 (TID 523) in 8701 ms on 10.0.0.133 (executor driver) (14/25) +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:53 INFO Executor: Finished task 14.0 in stage 31.0 (TID 524). 11071 bytes result sent to driver +26/04/01 06:38:53 INFO TaskSetManager: Starting task 18.0 in stage 31.0 (TID 528) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:53 INFO Executor: Running task 18.0 in stage 31.0 (TID 528) +26/04/01 06:38:53 INFO TaskSetManager: Finished task 14.0 in stage 31.0 (TID 524) in 8668 ms on 10.0.0.133 (executor driver) (15/25) +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:53 INFO Executor: Finished task 15.0 in stage 31.0 (TID 525). 11071 bytes result sent to driver +26/04/01 06:38:53 INFO TaskSetManager: Starting task 19.0 in stage 31.0 (TID 529) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9719 bytes) +26/04/01 06:38:53 INFO Executor: Running task 19.0 in stage 31.0 (TID 529) +26/04/01 06:38:53 INFO TaskSetManager: Finished task 15.0 in stage 31.0 (TID 525) in 8672 ms on 10.0.0.133 (executor driver) (16/25) +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:39:01 INFO Executor: Finished task 16.0 in stage 31.0 (TID 526). 11071 bytes result sent to driver +26/04/01 06:39:01 INFO TaskSetManager: Starting task 20.0 in stage 31.0 (TID 530) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9719 bytes) +26/04/01 06:39:01 INFO Executor: Running task 20.0 in stage 31.0 (TID 530) +26/04/01 06:39:01 INFO TaskSetManager: Finished task 16.0 in stage 31.0 (TID 526) in 8795 ms on 10.0.0.133 (executor driver) (17/25) +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:39:01 INFO Executor: Finished task 18.0 in stage 31.0 (TID 528). 11071 bytes result sent to driver +26/04/01 06:39:01 INFO TaskSetManager: Starting task 21.0 in stage 31.0 (TID 531) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9719 bytes) +26/04/01 06:39:01 INFO Executor: Running task 21.0 in stage 31.0 (TID 531) +26/04/01 06:39:01 INFO TaskSetManager: Finished task 18.0 in stage 31.0 (TID 528) in 8757 ms on 10.0.0.133 (executor driver) (18/25) +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:39:01 INFO Executor: Finished task 19.0 in stage 31.0 (TID 529). 11071 bytes result sent to driver +26/04/01 06:39:01 INFO TaskSetManager: Starting task 22.0 in stage 31.0 (TID 532) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9719 bytes) +26/04/01 06:39:01 INFO Executor: Running task 22.0 in stage 31.0 (TID 532) +26/04/01 06:39:01 INFO TaskSetManager: Finished task 19.0 in stage 31.0 (TID 529) in 8754 ms on 10.0.0.133 (executor driver) (19/25) +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:39:01 INFO Executor: Finished task 17.0 in stage 31.0 (TID 527). 11071 bytes result sent to driver +26/04/01 06:39:01 INFO TaskSetManager: Starting task 23.0 in stage 31.0 (TID 533) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9719 bytes) +26/04/01 06:39:01 INFO Executor: Running task 23.0 in stage 31.0 (TID 533) +26/04/01 06:39:01 INFO TaskSetManager: Finished task 17.0 in stage 31.0 (TID 527) in 8831 ms on 10.0.0.133 (executor driver) (20/25) +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (322.8 KiB) non-empty blocks including 4 (322.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (322.8 KiB) non-empty blocks including 4 (322.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:39:10 INFO Executor: Finished task 21.0 in stage 31.0 (TID 531). 11071 bytes result sent to driver +26/04/01 06:39:10 INFO TaskSetManager: Starting task 24.0 in stage 31.0 (TID 534) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9719 bytes) +26/04/01 06:39:10 INFO Executor: Running task 24.0 in stage 31.0 (TID 534) +26/04/01 06:39:10 INFO TaskSetManager: Finished task 21.0 in stage 31.0 (TID 531) in 8697 ms on 10.0.0.133 (executor driver) (21/25) +26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:39:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:39:10 INFO Executor: Finished task 20.0 in stage 31.0 (TID 530). 11071 bytes result sent to driver +26/04/01 06:39:10 INFO TaskSetManager: Finished task 20.0 in stage 31.0 (TID 530) in 8754 ms on 10.0.0.133 (executor driver) (22/25) +26/04/01 06:39:10 INFO Executor: Finished task 22.0 in stage 31.0 (TID 532). 11071 bytes result sent to driver +26/04/01 06:39:10 INFO TaskSetManager: Finished task 22.0 in stage 31.0 (TID 532) in 8748 ms on 10.0.0.133 (executor driver) (23/25) +26/04/01 06:39:10 INFO Executor: Finished task 23.0 in stage 31.0 (TID 533). 11071 bytes result sent to driver +26/04/01 06:39:10 INFO TaskSetManager: Finished task 23.0 in stage 31.0 (TID 533) in 8773 ms on 10.0.0.133 (executor driver) (24/25) +26/04/01 06:39:16 INFO Executor: Finished task 24.0 in stage 31.0 (TID 534). 11071 bytes result sent to driver +26/04/01 06:39:16 INFO TaskSetManager: Finished task 24.0 in stage 31.0 (TID 534) in 6282 ms on 10.0.0.133 (executor driver) (25/25) +26/04/01 06:39:16 INFO TaskSchedulerImpl: Removed TaskSet 31.0, whose tasks have all completed, from pool +26/04/01 06:39:16 INFO DAGScheduler: ShuffleMapStage 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 58.857 s +26/04/01 06:39:16 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:39:16 INFO DAGScheduler: running: Set() +26/04/01 06:39:16 INFO DAGScheduler: waiting: Set() +26/04/01 06:39:16 INFO DAGScheduler: failed: Set() +26/04/01 06:39:16 INFO ShufflePartitionsUtil: For shuffle(7), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 06:39:16 INFO CodeGenerator: Code generated in 62.440042 ms +26/04/01 06:39:17 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:39:17 INFO DAGScheduler: Got job 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:39:17 INFO DAGScheduler: Final stage: ResultStage 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:39:17 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 38) +26/04/01 06:39:17 INFO DAGScheduler: Missing parents: List() +26/04/01 06:39:17 INFO DAGScheduler: Submitting ResultStage 39 (MapPartitionsRDD[59] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:39:17 INFO MemoryStore: Block broadcast_28 stored as values in memory (estimated size 22.7 KiB, free 12.6 GiB) +26/04/01 06:39:17 INFO MemoryStore: Block broadcast_28_piece0 stored as bytes in memory (estimated size 8.3 KiB, free 12.6 GiB) +26/04/01 06:39:17 INFO BlockManagerInfo: Added broadcast_28_piece0 in memory on 10.0.0.133:58442 (size: 8.3 KiB, free: 12.6 GiB) +26/04/01 06:39:17 INFO SparkContext: Created broadcast 28 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:39:17 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 39 (MapPartitionsRDD[59] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:39:17 INFO TaskSchedulerImpl: Adding task set 39.0 with 1 tasks resource profile 0 +26/04/01 06:39:17 INFO TaskSetManager: Starting task 0.0 in stage 39.0 (TID 535) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) +26/04/01 06:39:17 INFO Executor: Running task 0.0 in stage 39.0 (TID 535) +26/04/01 06:39:17 INFO ShuffleBlockFetcherIterator: Getting 25 (54.3 KiB) non-empty blocks including 25 (54.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:39:17 INFO CodeGenerator: Code generated in 8.900458 ms +26/04/01 06:39:17 INFO Executor: Finished task 0.0 in stage 39.0 (TID 535). 5177 bytes result sent to driver +26/04/01 06:39:17 INFO TaskSetManager: Finished task 0.0 in stage 39.0 (TID 535) in 55 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:39:17 INFO TaskSchedulerImpl: Removed TaskSet 39.0, whose tasks have all completed, from pool +26/04/01 06:39:17 INFO DAGScheduler: ResultStage 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.057 s +26/04/01 06:39:17 INFO DAGScheduler: Job 19 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:39:17 INFO TaskSchedulerImpl: Killing all running tasks in stage 39: Stage finished +26/04/01 06:39:17 INFO DAGScheduler: Job 19 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.058802 s +26/04/01 06:39:17 INFO DAGScheduler: Registering RDD 60 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 8 +26/04/01 06:39:17 INFO DAGScheduler: Got map stage job 20 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:39:17 INFO DAGScheduler: Final stage: ShuffleMapStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:39:17 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 46) +26/04/01 06:39:17 INFO DAGScheduler: Missing parents: List() +26/04/01 06:39:17 INFO DAGScheduler: Submitting ShuffleMapStage 47 (MapPartitionsRDD[60] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:39:17 INFO MemoryStore: Block broadcast_29 stored as values in memory (estimated size 25.0 KiB, free 12.6 GiB) +26/04/01 06:39:17 INFO MemoryStore: Block broadcast_29_piece0 stored as bytes in memory (estimated size 9.3 KiB, free 12.6 GiB) +26/04/01 06:39:17 INFO BlockManagerInfo: Added broadcast_29_piece0 in memory on 10.0.0.133:58442 (size: 9.3 KiB, free: 12.6 GiB) +26/04/01 06:39:17 INFO SparkContext: Created broadcast 29 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:39:17 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 47 (MapPartitionsRDD[60] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:39:17 INFO TaskSchedulerImpl: Adding task set 47.0 with 1 tasks resource profile 0 +26/04/01 06:39:17 INFO TaskSetManager: Starting task 0.0 in stage 47.0 (TID 536) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9404 bytes) +26/04/01 06:39:17 INFO Executor: Running task 0.0 in stage 47.0 (TID 536) +26/04/01 06:39:17 INFO CodeGenerator: Code generated in 3.410292 ms +26/04/01 06:39:17 INFO ShuffleBlockFetcherIterator: Getting 25 (54.3 KiB) non-empty blocks including 25 (54.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:39:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:39:17 INFO Executor: Finished task 0.0 in stage 47.0 (TID 536). 6566 bytes result sent to driver +26/04/01 06:39:17 INFO TaskSetManager: Finished task 0.0 in stage 47.0 (TID 536) in 49 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:39:17 INFO TaskSchedulerImpl: Removed TaskSet 47.0, whose tasks have all completed, from pool +26/04/01 06:39:17 INFO DAGScheduler: ShuffleMapStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.052 s +26/04/01 06:39:17 INFO DAGScheduler: looking for newly runnable stages +26/04/01 06:39:17 INFO DAGScheduler: running: Set() +26/04/01 06:39:17 INFO DAGScheduler: waiting: Set() +26/04/01 06:39:17 INFO DAGScheduler: failed: Set() +26/04/01 06:39:17 INFO ShufflePartitionsUtil: For shuffle(8), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 06:39:17 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 06:39:17 INFO DAGScheduler: Got job 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 06:39:17 INFO DAGScheduler: Final stage: ResultStage 56 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 06:39:17 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 55) +26/04/01 06:39:17 INFO DAGScheduler: Missing parents: List() +26/04/01 06:39:17 INFO DAGScheduler: Submitting ResultStage 56 (MapPartitionsRDD[64] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 06:39:17 INFO MemoryStore: Block broadcast_30 stored as values in memory (estimated size 18.6 KiB, free 12.6 GiB) +26/04/01 06:39:17 INFO MemoryStore: Block broadcast_30_piece0 stored as bytes in memory (estimated size 8.8 KiB, free 12.6 GiB) +26/04/01 06:39:17 INFO BlockManagerInfo: Added broadcast_30_piece0 in memory on 10.0.0.133:58442 (size: 8.8 KiB, free: 12.6 GiB) +26/04/01 06:39:17 INFO SparkContext: Created broadcast 30 from broadcast at DAGScheduler.scala:1611 +26/04/01 06:39:17 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 56 (MapPartitionsRDD[64] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 06:39:17 INFO TaskSchedulerImpl: Adding task set 56.0 with 1 tasks resource profile 0 +26/04/01 06:39:17 INFO TaskSetManager: Starting task 0.0 in stage 56.0 (TID 537) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) +26/04/01 06:39:17 INFO Executor: Running task 0.0 in stage 56.0 (TID 537) +26/04/01 06:39:17 INFO ShuffleBlockFetcherIterator: Getting 1 (1620.0 B) non-empty blocks including 1 (1620.0 B) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 06:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 06:39:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB +26/04/01 06:39:17 INFO Executor: Finished task 0.0 in stage 56.0 (TID 537). 5280 bytes result sent to driver +26/04/01 06:39:17 INFO TaskSetManager: Finished task 0.0 in stage 56.0 (TID 537) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 06:39:17 INFO TaskSchedulerImpl: Removed TaskSet 56.0, whose tasks have all completed, from pool +26/04/01 06:39:17 INFO DAGScheduler: ResultStage 56 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.006 s +26/04/01 06:39:17 INFO DAGScheduler: Job 21 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 06:39:17 INFO TaskSchedulerImpl: Killing all running tasks in stage 56: Stage finished +26/04/01 06:39:17 INFO DAGScheduler: Job 21 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.007150 s +26/04/01 06:39:17 INFO SparkContext: SparkContext is stopping with exitCode 0. +26/04/01 06:39:17 INFO CometDriverPlugin: CometDriverPlugin shutdown +26/04/01 06:39:17 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! +26/04/01 06:39:17 INFO MemoryStore: MemoryStore cleared +26/04/01 06:39:17 INFO BlockManager: BlockManager stopped +26/04/01 06:39:17 INFO BlockManagerMaster: BlockManagerMaster stopped +26/04/01 06:39:17 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! +26/04/01 06:39:17 INFO SparkContext: Successfully stopped SparkContext +26/04/01 06:39:17 INFO ShutdownHookManager: Shutdown hook called +26/04/01 06:39:17 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-3bd87ee3-3f8e-4785-87c2-97740eb236dc/pyspark-d1a60d25-754f-493a-95c3-482062c25e17 +26/04/01 06:39:17 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-3bd87ee3-3f8e-4785-87c2-97740eb236dc +26/04/01 06:39:17 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-acac7b05-334c-4692-bbb8-80e26ae2634f + 487.97 real 2095.14 user 21.36 sys + 5704859648 maximum resident set size + 0 average shared memory size + 0 average unshared data size + 0 average unshared stack size + 373734 page reclaims + 36 page faults + 0 swaps + 0 block input operations + 0 block output operations + 1274 messages sent + 2360 messages received + 18 signals received + 38220 voluntary context switches + 761034 involuntary context switches + 33268443983164 instructions retired + 7705612041471 cycles elapsed + 2460538496 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.log b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.log new file mode 100644 index 0000000000..b4ba2b8a65 --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.log @@ -0,0 +1,281 @@ +Registering table customer from /opt/tpch/sf100/customer +Registering table lineitem from /opt/tpch/sf100/lineitem +Registering table nation from /opt/tpch/sf100/nation +Registering table orders from /opt/tpch/sf100/orders +Registering table part from /opt/tpch/sf100/part +Registering table partsupp from /opt/tpch/sf100/partsupp +Registering table region from /opt/tpch/sf100/region +Registering table supplier from /opt/tpch/sf100/supplier + +============================================================ +Starting iteration 1 of 1 +============================================================ + +Running query 9 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q9.sql +Executing: -- CometBench-H query 9 derived from TPC-H query 9 under the terms of the TPC Fair Use Policy. +-- TP... +== Physical Plan == +AdaptiveSparkPlan (46) ++- CometSort (45) + +- CometExchange (44) + +- CometHashAggregate (43) + +- CometExchange (42) + +- CometHashAggregate (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (35) + : +- CometSortMergeJoin (34) + : :- CometSort (29) + : : +- CometExchange (28) + : : +- CometProject (27) + : : +- CometSortMergeJoin (26) + : : :- CometSort (21) + : : : +- CometExchange (20) + : : : +- CometProject (19) + : : : +- CometSortMergeJoin (18) + : : : :- CometSort (13) + : : : : +- CometExchange (12) + : : : : +- CometProject (11) + : : : : +- CometSortMergeJoin (10) + : : : : :- CometSort (5) + : : : : : +- CometExchange (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometNativeScan parquet (1) + : : : : +- CometSort (9) + : : : : +- CometExchange (8) + : : : : +- CometFilter (7) + : : : : +- CometNativeScan parquet (6) + : : : +- CometSort (17) + : : : +- CometExchange (16) + : : : +- CometFilter (15) + : : : +- CometNativeScan parquet (14) + : : +- CometSort (25) + : : +- CometExchange (24) + : : +- CometFilter (23) + : : +- CometNativeScan parquet (22) + : +- CometSort (33) + : +- CometExchange (32) + : +- CometFilter (31) + : +- CometNativeScan parquet (30) + +- CometBroadcastExchange (38) + +- CometFilter (37) + +- CometNativeScan parquet (36) + + +(1) CometNativeScan parquet +Output [2]: [p_partkey#74L, p_name#75] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/part] +PushedFilters: [IsNotNull(p_name), StringContains(p_name,moccasin), IsNotNull(p_partkey)] +ReadSchema: struct + +(2) CometFilter +Input [2]: [p_partkey#74L, p_name#75] +Condition : ((isnotnull(p_name#75) AND Contains(p_name#75, moccasin)) AND isnotnull(p_partkey#74L)) + +(3) CometProject +Input [2]: [p_partkey#74L, p_name#75] +Arguments: [p_partkey#74L], [p_partkey#74L] + +(4) CometExchange +Input [1]: [p_partkey#74L] +Arguments: hashpartitioning(p_partkey#74L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=376] + +(5) CometSort +Input [1]: [p_partkey#74L] +Arguments: [p_partkey#74L], [p_partkey#74L ASC NULLS FIRST] + +(6) CometNativeScan parquet +Output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] +PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] +ReadSchema: struct + +(7) CometFilter +Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Condition : ((isnotnull(l_partkey#17L) AND isnotnull(l_suppkey#18L)) AND isnotnull(l_orderkey#16L)) + +(8) CometExchange +Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: hashpartitioning(l_partkey#17L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=380] + +(9) CometSort +Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22], [l_partkey#17L ASC NULLS FIRST] + +(10) CometSortMergeJoin +Left output [1]: [p_partkey#74L] +Right output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: [p_partkey#74L], [l_partkey#17L], Inner + +(11) CometProject +Input [7]: [p_partkey#74L, l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22], [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] + +(12) CometExchange +Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: hashpartitioning(l_suppkey#18L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=384] + +(13) CometSort +Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22], [l_suppkey#18L ASC NULLS FIRST] + +(14) CometNativeScan parquet +Output [2]: [s_suppkey#108L, s_nationkey#111L] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/supplier] +PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] +ReadSchema: struct + +(15) CometFilter +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Condition : (isnotnull(s_suppkey#108L) AND isnotnull(s_nationkey#111L)) + +(16) CometExchange +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: hashpartitioning(s_suppkey#108L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=388] + +(17) CometSort +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: [s_suppkey#108L, s_nationkey#111L], [s_suppkey#108L ASC NULLS FIRST] + +(18) CometSortMergeJoin +Left output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Right output [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: [l_suppkey#18L], [s_suppkey#108L], Inner + +(19) CometProject +Input [8]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_suppkey#108L, s_nationkey#111L] +Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L], [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] + +(20) CometExchange +Input [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] +Arguments: hashpartitioning(l_suppkey#18L, l_partkey#17L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=392] + +(21) CometSort +Input [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] +Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L], [l_suppkey#18L ASC NULLS FIRST, l_partkey#17L ASC NULLS FIRST] + +(22) CometNativeScan parquet +Output [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/partsupp] +PushedFilters: [IsNotNull(ps_suppkey), IsNotNull(ps_partkey)] +ReadSchema: struct + +(23) CometFilter +Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Condition : (isnotnull(ps_suppkey#93L) AND isnotnull(ps_partkey#92L)) + +(24) CometExchange +Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Arguments: hashpartitioning(ps_suppkey#93L, ps_partkey#92L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=396] + +(25) CometSort +Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Arguments: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95], [ps_suppkey#93L ASC NULLS FIRST, ps_partkey#92L ASC NULLS FIRST] + +(26) CometSortMergeJoin +Left output [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] +Right output [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Arguments: [l_suppkey#18L, l_partkey#17L], [ps_suppkey#93L, ps_partkey#92L], Inner + +(27) CometProject +Input [10]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Arguments: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95], [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] + +(28) CometExchange +Input [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] +Arguments: hashpartitioning(l_orderkey#16L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=400] + +(29) CometSort +Input [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] +Arguments: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95], [l_orderkey#16L ASC NULLS FIRST] + +(30) CometNativeScan parquet +Output [2]: [o_orderkey#56L, o_orderdate#60] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/orders] +PushedFilters: [IsNotNull(o_orderkey)] +ReadSchema: struct + +(31) CometFilter +Input [2]: [o_orderkey#56L, o_orderdate#60] +Condition : isnotnull(o_orderkey#56L) + +(32) CometExchange +Input [2]: [o_orderkey#56L, o_orderdate#60] +Arguments: hashpartitioning(o_orderkey#56L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=404] + +(33) CometSort +Input [2]: [o_orderkey#56L, o_orderdate#60] +Arguments: [o_orderkey#56L, o_orderdate#60], [o_orderkey#56L ASC NULLS FIRST] + +(34) CometSortMergeJoin +Left output [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] +Right output [2]: [o_orderkey#56L, o_orderdate#60] +Arguments: [l_orderkey#16L], [o_orderkey#56L], Inner + +(35) CometProject +Input [8]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderkey#56L, o_orderdate#60] +Arguments: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60], [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60] + +(36) CometNativeScan parquet +Output [2]: [n_nationkey#48L, n_name#49] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/nation] +PushedFilters: [IsNotNull(n_nationkey)] +ReadSchema: struct + +(37) CometFilter +Input [2]: [n_nationkey#48L, n_name#49] +Condition : isnotnull(n_nationkey#48L) + +(38) CometBroadcastExchange +Input [2]: [n_nationkey#48L, n_name#49] +Arguments: [n_nationkey#48L, n_name#49] + +(39) CometBroadcastHashJoin +Left output [6]: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60] +Right output [2]: [n_nationkey#48L, n_name#49] +Arguments: [s_nationkey#111L], [n_nationkey#48L], Inner, BuildRight + +(40) CometProject +Input [8]: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60, n_nationkey#48L, n_name#49] +Arguments: [nation#122, o_year#123, amount#124], [n_name#49 AS nation#122, year(o_orderdate#60) AS o_year#123, ((l_extendedprice#21 * (1 - l_discount#22)) - (ps_supplycost#95 * l_quantity#20)) AS amount#124] + +(41) CometHashAggregate +Input [3]: [nation#122, o_year#123, amount#124] +Keys [2]: [nation#122, o_year#123] +Functions [1]: [partial_sum(amount#124)] + +(42) CometExchange +Input [4]: [nation#122, o_year#123, sum#132, isEmpty#133] +Arguments: hashpartitioning(nation#122, o_year#123, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=413] + +(43) CometHashAggregate +Input [4]: [nation#122, o_year#123, sum#132, isEmpty#133] +Keys [2]: [nation#122, o_year#123] +Functions [1]: [sum(amount#124)] + +(44) CometExchange +Input [3]: [nation#122, o_year#123, sum_profit#125] +Arguments: rangepartitioning(nation#122 ASC NULLS FIRST, o_year#123 DESC NULLS LAST, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=415] + +(45) CometSort +Input [3]: [nation#122, o_year#123, sum_profit#125] +Arguments: [nation#122, o_year#123, sum_profit#125], [nation#122 ASC NULLS FIRST, o_year#123 DESC NULLS LAST] + +(46) AdaptiveSparkPlan +Output [3]: [nation#122, o_year#123, sum_profit#125] +Arguments: isFinalPlan=false + + +Query 9 returned 175 rows, hash=fff4ba3023e74505f304fa3243cbeeb6 +Query 9 took 374.36 seconds + +Iteration 1 took 374.36 seconds + +Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/comet-offheap8g-q9-tpch-1775055133970.json diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.time b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.time new file mode 100644 index 0000000000..1b7dbc2463 --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.time @@ -0,0 +1,6868 @@ +26/04/01 08:45:54 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) +26/04/01 08:45:54 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address +26/04/01 08:45:54 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +26/04/01 08:45:55 INFO SparkContext: Running Spark version 3.5.8 +26/04/01 08:45:55 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 08:45:55 INFO SparkContext: Java version 17.0.17 +26/04/01 08:45:55 INFO ResourceUtils: ============================================================== +26/04/01 08:45:55 INFO ResourceUtils: No custom resources configured for spark.driver. +26/04/01 08:45:55 INFO ResourceUtils: ============================================================== +26/04/01 08:45:55 INFO SparkContext: Submitted application: comet-offheap8g-q9 benchmark derived from tpch +26/04/01 08:45:55 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 8192, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) +26/04/01 08:45:55 INFO ResourceProfile: Limiting resource is cpu +26/04/01 08:45:55 INFO ResourceProfileManager: Added ResourceProfile id: 0 +26/04/01 08:45:55 INFO SecurityManager: Changing view acls to: andy +26/04/01 08:45:55 INFO SecurityManager: Changing modify acls to: andy +26/04/01 08:45:55 INFO SecurityManager: Changing view acls groups to: +26/04/01 08:45:55 INFO SecurityManager: Changing modify acls groups to: +26/04/01 08:45:55 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY +26/04/01 08:45:55 INFO Utils: Successfully started service 'sparkDriver' on port 59331. +26/04/01 08:45:55 INFO SparkEnv: Registering MapOutputTracker +26/04/01 08:45:55 INFO SparkEnv: Registering BlockManagerMaster +26/04/01 08:45:55 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information +26/04/01 08:45:55 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up +26/04/01 08:45:55 INFO SparkEnv: Registering BlockManagerMasterHeartbeat +26/04/01 08:45:55 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-592952cc-8fb6-4441-88fb-62d0d58a71d3 +26/04/01 08:45:55 INFO MemoryStore: MemoryStore started with capacity 12.6 GiB +26/04/01 08:45:55 INFO SparkEnv: Registering OutputCommitCoordinator +26/04/01 08:45:55 INFO SparkContext: Added JAR file:///Users/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar at spark://10.0.0.133:59331/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775054755287 +26/04/01 08:45:55 INFO CometDriverPlugin: CometDriverPlugin init +26/04/01 08:45:55 INFO CometDriverPlugin: Setting spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions +26/04/01 08:45:55 INFO CometDriverPlugin: Comet metrics reporting is disabled. Set spark.comet.metrics.enabled=true to enable. +26/04/01 08:45:55 INFO CometDriverPlugin: Comet is running in unified memory mode and sharing off-heap memory with Spark +26/04/01 08:45:55 INFO DriverPluginContainer: Initialized driver component for plugin org.apache.spark.CometPlugin. +26/04/01 08:45:55 INFO Executor: Starting executor ID driver on host 10.0.0.133 +26/04/01 08:45:55 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 08:45:55 INFO Executor: Java version 17.0.17 +26/04/01 08:45:55 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' +26/04/01 08:45:55 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@13820261 for default. +26/04/01 08:45:55 INFO Executor: Fetching spark://10.0.0.133:59331/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775054755287 +26/04/01 08:45:55 INFO TransportClientFactory: Successfully created connection to /10.0.0.133:59331 after 8 ms (0 ms spent in bootstraps) +26/04/01 08:45:55 INFO Utils: Fetching spark://10.0.0.133:59331/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-7b7c98ca-005e-4989-9d8f-bb80c5e1e895/userFiles-2ec5bf60-08ed-4482-8793-c54b3bc0e259/fetchFileTemp13161011170765808055.tmp +26/04/01 08:45:55 INFO Executor: Adding file:/private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-7b7c98ca-005e-4989-9d8f-bb80c5e1e895/userFiles-2ec5bf60-08ed-4482-8793-c54b3bc0e259/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to class loader default +26/04/01 08:45:55 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 59333. +26/04/01 08:45:55 INFO NettyBlockTransferService: Server created on 10.0.0.133:59333 +26/04/01 08:45:55 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy +26/04/01 08:45:55 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 59333, None) +26/04/01 08:45:55 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:59333 with 12.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 59333, None) +26/04/01 08:45:55 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 59333, None) +26/04/01 08:45:55 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 59333, None) +26/04/01 08:45:55 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. +26/04/01 08:45:55 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. +26/04/01 08:45:56 INFO InMemoryFileIndex: It took 17 ms to list leaf files for 1 paths. +26/04/01 08:45:56 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:45:56 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:45:56 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:45:56 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:45:56 INFO DAGScheduler: Missing parents: List() +26/04/01 08:45:56 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:45:56 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 08:45:56 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) +26/04/01 08:45:56 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:56 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:45:56 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:45:56 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 +26/04/01 08:45:56 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 08:45:56 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) +26/04/01 08:45:56 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 2041 bytes result sent to driver +26/04/01 08:45:56 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 125 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:45:56 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool +26/04/01 08:45:56 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.407 s +26/04/01 08:45:56 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:45:56 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished +26/04/01 08:45:56 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.418894 s +26/04/01 08:45:59 INFO NativeBase: Couldn't locate log file from either COMET_CONF_DIR or comet.log.file.path. Using default log configuration with INFO log level which emits to stderr +26/04/01 08:45:59 INFO core/src/lib.rs: Comet native library version 0.15.0 initialized +26/04/01 08:45:59 INFO NativeBase: Setting system property arrow.enable_unsafe_memory_access to true +26/04/01 08:45:59 INFO NativeBase: Setting system property arrow.enable_null_check_for_get to false +26/04/01 08:45:59 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. +26/04/01 08:45:59 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:45:59 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:45:59 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:45:59 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:45:59 INFO DAGScheduler: Missing parents: List() +26/04/01 08:45:59 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:45:59 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 08:45:59 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) +26/04/01 08:45:59 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:59 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:45:59 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:45:59 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 +26/04/01 08:45:59 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 08:45:59 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) +26/04/01 08:45:59 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2344 bytes result sent to driver +26/04/01 08:45:59 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 16 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:45:59 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool +26/04/01 08:45:59 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.024 s +26/04/01 08:45:59 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:45:59 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished +26/04/01 08:45:59 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.026210 s +26/04/01 08:45:59 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 08:45:59 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:45:59 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:45:59 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:45:59 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:45:59 INFO DAGScheduler: Missing parents: List() +26/04/01 08:45:59 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:45:59 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 08:45:59 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) +26/04/01 08:45:59 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:59 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:45:59 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:45:59 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 +26/04/01 08:45:59 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 08:45:59 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) +26/04/01 08:45:59 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver +26/04/01 08:45:59 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 9 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:45:59 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool +26/04/01 08:45:59 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.017 s +26/04/01 08:45:59 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:45:59 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished +26/04/01 08:45:59 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.018918 s +26/04/01 08:45:59 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. +26/04/01 08:45:59 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:45:59 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:45:59 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:45:59 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:45:59 INFO DAGScheduler: Missing parents: List() +26/04/01 08:45:59 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:45:59 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 08:45:59 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) +26/04/01 08:45:59 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:59 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:45:59 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:45:59 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 +26/04/01 08:45:59 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 08:45:59 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) +26/04/01 08:45:59 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver +26/04/01 08:45:59 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 7 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:45:59 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool +26/04/01 08:45:59 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s +26/04/01 08:45:59 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:45:59 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished +26/04/01 08:45:59 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.016055 s +26/04/01 08:45:59 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. +26/04/01 08:45:59 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:45:59 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:45:59 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:45:59 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:45:59 INFO DAGScheduler: Missing parents: List() +26/04/01 08:45:59 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:45:59 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 08:45:59 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) +26/04/01 08:45:59 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:59 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:45:59 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:45:59 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 +26/04/01 08:45:59 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9299 bytes) +26/04/01 08:45:59 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) +26/04/01 08:45:59 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver +26/04/01 08:45:59 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 7 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:45:59 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool +26/04/01 08:45:59 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 08:45:59 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:45:59 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished +26/04/01 08:45:59 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.015169 s +26/04/01 08:45:59 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 08:45:59 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:45:59 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:45:59 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:45:59 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:45:59 INFO DAGScheduler: Missing parents: List() +26/04/01 08:45:59 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:45:59 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 08:45:59 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) +26/04/01 08:45:59 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:59 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:45:59 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:45:59 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 +26/04/01 08:45:59 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 08:45:59 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) +26/04/01 08:45:59 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver +26/04/01 08:45:59 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:45:59 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool +26/04/01 08:45:59 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 08:45:59 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:45:59 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished +26/04/01 08:45:59 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.014371 s +26/04/01 08:45:59 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. +26/04/01 08:45:59 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:45:59 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:45:59 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:45:59 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:45:59 INFO DAGScheduler: Missing parents: List() +26/04/01 08:45:59 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:45:59 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 08:45:59 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) +26/04/01 08:45:59 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:59 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:45:59 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:45:59 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 +26/04/01 08:45:59 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) +26/04/01 08:45:59 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) +26/04/01 08:45:59 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver +26/04/01 08:45:59 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:45:59 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool +26/04/01 08:45:59 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 08:45:59 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:45:59 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished +26/04/01 08:45:59 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.013381 s +26/04/01 08:45:59 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 08:45:59 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:45:59 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:45:59 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:45:59 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:45:59 INFO DAGScheduler: Missing parents: List() +26/04/01 08:45:59 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:45:59 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) +26/04/01 08:45:59 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) +26/04/01 08:45:59 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:59 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:45:59 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:45:59 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 +26/04/01 08:45:59 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) +26/04/01 08:45:59 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) +26/04/01 08:45:59 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver +26/04/01 08:45:59 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:45:59 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool +26/04/01 08:45:59 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s +26/04/01 08:45:59 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:45:59 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished +26/04/01 08:45:59 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.014852 s +26/04/01 08:45:59 INFO FileSourceStrategy: Pushed Filters: IsNotNull(p_name),StringContains(p_name,moccasin),IsNotNull(p_partkey) +26/04/01 08:45:59 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(p_name#75),Contains(p_name#75, moccasin),isnotnull(p_partkey#74L) +26/04/01 08:45:59 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_partkey),IsNotNull(l_suppkey),IsNotNull(l_orderkey) +26/04/01 08:45:59 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_partkey#17L),isnotnull(l_suppkey#18L),isnotnull(l_orderkey#16L) +26/04/01 08:45:59 INFO FileSourceStrategy: Pushed Filters: IsNotNull(s_suppkey),IsNotNull(s_nationkey) +26/04/01 08:45:59 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(s_suppkey#108L),isnotnull(s_nationkey#111L) +26/04/01 08:45:59 INFO FileSourceStrategy: Pushed Filters: IsNotNull(ps_suppkey),IsNotNull(ps_partkey) +26/04/01 08:45:59 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(ps_suppkey#93L),isnotnull(ps_partkey#92L) +26/04/01 08:45:59 INFO FileSourceStrategy: Pushed Filters: IsNotNull(o_orderkey) +26/04/01 08:45:59 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(o_orderkey#56L) +26/04/01 08:45:59 INFO FileSourceStrategy: Pushed Filters: IsNotNull(n_nationkey) +26/04/01 08:45:59 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(n_nationkey#48L) +26/04/01 08:45:59 INFO CometScanExec: Planning scan with bin packing, max size: 102898790 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) +26/04/01 08:45:59 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:45:59 INFO CometScanExec: Planning scan with bin packing, max size: 27960063 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:45:59 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:45:59 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:45:59 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:46:00 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:46:00 INFO CometScanExec: Planning scan with bin packing, max size: 102898790 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:59333 (size: 35.4 KiB, free: 12.6 GiB) +26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:59333 (size: 35.4 KiB, free: 12.6 GiB) +26/04/01 08:46:00 INFO SparkContext: Created broadcast 9 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:46:00 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:46:00 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:46:00 INFO FileSourceScanExec: Planning scan with bin packing, max size: 102898790 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:46:00 INFO DAGScheduler: Registering RDD 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 +26/04/01 08:46:00 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:46:00 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 8 output partitions +26/04/01 08:46:00 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:46:00 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:46:00 INFO DAGScheduler: Missing parents: List() +26/04/01 08:46:00 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[21] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:46:00 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 20.5 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 8.1 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:59333 (size: 8.1 KiB, free: 12.6 GiB) +26/04/01 08:46:00 INFO SparkContext: Created broadcast 11 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:46:00 INFO DAGScheduler: Submitting 8 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[21] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7)) +26/04/01 08:46:00 INFO TaskSchedulerImpl: Adding task set 8.0 with 8 tasks resource profile 0 +26/04/01 08:46:00 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 08:46:00 INFO DAGScheduler: Final stage: ResultStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:46:00 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:46:00 INFO DAGScheduler: Missing parents: List() +26/04/01 08:46:00 INFO DAGScheduler: Submitting ResultStage 9 (MapPartitionsRDD[20] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:46:00 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9429 bytes) +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO TaskSetManager: Starting task 1.0 in stage 8.0 (TID 9) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9431 bytes) +26/04/01 08:46:00 INFO TaskSetManager: Starting task 2.0 in stage 8.0 (TID 10) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9431 bytes) +26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:59333 (size: 35.4 KiB, free: 12.6 GiB) +26/04/01 08:46:00 INFO TaskSetManager: Starting task 3.0 in stage 8.0 (TID 11) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9431 bytes) +26/04/01 08:46:00 INFO TaskSetManager: Starting task 4.0 in stage 8.0 (TID 12) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9430 bytes) +26/04/01 08:46:00 INFO SparkContext: Created broadcast 10 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:46:00 INFO TaskSetManager: Starting task 5.0 in stage 8.0 (TID 13) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9432 bytes) +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 10.8 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO TaskSetManager: Starting task 6.0 in stage 8.0 (TID 14) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9431 bytes) +26/04/01 08:46:00 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:46:00 INFO TaskSetManager: Starting task 7.0 in stage 8.0 (TID 15) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9432 bytes) +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 5.3 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) +26/04/01 08:46:00 INFO Executor: Running task 1.0 in stage 8.0 (TID 9) +26/04/01 08:46:00 INFO Executor: Running task 2.0 in stage 8.0 (TID 10) +26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:59333 (size: 5.3 KiB, free: 12.6 GiB) +26/04/01 08:46:00 INFO Executor: Running task 3.0 in stage 8.0 (TID 11) +26/04/01 08:46:00 INFO Executor: Running task 4.0 in stage 8.0 (TID 12) +26/04/01 08:46:00 INFO SparkContext: Created broadcast 12 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:46:00 INFO Executor: Running task 5.0 in stage 8.0 (TID 13) +26/04/01 08:46:00 INFO Executor: Running task 6.0 in stage 8.0 (TID 14) +26/04/01 08:46:00 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 9 (MapPartitionsRDD[20] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:46:00 INFO TaskSchedulerImpl: Adding task set 9.0 with 1 tasks resource profile 0 +26/04/01 08:46:00 INFO Executor: Running task 7.0 in stage 8.0 (TID 15) +26/04/01 08:46:00 INFO DAGScheduler: Registering RDD 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 +26/04/01 08:46:00 INFO CometScanExec: Planning scan with bin packing, max size: 27960063 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:46:00 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions +26/04/01 08:46:00 INFO DAGScheduler: Final stage: ShuffleMapStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:46:00 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:46:00 INFO DAGScheduler: Missing parents: List() +26/04/01 08:46:00 INFO DAGScheduler: Submitting ShuffleMapStage 10 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_13 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_14 stored as values in memory (estimated size 20.6 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_14_piece0 stored as bytes in memory (estimated size 8.2 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_14_piece0 in memory on 10.0.0.133:59333 (size: 8.2 KiB, free: 12.6 GiB) +26/04/01 08:46:00 INFO SparkContext: Created broadcast 14 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:46:00 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 10 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:46:00 INFO TaskSchedulerImpl: Adding task set 10.0 with 208 tasks resource profile 0 +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_13_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_13_piece0 in memory on 10.0.0.133:59333 (size: 35.4 KiB, free: 12.6 GiB) +26/04/01 08:46:00 INFO SparkContext: Created broadcast 13 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:46:00 INFO FileSourceScanExec: Planning scan with bin packing, max size: 27960063 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:46:00 INFO DAGScheduler: Registering RDD 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 2 +26/04/01 08:46:00 INFO DAGScheduler: Got map stage job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 8 output partitions +26/04/01 08:46:00 INFO DAGScheduler: Final stage: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:46:00 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:46:00 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:46:00 INFO DAGScheduler: Missing parents: List() +26/04/01 08:46:00 INFO DAGScheduler: Submitting ShuffleMapStage 11 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_15 stored as values in memory (estimated size 18.0 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_15_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_15_piece0 in memory on 10.0.0.133:59333 (size: 7.9 KiB, free: 12.6 GiB) +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_16 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO SparkContext: Created broadcast 15 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:46:00 INFO DAGScheduler: Submitting 8 missing tasks from ShuffleMapStage 11 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7)) +26/04/01 08:46:00 INFO TaskSchedulerImpl: Adding task set 11.0 with 8 tasks resource profile 0 +26/04/01 08:46:00 INFO BaseAllocator: Debug mode disabled. Enable with the VM option -Darrow.memory.debug.allocator=true. +26/04/01 08:46:00 INFO DefaultAllocationManagerOption: allocation manager type not specified, using netty as the default type +26/04/01 08:46:00 INFO CheckAllocator: Using DefaultAllocationManager at memory/unsafe/DefaultAllocationManagerFactory.class +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_16_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_16_piece0 in memory on 10.0.0.133:59333 (size: 35.4 KiB, free: 12.6 GiB) +26/04/01 08:46:00 INFO SparkContext: Created broadcast 16 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:46:00 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:46:00 INFO DAGScheduler: Registering RDD 30 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 3 +26/04/01 08:46:00 INFO DAGScheduler: Got map stage job 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 37 output partitions +26/04/01 08:46:00 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:46:00 INFO DAGScheduler: Final stage: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:46:00 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:46:00 INFO DAGScheduler: Missing parents: List() +26/04/01 08:46:00 INFO DAGScheduler: Submitting ShuffleMapStage 12 (MapPartitionsRDD[30] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_18 stored as values in memory (estimated size 18.2 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_18_piece0 stored as bytes in memory (estimated size 8.0 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_17 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_18_piece0 in memory on 10.0.0.133:59333 (size: 8.0 KiB, free: 12.6 GiB) +26/04/01 08:46:00 INFO core/src/execution/jni_api.rs: Comet tokio runtime: using spark.executor.cores=8 worker threads +26/04/01 08:46:00 INFO SparkContext: Created broadcast 18 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:46:00 INFO DAGScheduler: Submitting 37 missing tasks from ShuffleMapStage 12 (MapPartitionsRDD[30] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:46:00 INFO TaskSchedulerImpl: Adding task set 12.0 with 37 tasks resource profile 0 +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_17_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_17_piece0 in memory on 10.0.0.133:59333 (size: 35.4 KiB, free: 12.6 GiB) +26/04/01 08:46:00 INFO SparkContext: Created broadcast 17 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:46:00 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:46:00 INFO DAGScheduler: Registering RDD 33 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 4 +26/04/01 08:46:00 INFO DAGScheduler: Got map stage job 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 64 output partitions +26/04/01 08:46:00 INFO DAGScheduler: Final stage: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:46:00 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:46:00 INFO DAGScheduler: Missing parents: List() +26/04/01 08:46:00 INFO DAGScheduler: Submitting ShuffleMapStage 13 (MapPartitionsRDD[33] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_19 stored as values in memory (estimated size 16.6 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO MemoryStore: Block broadcast_19_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 12.6 GiB) +26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_19_piece0 in memory on 10.0.0.133:59333 (size: 7.9 KiB, free: 12.6 GiB) +26/04/01 08:46:00 INFO SparkContext: Created broadcast 19 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:46:00 INFO DAGScheduler: Submitting 64 missing tasks from ShuffleMapStage 13 (MapPartitionsRDD[33] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:46:00 INFO TaskSchedulerImpl: Adding task set 13.0 with 64 tasks resource profile 0 +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO Executor: Finished task 6.0 in stage 8.0 (TID 14). 6761 bytes result sent to driver +26/04/01 08:46:02 INFO Executor: Finished task 5.0 in stage 8.0 (TID 13). 6761 bytes result sent to driver +26/04/01 08:46:02 INFO Executor: Finished task 2.0 in stage 8.0 (TID 10). 6761 bytes result sent to driver +26/04/01 08:46:02 INFO Executor: Finished task 1.0 in stage 8.0 (TID 9). 6761 bytes result sent to driver +26/04/01 08:46:02 INFO Executor: Finished task 3.0 in stage 8.0 (TID 11). 6761 bytes result sent to driver +26/04/01 08:46:02 INFO Executor: Finished task 7.0 in stage 8.0 (TID 15). 6761 bytes result sent to driver +26/04/01 08:46:02 INFO Executor: Finished task 4.0 in stage 8.0 (TID 12). 6761 bytes result sent to driver +26/04/01 08:46:02 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 6761 bytes result sent to driver +26/04/01 08:46:02 INFO TaskSetManager: Starting task 0.0 in stage 9.0 (TID 16) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:46:02 INFO Executor: Running task 0.0 in stage 9.0 (TID 16) +26/04/01 08:46:02 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 17) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:46:02 INFO TaskSetManager: Finished task 6.0 in stage 8.0 (TID 14) in 2484 ms on 10.0.0.133 (executor driver) (1/8) +26/04/01 08:46:02 INFO Executor: Running task 0.0 in stage 10.0 (TID 17) +26/04/01 08:46:02 INFO TaskSetManager: Starting task 1.0 in stage 10.0 (TID 18) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:02 INFO TaskSetManager: Finished task 5.0 in stage 8.0 (TID 13) in 2484 ms on 10.0.0.133 (executor driver) (2/8) +26/04/01 08:46:02 INFO Executor: Running task 1.0 in stage 10.0 (TID 18) +26/04/01 08:46:02 INFO TaskSetManager: Starting task 2.0 in stage 10.0 (TID 19) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:02 INFO Executor: Running task 2.0 in stage 10.0 (TID 19) +26/04/01 08:46:02 INFO TaskSetManager: Starting task 3.0 in stage 10.0 (TID 20) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:02 INFO TaskSetManager: Finished task 1.0 in stage 8.0 (TID 9) in 2485 ms on 10.0.0.133 (executor driver) (3/8) +26/04/01 08:46:02 INFO Executor: Running task 3.0 in stage 10.0 (TID 20) +26/04/01 08:46:02 INFO TaskSetManager: Starting task 4.0 in stage 10.0 (TID 21) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:02 INFO Executor: Running task 4.0 in stage 10.0 (TID 21) +26/04/01 08:46:02 INFO TaskSetManager: Starting task 5.0 in stage 10.0 (TID 22) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:02 INFO TaskSetManager: Finished task 7.0 in stage 8.0 (TID 15) in 2485 ms on 10.0.0.133 (executor driver) (4/8) +26/04/01 08:46:02 INFO Executor: Running task 5.0 in stage 10.0 (TID 22) +26/04/01 08:46:02 INFO TaskSetManager: Starting task 6.0 in stage 10.0 (TID 23) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:46:02 INFO TaskSetManager: Finished task 3.0 in stage 8.0 (TID 11) in 2486 ms on 10.0.0.133 (executor driver) (5/8) +26/04/01 08:46:02 INFO Executor: Running task 6.0 in stage 10.0 (TID 23) +26/04/01 08:46:02 INFO TaskSetManager: Finished task 2.0 in stage 8.0 (TID 10) in 2486 ms on 10.0.0.133 (executor driver) (6/8) +26/04/01 08:46:02 INFO TaskSetManager: Finished task 4.0 in stage 8.0 (TID 12) in 2487 ms on 10.0.0.133 (executor driver) (7/8) +26/04/01 08:46:02 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 2488 ms on 10.0.0.133 (executor driver) (8/8) +26/04/01 08:46:02 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool +26/04/01 08:46:02 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.498 s +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:46:02 INFO DAGScheduler: running: Set(ShuffleMapStage 12, ResultStage 9, ShuffleMapStage 13, ShuffleMapStage 10, ShuffleMapStage 11) +26/04/01 08:46:02 INFO DAGScheduler: waiting: Set() +26/04/01 08:46:02 INFO DAGScheduler: failed: Set() +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO ShufflePartitionsUtil: For shuffle(0), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 08:46:02 INFO Executor: Finished task 0.0 in stage 9.0 (TID 16). 4805 bytes result sent to driver +26/04/01 08:46:02 INFO TaskSetManager: Starting task 7.0 in stage 10.0 (TID 24) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:02 INFO Executor: Running task 7.0 in stage 10.0 (TID 24) +26/04/01 08:46:02 INFO TaskSetManager: Finished task 0.0 in stage 9.0 (TID 16) in 44 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:46:02 INFO TaskSchedulerImpl: Removed TaskSet 9.0, whose tasks have all completed, from pool +26/04/01 08:46:02 INFO DAGScheduler: ResultStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.527 s +26/04/01 08:46:02 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:46:02 INFO TaskSchedulerImpl: Killing all running tasks in stage 9: Stage finished +26/04/01 08:46:02 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 2.537089 s +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:02 INFO Utils: Coalesced 1 broadcast batches into 1 (25 rows) +26/04/01 08:46:02 INFO MemoryStore: Block broadcast_20 stored as values in memory (estimated size 880.0 B, free 12.6 GiB) +26/04/01 08:46:02 INFO MemoryStore: Block broadcast_20_piece0 stored as bytes in memory (estimated size 831.0 B, free 12.6 GiB) +26/04/01 08:46:02 INFO BlockManagerInfo: Added broadcast_20_piece0 in memory on 10.0.0.133:59333 (size: 831.0 B, free: 12.6 GiB) +26/04/01 08:46:02 INFO SparkContext: Created broadcast 20 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:46:02 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:46:02 INFO DAGScheduler: Got job 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions +26/04/01 08:46:02 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:46:02 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 14) +26/04/01 08:46:02 INFO DAGScheduler: Missing parents: List() +26/04/01 08:46:02 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[35] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:46:02 INFO MemoryStore: Block broadcast_21 stored as values in memory (estimated size 14.5 KiB, free 12.6 GiB) +26/04/01 08:46:02 INFO MemoryStore: Block broadcast_21_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 12.6 GiB) +26/04/01 08:46:02 INFO BlockManagerInfo: Added broadcast_21_piece0 in memory on 10.0.0.133:59333 (size: 7.9 KiB, free: 12.6 GiB) +26/04/01 08:46:02 INFO SparkContext: Created broadcast 21 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:46:02 INFO DAGScheduler: Submitting 200 missing tasks from ResultStage 15 (MapPartitionsRDD[35] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:46:02 INFO TaskSchedulerImpl: Adding task set 15.0 with 200 tasks resource profile 0 +26/04/01 08:46:10 INFO Executor: Finished task 1.0 in stage 10.0 (TID 18). 6649 bytes result sent to driver +26/04/01 08:46:10 INFO TaskSetManager: Starting task 8.0 in stage 10.0 (TID 25) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:10 INFO Executor: Running task 8.0 in stage 10.0 (TID 25) +26/04/01 08:46:10 INFO TaskSetManager: Finished task 1.0 in stage 10.0 (TID 18) in 8046 ms on 10.0.0.133 (executor driver) (1/208) +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:10 INFO Executor: Finished task 0.0 in stage 10.0 (TID 17). 6649 bytes result sent to driver +26/04/01 08:46:10 INFO TaskSetManager: Starting task 9.0 in stage 10.0 (TID 26) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:10 INFO Executor: Running task 9.0 in stage 10.0 (TID 26) +26/04/01 08:46:10 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 17) in 8054 ms on 10.0.0.133 (executor driver) (2/208) +26/04/01 08:46:10 INFO Executor: Finished task 5.0 in stage 10.0 (TID 22). 6649 bytes result sent to driver +26/04/01 08:46:10 INFO TaskSetManager: Starting task 10.0 in stage 10.0 (TID 27) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:10 INFO Executor: Running task 10.0 in stage 10.0 (TID 27) +26/04/01 08:46:10 INFO TaskSetManager: Finished task 5.0 in stage 10.0 (TID 22) in 8053 ms on 10.0.0.133 (executor driver) (3/208) +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:10 INFO Executor: Finished task 6.0 in stage 10.0 (TID 23). 6649 bytes result sent to driver +26/04/01 08:46:10 INFO TaskSetManager: Starting task 11.0 in stage 10.0 (TID 28) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:10 INFO TaskSetManager: Finished task 6.0 in stage 10.0 (TID 23) in 8059 ms on 10.0.0.133 (executor driver) (4/208) +26/04/01 08:46:10 INFO Executor: Running task 11.0 in stage 10.0 (TID 28) +26/04/01 08:46:10 INFO Executor: Finished task 7.0 in stage 10.0 (TID 24). 6649 bytes result sent to driver +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:10 INFO TaskSetManager: Starting task 12.0 in stage 10.0 (TID 29) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:46:10 INFO TaskSetManager: Finished task 7.0 in stage 10.0 (TID 24) in 8021 ms on 10.0.0.133 (executor driver) (5/208) +26/04/01 08:46:10 INFO Executor: Running task 12.0 in stage 10.0 (TID 29) +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:10 INFO Executor: Finished task 2.0 in stage 10.0 (TID 19). 6649 bytes result sent to driver +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:10 INFO TaskSetManager: Starting task 13.0 in stage 10.0 (TID 30) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:10 INFO Executor: Running task 13.0 in stage 10.0 (TID 30) +26/04/01 08:46:10 INFO TaskSetManager: Finished task 2.0 in stage 10.0 (TID 19) in 8066 ms on 10.0.0.133 (executor driver) (6/208) +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:10 INFO Executor: Finished task 4.0 in stage 10.0 (TID 21). 6649 bytes result sent to driver +26/04/01 08:46:10 INFO TaskSetManager: Starting task 14.0 in stage 10.0 (TID 31) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:10 INFO Executor: Running task 14.0 in stage 10.0 (TID 31) +26/04/01 08:46:10 INFO TaskSetManager: Finished task 4.0 in stage 10.0 (TID 21) in 8080 ms on 10.0.0.133 (executor driver) (7/208) +26/04/01 08:46:10 INFO Executor: Finished task 3.0 in stage 10.0 (TID 20). 6649 bytes result sent to driver +26/04/01 08:46:10 INFO TaskSetManager: Starting task 15.0 in stage 10.0 (TID 32) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:10 INFO Executor: Running task 15.0 in stage 10.0 (TID 32) +26/04/01 08:46:10 INFO TaskSetManager: Finished task 3.0 in stage 10.0 (TID 20) in 8082 ms on 10.0.0.133 (executor driver) (8/208) +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO Executor: Finished task 9.0 in stage 10.0 (TID 26). 6649 bytes result sent to driver +26/04/01 08:46:18 INFO TaskSetManager: Starting task 16.0 in stage 10.0 (TID 33) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:18 INFO TaskSetManager: Finished task 9.0 in stage 10.0 (TID 26) in 7900 ms on 10.0.0.133 (executor driver) (9/208) +26/04/01 08:46:18 INFO Executor: Running task 16.0 in stage 10.0 (TID 33) +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO Executor: Finished task 14.0 in stage 10.0 (TID 31). 6606 bytes result sent to driver +26/04/01 08:46:18 INFO TaskSetManager: Starting task 17.0 in stage 10.0 (TID 34) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:18 INFO Executor: Finished task 8.0 in stage 10.0 (TID 25). 6649 bytes result sent to driver +26/04/01 08:46:18 INFO Executor: Running task 17.0 in stage 10.0 (TID 34) +26/04/01 08:46:18 INFO TaskSetManager: Finished task 14.0 in stage 10.0 (TID 31) in 7890 ms on 10.0.0.133 (executor driver) (10/208) +26/04/01 08:46:18 INFO TaskSetManager: Starting task 18.0 in stage 10.0 (TID 35) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:46:18 INFO Executor: Running task 18.0 in stage 10.0 (TID 35) +26/04/01 08:46:18 INFO TaskSetManager: Finished task 8.0 in stage 10.0 (TID 25) in 7926 ms on 10.0.0.133 (executor driver) (11/208) +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO Executor: Finished task 10.0 in stage 10.0 (TID 27). 6606 bytes result sent to driver +26/04/01 08:46:18 INFO TaskSetManager: Starting task 19.0 in stage 10.0 (TID 36) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:46:18 INFO TaskSetManager: Finished task 10.0 in stage 10.0 (TID 27) in 7922 ms on 10.0.0.133 (executor driver) (12/208) +26/04/01 08:46:18 INFO Executor: Running task 19.0 in stage 10.0 (TID 36) +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO Executor: Finished task 15.0 in stage 10.0 (TID 32). 6606 bytes result sent to driver +26/04/01 08:46:18 INFO Executor: Finished task 12.0 in stage 10.0 (TID 29). 6606 bytes result sent to driver +26/04/01 08:46:18 INFO TaskSetManager: Starting task 20.0 in stage 10.0 (TID 37) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:18 INFO Executor: Running task 20.0 in stage 10.0 (TID 37) +26/04/01 08:46:18 INFO TaskSetManager: Starting task 21.0 in stage 10.0 (TID 38) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:18 INFO TaskSetManager: Finished task 15.0 in stage 10.0 (TID 32) in 7935 ms on 10.0.0.133 (executor driver) (13/208) +26/04/01 08:46:18 INFO Executor: Running task 21.0 in stage 10.0 (TID 38) +26/04/01 08:46:18 INFO TaskSetManager: Finished task 12.0 in stage 10.0 (TID 29) in 7955 ms on 10.0.0.133 (executor driver) (14/208) +26/04/01 08:46:18 INFO Executor: Finished task 13.0 in stage 10.0 (TID 30). 6606 bytes result sent to driver +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO TaskSetManager: Starting task 22.0 in stage 10.0 (TID 39) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:18 INFO Executor: Running task 22.0 in stage 10.0 (TID 39) +26/04/01 08:46:18 INFO TaskSetManager: Finished task 13.0 in stage 10.0 (TID 30) in 7953 ms on 10.0.0.133 (executor driver) (15/208) +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO Executor: Finished task 11.0 in stage 10.0 (TID 28). 6606 bytes result sent to driver +26/04/01 08:46:18 INFO TaskSetManager: Starting task 23.0 in stage 10.0 (TID 40) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:18 INFO Executor: Running task 23.0 in stage 10.0 (TID 40) +26/04/01 08:46:18 INFO TaskSetManager: Finished task 11.0 in stage 10.0 (TID 28) in 7966 ms on 10.0.0.133 (executor driver) (16/208) +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO Executor: Finished task 17.0 in stage 10.0 (TID 34). 6606 bytes result sent to driver +26/04/01 08:46:26 INFO TaskSetManager: Starting task 24.0 in stage 10.0 (TID 41) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:46:26 INFO TaskSetManager: Finished task 17.0 in stage 10.0 (TID 34) in 7926 ms on 10.0.0.133 (executor driver) (17/208) +26/04/01 08:46:26 INFO Executor: Running task 24.0 in stage 10.0 (TID 41) +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO Executor: Finished task 18.0 in stage 10.0 (TID 35). 6606 bytes result sent to driver +26/04/01 08:46:26 INFO TaskSetManager: Starting task 25.0 in stage 10.0 (TID 42) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:26 INFO TaskSetManager: Finished task 18.0 in stage 10.0 (TID 35) in 7930 ms on 10.0.0.133 (executor driver) (18/208) +26/04/01 08:46:26 INFO Executor: Running task 25.0 in stage 10.0 (TID 42) +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO Executor: Finished task 16.0 in stage 10.0 (TID 33). 6606 bytes result sent to driver +26/04/01 08:46:26 INFO TaskSetManager: Starting task 26.0 in stage 10.0 (TID 43) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:26 INFO TaskSetManager: Finished task 16.0 in stage 10.0 (TID 33) in 7955 ms on 10.0.0.133 (executor driver) (19/208) +26/04/01 08:46:26 INFO Executor: Running task 26.0 in stage 10.0 (TID 43) +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO Executor: Finished task 19.0 in stage 10.0 (TID 36). 6606 bytes result sent to driver +26/04/01 08:46:26 INFO TaskSetManager: Starting task 27.0 in stage 10.0 (TID 44) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO TaskSetManager: Finished task 19.0 in stage 10.0 (TID 36) in 7934 ms on 10.0.0.133 (executor driver) (20/208) +26/04/01 08:46:26 INFO Executor: Running task 27.0 in stage 10.0 (TID 44) +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO Executor: Finished task 22.0 in stage 10.0 (TID 39). 6606 bytes result sent to driver +26/04/01 08:46:26 INFO TaskSetManager: Starting task 28.0 in stage 10.0 (TID 45) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:26 INFO Executor: Running task 28.0 in stage 10.0 (TID 45) +26/04/01 08:46:26 INFO TaskSetManager: Finished task 22.0 in stage 10.0 (TID 39) in 7900 ms on 10.0.0.133 (executor driver) (21/208) +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO Executor: Finished task 21.0 in stage 10.0 (TID 38). 6606 bytes result sent to driver +26/04/01 08:46:26 INFO TaskSetManager: Starting task 29.0 in stage 10.0 (TID 46) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:26 INFO TaskSetManager: Finished task 21.0 in stage 10.0 (TID 38) in 7924 ms on 10.0.0.133 (executor driver) (22/208) +26/04/01 08:46:26 INFO Executor: Running task 29.0 in stage 10.0 (TID 46) +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO Executor: Finished task 23.0 in stage 10.0 (TID 40). 6606 bytes result sent to driver +26/04/01 08:46:26 INFO TaskSetManager: Starting task 30.0 in stage 10.0 (TID 47) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:46:26 INFO TaskSetManager: Finished task 23.0 in stage 10.0 (TID 40) in 7927 ms on 10.0.0.133 (executor driver) (23/208) +26/04/01 08:46:26 INFO Executor: Running task 30.0 in stage 10.0 (TID 47) +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO Executor: Finished task 20.0 in stage 10.0 (TID 37). 6606 bytes result sent to driver +26/04/01 08:46:26 INFO TaskSetManager: Starting task 31.0 in stage 10.0 (TID 48) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:26 INFO TaskSetManager: Finished task 20.0 in stage 10.0 (TID 37) in 7944 ms on 10.0.0.133 (executor driver) (24/208) +26/04/01 08:46:26 INFO Executor: Running task 31.0 in stage 10.0 (TID 48) +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO Executor: Finished task 25.0 in stage 10.0 (TID 42). 6649 bytes result sent to driver +26/04/01 08:46:34 INFO TaskSetManager: Starting task 32.0 in stage 10.0 (TID 49) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:34 INFO Executor: Running task 32.0 in stage 10.0 (TID 49) +26/04/01 08:46:34 INFO TaskSetManager: Finished task 25.0 in stage 10.0 (TID 42) in 7892 ms on 10.0.0.133 (executor driver) (25/208) +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO Executor: Finished task 28.0 in stage 10.0 (TID 45). 6606 bytes result sent to driver +26/04/01 08:46:34 INFO TaskSetManager: Starting task 33.0 in stage 10.0 (TID 50) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:34 INFO TaskSetManager: Finished task 28.0 in stage 10.0 (TID 45) in 7914 ms on 10.0.0.133 (executor driver) (26/208) +26/04/01 08:46:34 INFO Executor: Running task 33.0 in stage 10.0 (TID 50) +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO Executor: Finished task 26.0 in stage 10.0 (TID 43). 6606 bytes result sent to driver +26/04/01 08:46:34 INFO TaskSetManager: Starting task 34.0 in stage 10.0 (TID 51) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:34 INFO TaskSetManager: Finished task 26.0 in stage 10.0 (TID 43) in 7929 ms on 10.0.0.133 (executor driver) (27/208) +26/04/01 08:46:34 INFO Executor: Running task 34.0 in stage 10.0 (TID 51) +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO Executor: Finished task 27.0 in stage 10.0 (TID 44). 6606 bytes result sent to driver +26/04/01 08:46:34 INFO TaskSetManager: Starting task 35.0 in stage 10.0 (TID 52) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:34 INFO TaskSetManager: Finished task 27.0 in stage 10.0 (TID 44) in 7929 ms on 10.0.0.133 (executor driver) (28/208) +26/04/01 08:46:34 INFO Executor: Running task 35.0 in stage 10.0 (TID 52) +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO Executor: Finished task 24.0 in stage 10.0 (TID 41). 6606 bytes result sent to driver +26/04/01 08:46:34 INFO TaskSetManager: Starting task 36.0 in stage 10.0 (TID 53) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:46:34 INFO Executor: Running task 36.0 in stage 10.0 (TID 53) +26/04/01 08:46:34 INFO TaskSetManager: Finished task 24.0 in stage 10.0 (TID 41) in 7946 ms on 10.0.0.133 (executor driver) (29/208) +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO Executor: Finished task 29.0 in stage 10.0 (TID 46). 6606 bytes result sent to driver +26/04/01 08:46:34 INFO TaskSetManager: Starting task 37.0 in stage 10.0 (TID 54) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:34 INFO Executor: Running task 37.0 in stage 10.0 (TID 54) +26/04/01 08:46:34 INFO TaskSetManager: Finished task 29.0 in stage 10.0 (TID 46) in 7908 ms on 10.0.0.133 (executor driver) (30/208) +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO Executor: Finished task 31.0 in stage 10.0 (TID 48). 6606 bytes result sent to driver +26/04/01 08:46:34 INFO TaskSetManager: Starting task 38.0 in stage 10.0 (TID 55) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:34 INFO TaskSetManager: Finished task 31.0 in stage 10.0 (TID 48) in 7918 ms on 10.0.0.133 (executor driver) (31/208) +26/04/01 08:46:34 INFO Executor: Running task 38.0 in stage 10.0 (TID 55) +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO Executor: Finished task 30.0 in stage 10.0 (TID 47). 6606 bytes result sent to driver +26/04/01 08:46:34 INFO TaskSetManager: Starting task 39.0 in stage 10.0 (TID 56) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:34 INFO Executor: Running task 39.0 in stage 10.0 (TID 56) +26/04/01 08:46:34 INFO TaskSetManager: Finished task 30.0 in stage 10.0 (TID 47) in 7938 ms on 10.0.0.133 (executor driver) (32/208) +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO Executor: Finished task 32.0 in stage 10.0 (TID 49). 6606 bytes result sent to driver +26/04/01 08:46:42 INFO TaskSetManager: Starting task 40.0 in stage 10.0 (TID 57) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:42 INFO TaskSetManager: Finished task 32.0 in stage 10.0 (TID 49) in 7939 ms on 10.0.0.133 (executor driver) (33/208) +26/04/01 08:46:42 INFO Executor: Running task 40.0 in stage 10.0 (TID 57) +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO Executor: Finished task 34.0 in stage 10.0 (TID 51). 6606 bytes result sent to driver +26/04/01 08:46:42 INFO TaskSetManager: Starting task 41.0 in stage 10.0 (TID 58) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:42 INFO TaskSetManager: Finished task 34.0 in stage 10.0 (TID 51) in 7922 ms on 10.0.0.133 (executor driver) (34/208) +26/04/01 08:46:42 INFO Executor: Running task 41.0 in stage 10.0 (TID 58) +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO Executor: Finished task 36.0 in stage 10.0 (TID 53). 6606 bytes result sent to driver +26/04/01 08:46:42 INFO TaskSetManager: Starting task 42.0 in stage 10.0 (TID 59) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:46:42 INFO TaskSetManager: Finished task 36.0 in stage 10.0 (TID 53) in 7918 ms on 10.0.0.133 (executor driver) (35/208) +26/04/01 08:46:42 INFO Executor: Running task 42.0 in stage 10.0 (TID 59) +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO Executor: Finished task 35.0 in stage 10.0 (TID 52). 6606 bytes result sent to driver +26/04/01 08:46:42 INFO TaskSetManager: Starting task 43.0 in stage 10.0 (TID 60) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:46:42 INFO Executor: Finished task 33.0 in stage 10.0 (TID 50). 6606 bytes result sent to driver +26/04/01 08:46:42 INFO Executor: Running task 43.0 in stage 10.0 (TID 60) +26/04/01 08:46:42 INFO TaskSetManager: Starting task 44.0 in stage 10.0 (TID 61) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:42 INFO TaskSetManager: Finished task 35.0 in stage 10.0 (TID 52) in 7950 ms on 10.0.0.133 (executor driver) (36/208) +26/04/01 08:46:42 INFO Executor: Running task 44.0 in stage 10.0 (TID 61) +26/04/01 08:46:42 INFO TaskSetManager: Finished task 33.0 in stage 10.0 (TID 50) in 7957 ms on 10.0.0.133 (executor driver) (37/208) +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO Executor: Finished task 37.0 in stage 10.0 (TID 54). 6606 bytes result sent to driver +26/04/01 08:46:42 INFO TaskSetManager: Starting task 45.0 in stage 10.0 (TID 62) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:42 INFO TaskSetManager: Finished task 37.0 in stage 10.0 (TID 54) in 7944 ms on 10.0.0.133 (executor driver) (38/208) +26/04/01 08:46:42 INFO Executor: Running task 45.0 in stage 10.0 (TID 62) +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO Executor: Finished task 38.0 in stage 10.0 (TID 55). 6606 bytes result sent to driver +26/04/01 08:46:42 INFO TaskSetManager: Starting task 46.0 in stage 10.0 (TID 63) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:42 INFO Executor: Running task 46.0 in stage 10.0 (TID 63) +26/04/01 08:46:42 INFO TaskSetManager: Finished task 38.0 in stage 10.0 (TID 55) in 7917 ms on 10.0.0.133 (executor driver) (39/208) +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO Executor: Finished task 39.0 in stage 10.0 (TID 56). 6606 bytes result sent to driver +26/04/01 08:46:42 INFO TaskSetManager: Starting task 47.0 in stage 10.0 (TID 64) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:42 INFO Executor: Running task 47.0 in stage 10.0 (TID 64) +26/04/01 08:46:42 INFO TaskSetManager: Finished task 39.0 in stage 10.0 (TID 56) in 7964 ms on 10.0.0.133 (executor driver) (40/208) +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:49 INFO Executor: Finished task 45.0 in stage 10.0 (TID 62). 6606 bytes result sent to driver +26/04/01 08:46:49 INFO TaskSetManager: Starting task 48.0 in stage 10.0 (TID 65) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:46:49 INFO TaskSetManager: Finished task 45.0 in stage 10.0 (TID 62) in 7595 ms on 10.0.0.133 (executor driver) (41/208) +26/04/01 08:46:49 INFO Executor: Running task 48.0 in stage 10.0 (TID 65) +26/04/01 08:46:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:50 INFO Executor: Finished task 40.0 in stage 10.0 (TID 57). 6606 bytes result sent to driver +26/04/01 08:46:50 INFO TaskSetManager: Starting task 49.0 in stage 10.0 (TID 66) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:50 INFO TaskSetManager: Finished task 40.0 in stage 10.0 (TID 57) in 7902 ms on 10.0.0.133 (executor driver) (42/208) +26/04/01 08:46:50 INFO Executor: Running task 49.0 in stage 10.0 (TID 66) +26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:50 INFO Executor: Finished task 42.0 in stage 10.0 (TID 59). 6606 bytes result sent to driver +26/04/01 08:46:50 INFO TaskSetManager: Starting task 50.0 in stage 10.0 (TID 67) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:50 INFO TaskSetManager: Finished task 42.0 in stage 10.0 (TID 59) in 7890 ms on 10.0.0.133 (executor driver) (43/208) +26/04/01 08:46:50 INFO Executor: Running task 50.0 in stage 10.0 (TID 67) +26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:50 INFO Executor: Finished task 41.0 in stage 10.0 (TID 58). 6606 bytes result sent to driver +26/04/01 08:46:50 INFO TaskSetManager: Starting task 51.0 in stage 10.0 (TID 68) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:50 INFO TaskSetManager: Finished task 41.0 in stage 10.0 (TID 58) in 7905 ms on 10.0.0.133 (executor driver) (44/208) +26/04/01 08:46:50 INFO Executor: Running task 51.0 in stage 10.0 (TID 68) +26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:50 INFO Executor: Finished task 46.0 in stage 10.0 (TID 63). 6606 bytes result sent to driver +26/04/01 08:46:50 INFO TaskSetManager: Starting task 52.0 in stage 10.0 (TID 69) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:50 INFO Executor: Running task 52.0 in stage 10.0 (TID 69) +26/04/01 08:46:50 INFO TaskSetManager: Finished task 46.0 in stage 10.0 (TID 63) in 7883 ms on 10.0.0.133 (executor driver) (45/208) +26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:50 INFO Executor: Finished task 43.0 in stage 10.0 (TID 60). 6606 bytes result sent to driver +26/04/01 08:46:50 INFO TaskSetManager: Starting task 53.0 in stage 10.0 (TID 70) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:50 INFO Executor: Running task 53.0 in stage 10.0 (TID 70) +26/04/01 08:46:50 INFO TaskSetManager: Finished task 43.0 in stage 10.0 (TID 60) in 7899 ms on 10.0.0.133 (executor driver) (46/208) +26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:50 INFO Executor: Finished task 47.0 in stage 10.0 (TID 64). 6606 bytes result sent to driver +26/04/01 08:46:50 INFO TaskSetManager: Starting task 54.0 in stage 10.0 (TID 71) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:46:50 INFO Executor: Running task 54.0 in stage 10.0 (TID 71) +26/04/01 08:46:50 INFO TaskSetManager: Finished task 47.0 in stage 10.0 (TID 64) in 7875 ms on 10.0.0.133 (executor driver) (47/208) +26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:50 INFO Executor: Finished task 44.0 in stage 10.0 (TID 61). 6606 bytes result sent to driver +26/04/01 08:46:50 INFO TaskSetManager: Starting task 55.0 in stage 10.0 (TID 72) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:50 INFO TaskSetManager: Finished task 44.0 in stage 10.0 (TID 61) in 8174 ms on 10.0.0.133 (executor driver) (48/208) +26/04/01 08:46:50 INFO Executor: Running task 55.0 in stage 10.0 (TID 72) +26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:57 INFO Executor: Finished task 48.0 in stage 10.0 (TID 65). 6606 bytes result sent to driver +26/04/01 08:46:57 INFO TaskSetManager: Starting task 56.0 in stage 10.0 (TID 73) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:57 INFO TaskSetManager: Finished task 48.0 in stage 10.0 (TID 65) in 7957 ms on 10.0.0.133 (executor driver) (49/208) +26/04/01 08:46:57 INFO Executor: Running task 56.0 in stage 10.0 (TID 73) +26/04/01 08:46:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:58 INFO Executor: Finished task 49.0 in stage 10.0 (TID 66). 6606 bytes result sent to driver +26/04/01 08:46:58 INFO TaskSetManager: Starting task 57.0 in stage 10.0 (TID 74) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:58 INFO TaskSetManager: Finished task 49.0 in stage 10.0 (TID 66) in 7910 ms on 10.0.0.133 (executor driver) (50/208) +26/04/01 08:46:58 INFO Executor: Running task 57.0 in stage 10.0 (TID 74) +26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:58 INFO Executor: Finished task 50.0 in stage 10.0 (TID 67). 6606 bytes result sent to driver +26/04/01 08:46:58 INFO TaskSetManager: Starting task 58.0 in stage 10.0 (TID 75) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:58 INFO Executor: Running task 58.0 in stage 10.0 (TID 75) +26/04/01 08:46:58 INFO TaskSetManager: Finished task 50.0 in stage 10.0 (TID 67) in 7929 ms on 10.0.0.133 (executor driver) (51/208) +26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:58 INFO Executor: Finished task 51.0 in stage 10.0 (TID 68). 6606 bytes result sent to driver +26/04/01 08:46:58 INFO TaskSetManager: Starting task 59.0 in stage 10.0 (TID 76) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:58 INFO TaskSetManager: Finished task 51.0 in stage 10.0 (TID 68) in 7932 ms on 10.0.0.133 (executor driver) (52/208) +26/04/01 08:46:58 INFO Executor: Running task 59.0 in stage 10.0 (TID 76) +26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:58 INFO Executor: Finished task 53.0 in stage 10.0 (TID 70). 6606 bytes result sent to driver +26/04/01 08:46:58 INFO Executor: Finished task 52.0 in stage 10.0 (TID 69). 6606 bytes result sent to driver +26/04/01 08:46:58 INFO TaskSetManager: Starting task 60.0 in stage 10.0 (TID 77) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:46:58 INFO Executor: Running task 60.0 in stage 10.0 (TID 77) +26/04/01 08:46:58 INFO TaskSetManager: Starting task 61.0 in stage 10.0 (TID 78) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:46:58 INFO TaskSetManager: Finished task 53.0 in stage 10.0 (TID 70) in 7933 ms on 10.0.0.133 (executor driver) (53/208) +26/04/01 08:46:58 INFO Executor: Running task 61.0 in stage 10.0 (TID 78) +26/04/01 08:46:58 INFO TaskSetManager: Finished task 52.0 in stage 10.0 (TID 69) in 7944 ms on 10.0.0.133 (executor driver) (54/208) +26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:58 INFO Executor: Finished task 54.0 in stage 10.0 (TID 71). 6606 bytes result sent to driver +26/04/01 08:46:58 INFO TaskSetManager: Starting task 62.0 in stage 10.0 (TID 79) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:58 INFO TaskSetManager: Finished task 54.0 in stage 10.0 (TID 71) in 7959 ms on 10.0.0.133 (executor driver) (55/208) +26/04/01 08:46:58 INFO Executor: Running task 62.0 in stage 10.0 (TID 79) +26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:58 INFO Executor: Finished task 55.0 in stage 10.0 (TID 72). 6606 bytes result sent to driver +26/04/01 08:46:58 INFO TaskSetManager: Starting task 63.0 in stage 10.0 (TID 80) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:46:58 INFO TaskSetManager: Finished task 55.0 in stage 10.0 (TID 72) in 7933 ms on 10.0.0.133 (executor driver) (56/208) +26/04/01 08:46:58 INFO Executor: Running task 63.0 in stage 10.0 (TID 80) +26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:05 INFO Executor: Finished task 56.0 in stage 10.0 (TID 73). 6606 bytes result sent to driver +26/04/01 08:47:05 INFO TaskSetManager: Starting task 64.0 in stage 10.0 (TID 81) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:05 INFO TaskSetManager: Finished task 56.0 in stage 10.0 (TID 73) in 7952 ms on 10.0.0.133 (executor driver) (57/208) +26/04/01 08:47:05 INFO Executor: Running task 64.0 in stage 10.0 (TID 81) +26/04/01 08:47:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:06 INFO Executor: Finished task 57.0 in stage 10.0 (TID 74). 6606 bytes result sent to driver +26/04/01 08:47:06 INFO TaskSetManager: Starting task 65.0 in stage 10.0 (TID 82) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:06 INFO TaskSetManager: Finished task 57.0 in stage 10.0 (TID 74) in 7949 ms on 10.0.0.133 (executor driver) (58/208) +26/04/01 08:47:06 INFO Executor: Running task 65.0 in stage 10.0 (TID 82) +26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:06 INFO Executor: Finished task 59.0 in stage 10.0 (TID 76). 6606 bytes result sent to driver +26/04/01 08:47:06 INFO TaskSetManager: Starting task 66.0 in stage 10.0 (TID 83) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:47:06 INFO TaskSetManager: Finished task 59.0 in stage 10.0 (TID 76) in 7902 ms on 10.0.0.133 (executor driver) (59/208) +26/04/01 08:47:06 INFO Executor: Running task 66.0 in stage 10.0 (TID 83) +26/04/01 08:47:06 INFO Executor: Finished task 58.0 in stage 10.0 (TID 75). 6606 bytes result sent to driver +26/04/01 08:47:06 INFO TaskSetManager: Starting task 67.0 in stage 10.0 (TID 84) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:47:06 INFO TaskSetManager: Finished task 58.0 in stage 10.0 (TID 75) in 7918 ms on 10.0.0.133 (executor driver) (60/208) +26/04/01 08:47:06 INFO Executor: Running task 67.0 in stage 10.0 (TID 84) +26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:06 INFO Executor: Finished task 60.0 in stage 10.0 (TID 77). 6606 bytes result sent to driver +26/04/01 08:47:06 INFO TaskSetManager: Starting task 68.0 in stage 10.0 (TID 85) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:06 INFO TaskSetManager: Finished task 60.0 in stage 10.0 (TID 77) in 7937 ms on 10.0.0.133 (executor driver) (61/208) +26/04/01 08:47:06 INFO Executor: Running task 68.0 in stage 10.0 (TID 85) +26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:06 INFO Executor: Finished task 61.0 in stage 10.0 (TID 78). 6606 bytes result sent to driver +26/04/01 08:47:06 INFO TaskSetManager: Starting task 69.0 in stage 10.0 (TID 86) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:06 INFO TaskSetManager: Finished task 61.0 in stage 10.0 (TID 78) in 7949 ms on 10.0.0.133 (executor driver) (62/208) +26/04/01 08:47:06 INFO Executor: Running task 69.0 in stage 10.0 (TID 86) +26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:06 INFO Executor: Finished task 62.0 in stage 10.0 (TID 79). 6606 bytes result sent to driver +26/04/01 08:47:06 INFO TaskSetManager: Starting task 70.0 in stage 10.0 (TID 87) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:06 INFO TaskSetManager: Finished task 62.0 in stage 10.0 (TID 79) in 7950 ms on 10.0.0.133 (executor driver) (63/208) +26/04/01 08:47:06 INFO Executor: Running task 70.0 in stage 10.0 (TID 87) +26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:06 INFO Executor: Finished task 63.0 in stage 10.0 (TID 80). 6606 bytes result sent to driver +26/04/01 08:47:06 INFO TaskSetManager: Starting task 71.0 in stage 10.0 (TID 88) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:06 INFO TaskSetManager: Finished task 63.0 in stage 10.0 (TID 80) in 7929 ms on 10.0.0.133 (executor driver) (64/208) +26/04/01 08:47:06 INFO Executor: Running task 71.0 in stage 10.0 (TID 88) +26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:13 INFO Executor: Finished task 64.0 in stage 10.0 (TID 81). 6606 bytes result sent to driver +26/04/01 08:47:13 INFO TaskSetManager: Starting task 72.0 in stage 10.0 (TID 89) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:47:13 INFO TaskSetManager: Finished task 64.0 in stage 10.0 (TID 81) in 7953 ms on 10.0.0.133 (executor driver) (65/208) +26/04/01 08:47:13 INFO Executor: Running task 72.0 in stage 10.0 (TID 89) +26/04/01 08:47:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:14 INFO Executor: Finished task 65.0 in stage 10.0 (TID 82). 6606 bytes result sent to driver +26/04/01 08:47:14 INFO TaskSetManager: Starting task 73.0 in stage 10.0 (TID 90) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:47:14 INFO TaskSetManager: Finished task 65.0 in stage 10.0 (TID 82) in 7940 ms on 10.0.0.133 (executor driver) (66/208) +26/04/01 08:47:14 INFO Executor: Running task 73.0 in stage 10.0 (TID 90) +26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:14 INFO Executor: Finished task 67.0 in stage 10.0 (TID 84). 6606 bytes result sent to driver +26/04/01 08:47:14 INFO TaskSetManager: Starting task 74.0 in stage 10.0 (TID 91) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:14 INFO Executor: Running task 74.0 in stage 10.0 (TID 91) +26/04/01 08:47:14 INFO TaskSetManager: Finished task 67.0 in stage 10.0 (TID 84) in 7960 ms on 10.0.0.133 (executor driver) (67/208) +26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:14 INFO Executor: Finished task 66.0 in stage 10.0 (TID 83). 6606 bytes result sent to driver +26/04/01 08:47:14 INFO TaskSetManager: Starting task 75.0 in stage 10.0 (TID 92) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:14 INFO TaskSetManager: Finished task 66.0 in stage 10.0 (TID 83) in 7968 ms on 10.0.0.133 (executor driver) (68/208) +26/04/01 08:47:14 INFO Executor: Running task 75.0 in stage 10.0 (TID 92) +26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:14 INFO Executor: Finished task 68.0 in stage 10.0 (TID 85). 6606 bytes result sent to driver +26/04/01 08:47:14 INFO TaskSetManager: Starting task 76.0 in stage 10.0 (TID 93) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:14 INFO TaskSetManager: Finished task 68.0 in stage 10.0 (TID 85) in 7934 ms on 10.0.0.133 (executor driver) (69/208) +26/04/01 08:47:14 INFO Executor: Running task 76.0 in stage 10.0 (TID 93) +26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:14 INFO Executor: Finished task 69.0 in stage 10.0 (TID 86). 6606 bytes result sent to driver +26/04/01 08:47:14 INFO TaskSetManager: Starting task 77.0 in stage 10.0 (TID 94) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:14 INFO TaskSetManager: Finished task 69.0 in stage 10.0 (TID 86) in 7946 ms on 10.0.0.133 (executor driver) (70/208) +26/04/01 08:47:14 INFO Executor: Running task 77.0 in stage 10.0 (TID 94) +26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:14 INFO Executor: Finished task 70.0 in stage 10.0 (TID 87). 6606 bytes result sent to driver +26/04/01 08:47:14 INFO TaskSetManager: Starting task 78.0 in stage 10.0 (TID 95) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:47:14 INFO TaskSetManager: Finished task 70.0 in stage 10.0 (TID 87) in 7929 ms on 10.0.0.133 (executor driver) (71/208) +26/04/01 08:47:14 INFO Executor: Running task 78.0 in stage 10.0 (TID 95) +26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:14 INFO Executor: Finished task 71.0 in stage 10.0 (TID 88). 6606 bytes result sent to driver +26/04/01 08:47:14 INFO TaskSetManager: Starting task 79.0 in stage 10.0 (TID 96) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:14 INFO TaskSetManager: Finished task 71.0 in stage 10.0 (TID 88) in 7949 ms on 10.0.0.133 (executor driver) (72/208) +26/04/01 08:47:14 INFO Executor: Running task 79.0 in stage 10.0 (TID 96) +26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:21 INFO Executor: Finished task 72.0 in stage 10.0 (TID 89). 6606 bytes result sent to driver +26/04/01 08:47:21 INFO TaskSetManager: Starting task 80.0 in stage 10.0 (TID 97) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:21 INFO Executor: Running task 80.0 in stage 10.0 (TID 97) +26/04/01 08:47:21 INFO TaskSetManager: Finished task 72.0 in stage 10.0 (TID 89) in 7943 ms on 10.0.0.133 (executor driver) (73/208) +26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:21 INFO Executor: Finished task 74.0 in stage 10.0 (TID 91). 6606 bytes result sent to driver +26/04/01 08:47:21 INFO TaskSetManager: Starting task 81.0 in stage 10.0 (TID 98) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:21 INFO TaskSetManager: Finished task 74.0 in stage 10.0 (TID 91) in 7873 ms on 10.0.0.133 (executor driver) (74/208) +26/04/01 08:47:21 INFO Executor: Running task 81.0 in stage 10.0 (TID 98) +26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:21 INFO Executor: Finished task 73.0 in stage 10.0 (TID 90). 6606 bytes result sent to driver +26/04/01 08:47:21 INFO TaskSetManager: Starting task 82.0 in stage 10.0 (TID 99) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:21 INFO TaskSetManager: Finished task 73.0 in stage 10.0 (TID 90) in 7952 ms on 10.0.0.133 (executor driver) (75/208) +26/04/01 08:47:21 INFO Executor: Running task 82.0 in stage 10.0 (TID 99) +26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:21 INFO Executor: Finished task 75.0 in stage 10.0 (TID 92). 6606 bytes result sent to driver +26/04/01 08:47:21 INFO TaskSetManager: Starting task 83.0 in stage 10.0 (TID 100) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:21 INFO TaskSetManager: Finished task 75.0 in stage 10.0 (TID 92) in 7945 ms on 10.0.0.133 (executor driver) (76/208) +26/04/01 08:47:21 INFO Executor: Running task 83.0 in stage 10.0 (TID 100) +26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:22 INFO Executor: Finished task 76.0 in stage 10.0 (TID 93). 6649 bytes result sent to driver +26/04/01 08:47:22 INFO TaskSetManager: Starting task 84.0 in stage 10.0 (TID 101) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:47:22 INFO TaskSetManager: Finished task 76.0 in stage 10.0 (TID 93) in 7951 ms on 10.0.0.133 (executor driver) (77/208) +26/04/01 08:47:22 INFO Executor: Running task 84.0 in stage 10.0 (TID 101) +26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:22 INFO Executor: Finished task 77.0 in stage 10.0 (TID 94). 6606 bytes result sent to driver +26/04/01 08:47:22 INFO TaskSetManager: Starting task 85.0 in stage 10.0 (TID 102) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:22 INFO Executor: Running task 85.0 in stage 10.0 (TID 102) +26/04/01 08:47:22 INFO TaskSetManager: Finished task 77.0 in stage 10.0 (TID 94) in 7948 ms on 10.0.0.133 (executor driver) (78/208) +26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:22 INFO Executor: Finished task 78.0 in stage 10.0 (TID 95). 6606 bytes result sent to driver +26/04/01 08:47:22 INFO TaskSetManager: Starting task 86.0 in stage 10.0 (TID 103) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:22 INFO TaskSetManager: Finished task 78.0 in stage 10.0 (TID 95) in 7941 ms on 10.0.0.133 (executor driver) (79/208) +26/04/01 08:47:22 INFO Executor: Running task 86.0 in stage 10.0 (TID 103) +26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:22 INFO Executor: Finished task 79.0 in stage 10.0 (TID 96). 6606 bytes result sent to driver +26/04/01 08:47:22 INFO TaskSetManager: Starting task 87.0 in stage 10.0 (TID 104) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:22 INFO TaskSetManager: Finished task 79.0 in stage 10.0 (TID 96) in 7910 ms on 10.0.0.133 (executor driver) (80/208) +26/04/01 08:47:22 INFO Executor: Running task 87.0 in stage 10.0 (TID 104) +26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:29 INFO Executor: Finished task 80.0 in stage 10.0 (TID 97). 6606 bytes result sent to driver +26/04/01 08:47:29 INFO TaskSetManager: Starting task 88.0 in stage 10.0 (TID 105) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:29 INFO TaskSetManager: Finished task 80.0 in stage 10.0 (TID 97) in 7929 ms on 10.0.0.133 (executor driver) (81/208) +26/04/01 08:47:29 INFO Executor: Running task 88.0 in stage 10.0 (TID 105) +26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:29 INFO Executor: Finished task 81.0 in stage 10.0 (TID 98). 6606 bytes result sent to driver +26/04/01 08:47:29 INFO TaskSetManager: Starting task 89.0 in stage 10.0 (TID 106) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:29 INFO TaskSetManager: Finished task 81.0 in stage 10.0 (TID 98) in 7941 ms on 10.0.0.133 (executor driver) (82/208) +26/04/01 08:47:29 INFO Executor: Running task 89.0 in stage 10.0 (TID 106) +26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:29 INFO Executor: Finished task 82.0 in stage 10.0 (TID 99). 6606 bytes result sent to driver +26/04/01 08:47:29 INFO TaskSetManager: Starting task 90.0 in stage 10.0 (TID 107) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:47:29 INFO TaskSetManager: Finished task 82.0 in stage 10.0 (TID 99) in 7938 ms on 10.0.0.133 (executor driver) (83/208) +26/04/01 08:47:29 INFO Executor: Running task 90.0 in stage 10.0 (TID 107) +26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:29 INFO Executor: Finished task 83.0 in stage 10.0 (TID 100). 6606 bytes result sent to driver +26/04/01 08:47:29 INFO TaskSetManager: Starting task 91.0 in stage 10.0 (TID 108) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:29 INFO TaskSetManager: Finished task 83.0 in stage 10.0 (TID 100) in 7924 ms on 10.0.0.133 (executor driver) (84/208) +26/04/01 08:47:29 INFO Executor: Running task 91.0 in stage 10.0 (TID 108) +26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:29 INFO Executor: Finished task 84.0 in stage 10.0 (TID 101). 6606 bytes result sent to driver +26/04/01 08:47:29 INFO TaskSetManager: Starting task 92.0 in stage 10.0 (TID 109) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:29 INFO TaskSetManager: Finished task 84.0 in stage 10.0 (TID 101) in 7917 ms on 10.0.0.133 (executor driver) (85/208) +26/04/01 08:47:29 INFO Executor: Running task 92.0 in stage 10.0 (TID 109) +26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:29 INFO Executor: Finished task 85.0 in stage 10.0 (TID 102). 6606 bytes result sent to driver +26/04/01 08:47:29 INFO TaskSetManager: Starting task 93.0 in stage 10.0 (TID 110) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:29 INFO Executor: Running task 93.0 in stage 10.0 (TID 110) +26/04/01 08:47:29 INFO TaskSetManager: Finished task 85.0 in stage 10.0 (TID 102) in 7940 ms on 10.0.0.133 (executor driver) (86/208) +26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:30 INFO Executor: Finished task 86.0 in stage 10.0 (TID 103). 6606 bytes result sent to driver +26/04/01 08:47:30 INFO TaskSetManager: Starting task 94.0 in stage 10.0 (TID 111) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:30 INFO TaskSetManager: Finished task 86.0 in stage 10.0 (TID 103) in 7933 ms on 10.0.0.133 (executor driver) (87/208) +26/04/01 08:47:30 INFO Executor: Running task 94.0 in stage 10.0 (TID 111) +26/04/01 08:47:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:30 INFO Executor: Finished task 87.0 in stage 10.0 (TID 104). 6606 bytes result sent to driver +26/04/01 08:47:30 INFO TaskSetManager: Starting task 95.0 in stage 10.0 (TID 112) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:30 INFO TaskSetManager: Finished task 87.0 in stage 10.0 (TID 104) in 7936 ms on 10.0.0.133 (executor driver) (88/208) +26/04/01 08:47:30 INFO Executor: Running task 95.0 in stage 10.0 (TID 112) +26/04/01 08:47:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:37 INFO Executor: Finished task 91.0 in stage 10.0 (TID 108). 6606 bytes result sent to driver +26/04/01 08:47:37 INFO TaskSetManager: Starting task 96.0 in stage 10.0 (TID 113) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:47:37 INFO TaskSetManager: Finished task 91.0 in stage 10.0 (TID 108) in 7659 ms on 10.0.0.133 (executor driver) (89/208) +26/04/01 08:47:37 INFO Executor: Running task 96.0 in stage 10.0 (TID 113) +26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:37 INFO Executor: Finished task 88.0 in stage 10.0 (TID 105). 6606 bytes result sent to driver +26/04/01 08:47:37 INFO TaskSetManager: Starting task 97.0 in stage 10.0 (TID 114) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:47:37 INFO TaskSetManager: Finished task 88.0 in stage 10.0 (TID 105) in 7941 ms on 10.0.0.133 (executor driver) (90/208) +26/04/01 08:47:37 INFO Executor: Running task 97.0 in stage 10.0 (TID 114) +26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:37 INFO Executor: Finished task 89.0 in stage 10.0 (TID 106). 6606 bytes result sent to driver +26/04/01 08:47:37 INFO TaskSetManager: Starting task 98.0 in stage 10.0 (TID 115) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:37 INFO TaskSetManager: Finished task 89.0 in stage 10.0 (TID 106) in 7958 ms on 10.0.0.133 (executor driver) (91/208) +26/04/01 08:47:37 INFO Executor: Running task 98.0 in stage 10.0 (TID 115) +26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:37 INFO Executor: Finished task 92.0 in stage 10.0 (TID 109). 6606 bytes result sent to driver +26/04/01 08:47:37 INFO TaskSetManager: Starting task 99.0 in stage 10.0 (TID 116) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:37 INFO TaskSetManager: Finished task 92.0 in stage 10.0 (TID 109) in 7909 ms on 10.0.0.133 (executor driver) (92/208) +26/04/01 08:47:37 INFO Executor: Running task 99.0 in stage 10.0 (TID 116) +26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:37 INFO Executor: Finished task 93.0 in stage 10.0 (TID 110). 6606 bytes result sent to driver +26/04/01 08:47:37 INFO TaskSetManager: Starting task 100.0 in stage 10.0 (TID 117) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:37 INFO TaskSetManager: Finished task 93.0 in stage 10.0 (TID 110) in 7943 ms on 10.0.0.133 (executor driver) (93/208) +26/04/01 08:47:37 INFO Executor: Running task 100.0 in stage 10.0 (TID 117) +26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:37 INFO Executor: Finished task 94.0 in stage 10.0 (TID 111). 6606 bytes result sent to driver +26/04/01 08:47:37 INFO TaskSetManager: Starting task 101.0 in stage 10.0 (TID 118) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:37 INFO TaskSetManager: Finished task 94.0 in stage 10.0 (TID 111) in 7911 ms on 10.0.0.133 (executor driver) (94/208) +26/04/01 08:47:37 INFO Executor: Running task 101.0 in stage 10.0 (TID 118) +26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:38 INFO Executor: Finished task 90.0 in stage 10.0 (TID 107). 6606 bytes result sent to driver +26/04/01 08:47:38 INFO TaskSetManager: Starting task 102.0 in stage 10.0 (TID 119) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:47:38 INFO TaskSetManager: Finished task 90.0 in stage 10.0 (TID 107) in 8209 ms on 10.0.0.133 (executor driver) (95/208) +26/04/01 08:47:38 INFO Executor: Running task 102.0 in stage 10.0 (TID 119) +26/04/01 08:47:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:38 INFO Executor: Finished task 95.0 in stage 10.0 (TID 112). 6606 bytes result sent to driver +26/04/01 08:47:38 INFO TaskSetManager: Starting task 103.0 in stage 10.0 (TID 120) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:38 INFO TaskSetManager: Finished task 95.0 in stage 10.0 (TID 112) in 7936 ms on 10.0.0.133 (executor driver) (96/208) +26/04/01 08:47:38 INFO Executor: Running task 103.0 in stage 10.0 (TID 120) +26/04/01 08:47:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:45 INFO Executor: Finished task 96.0 in stage 10.0 (TID 113). 6606 bytes result sent to driver +26/04/01 08:47:45 INFO TaskSetManager: Starting task 104.0 in stage 10.0 (TID 121) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:45 INFO TaskSetManager: Finished task 96.0 in stage 10.0 (TID 113) in 7987 ms on 10.0.0.133 (executor driver) (97/208) +26/04/01 08:47:45 INFO Executor: Running task 104.0 in stage 10.0 (TID 121) +26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:45 INFO Executor: Finished task 97.0 in stage 10.0 (TID 114). 6606 bytes result sent to driver +26/04/01 08:47:45 INFO TaskSetManager: Starting task 105.0 in stage 10.0 (TID 122) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:45 INFO TaskSetManager: Finished task 97.0 in stage 10.0 (TID 114) in 7959 ms on 10.0.0.133 (executor driver) (98/208) +26/04/01 08:47:45 INFO Executor: Running task 105.0 in stage 10.0 (TID 122) +26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:45 INFO Executor: Finished task 98.0 in stage 10.0 (TID 115). 6606 bytes result sent to driver +26/04/01 08:47:45 INFO TaskSetManager: Starting task 106.0 in stage 10.0 (TID 123) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:45 INFO Executor: Running task 106.0 in stage 10.0 (TID 123) +26/04/01 08:47:45 INFO TaskSetManager: Finished task 98.0 in stage 10.0 (TID 115) in 7930 ms on 10.0.0.133 (executor driver) (99/208) +26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:45 INFO Executor: Finished task 99.0 in stage 10.0 (TID 116). 6606 bytes result sent to driver +26/04/01 08:47:45 INFO TaskSetManager: Starting task 107.0 in stage 10.0 (TID 124) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:45 INFO TaskSetManager: Finished task 99.0 in stage 10.0 (TID 116) in 7946 ms on 10.0.0.133 (executor driver) (100/208) +26/04/01 08:47:45 INFO Executor: Running task 107.0 in stage 10.0 (TID 124) +26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:45 INFO Executor: Finished task 100.0 in stage 10.0 (TID 117). 6606 bytes result sent to driver +26/04/01 08:47:45 INFO TaskSetManager: Starting task 108.0 in stage 10.0 (TID 125) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:47:45 INFO TaskSetManager: Finished task 100.0 in stage 10.0 (TID 117) in 7930 ms on 10.0.0.133 (executor driver) (101/208) +26/04/01 08:47:45 INFO Executor: Running task 108.0 in stage 10.0 (TID 125) +26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:45 INFO Executor: Finished task 101.0 in stage 10.0 (TID 118). 6606 bytes result sent to driver +26/04/01 08:47:45 INFO TaskSetManager: Starting task 109.0 in stage 10.0 (TID 126) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:47:45 INFO TaskSetManager: Finished task 101.0 in stage 10.0 (TID 118) in 7940 ms on 10.0.0.133 (executor driver) (102/208) +26/04/01 08:47:45 INFO Executor: Running task 109.0 in stage 10.0 (TID 126) +26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:46 INFO Executor: Finished task 102.0 in stage 10.0 (TID 119). 6606 bytes result sent to driver +26/04/01 08:47:46 INFO TaskSetManager: Starting task 110.0 in stage 10.0 (TID 127) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:46 INFO TaskSetManager: Finished task 102.0 in stage 10.0 (TID 119) in 7942 ms on 10.0.0.133 (executor driver) (103/208) +26/04/01 08:47:46 INFO Executor: Running task 110.0 in stage 10.0 (TID 127) +26/04/01 08:47:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:46 INFO Executor: Finished task 103.0 in stage 10.0 (TID 120). 6606 bytes result sent to driver +26/04/01 08:47:46 INFO TaskSetManager: Starting task 111.0 in stage 10.0 (TID 128) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:46 INFO TaskSetManager: Finished task 103.0 in stage 10.0 (TID 120) in 7947 ms on 10.0.0.133 (executor driver) (104/208) +26/04/01 08:47:46 INFO Executor: Running task 111.0 in stage 10.0 (TID 128) +26/04/01 08:47:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:53 INFO Executor: Finished task 104.0 in stage 10.0 (TID 121). 6606 bytes result sent to driver +26/04/01 08:47:53 INFO TaskSetManager: Starting task 112.0 in stage 10.0 (TID 129) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:53 INFO TaskSetManager: Finished task 104.0 in stage 10.0 (TID 121) in 7931 ms on 10.0.0.133 (executor driver) (105/208) +26/04/01 08:47:53 INFO Executor: Running task 112.0 in stage 10.0 (TID 129) +26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:53 INFO Executor: Finished task 105.0 in stage 10.0 (TID 122). 6606 bytes result sent to driver +26/04/01 08:47:53 INFO TaskSetManager: Starting task 113.0 in stage 10.0 (TID 130) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:53 INFO TaskSetManager: Finished task 105.0 in stage 10.0 (TID 122) in 7953 ms on 10.0.0.133 (executor driver) (106/208) +26/04/01 08:47:53 INFO Executor: Running task 113.0 in stage 10.0 (TID 130) +26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:53 INFO Executor: Finished task 106.0 in stage 10.0 (TID 123). 6606 bytes result sent to driver +26/04/01 08:47:53 INFO TaskSetManager: Starting task 114.0 in stage 10.0 (TID 131) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:47:53 INFO TaskSetManager: Finished task 106.0 in stage 10.0 (TID 123) in 7951 ms on 10.0.0.133 (executor driver) (107/208) +26/04/01 08:47:53 INFO Executor: Running task 114.0 in stage 10.0 (TID 131) +26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:53 INFO Executor: Finished task 107.0 in stage 10.0 (TID 124). 6606 bytes result sent to driver +26/04/01 08:47:53 INFO TaskSetManager: Starting task 115.0 in stage 10.0 (TID 132) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:47:53 INFO Executor: Running task 115.0 in stage 10.0 (TID 132) +26/04/01 08:47:53 INFO TaskSetManager: Finished task 107.0 in stage 10.0 (TID 124) in 7951 ms on 10.0.0.133 (executor driver) (108/208) +26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:53 INFO Executor: Finished task 108.0 in stage 10.0 (TID 125). 6606 bytes result sent to driver +26/04/01 08:47:53 INFO TaskSetManager: Starting task 116.0 in stage 10.0 (TID 133) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:53 INFO TaskSetManager: Finished task 108.0 in stage 10.0 (TID 125) in 7926 ms on 10.0.0.133 (executor driver) (109/208) +26/04/01 08:47:53 INFO Executor: Running task 116.0 in stage 10.0 (TID 133) +26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:53 INFO Executor: Finished task 109.0 in stage 10.0 (TID 126). 6606 bytes result sent to driver +26/04/01 08:47:53 INFO TaskSetManager: Starting task 117.0 in stage 10.0 (TID 134) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:53 INFO Executor: Running task 117.0 in stage 10.0 (TID 134) +26/04/01 08:47:53 INFO TaskSetManager: Finished task 109.0 in stage 10.0 (TID 126) in 7933 ms on 10.0.0.133 (executor driver) (110/208) +26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:53 INFO Executor: Finished task 110.0 in stage 10.0 (TID 127). 6606 bytes result sent to driver +26/04/01 08:47:53 INFO TaskSetManager: Starting task 118.0 in stage 10.0 (TID 135) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:53 INFO Executor: Running task 118.0 in stage 10.0 (TID 135) +26/04/01 08:47:53 INFO TaskSetManager: Finished task 110.0 in stage 10.0 (TID 127) in 7919 ms on 10.0.0.133 (executor driver) (111/208) +26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:54 INFO Executor: Finished task 111.0 in stage 10.0 (TID 128). 6606 bytes result sent to driver +26/04/01 08:47:54 INFO TaskSetManager: Starting task 119.0 in stage 10.0 (TID 136) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:47:54 INFO Executor: Running task 119.0 in stage 10.0 (TID 136) +26/04/01 08:47:54 INFO TaskSetManager: Finished task 111.0 in stage 10.0 (TID 128) in 7942 ms on 10.0.0.133 (executor driver) (112/208) +26/04/01 08:47:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:47:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO Executor: Finished task 112.0 in stage 10.0 (TID 129). 6563 bytes result sent to driver +26/04/01 08:48:01 INFO TaskSetManager: Starting task 120.0 in stage 10.0 (TID 137) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:48:01 INFO Executor: Running task 120.0 in stage 10.0 (TID 137) +26/04/01 08:48:01 INFO TaskSetManager: Finished task 112.0 in stage 10.0 (TID 129) in 7974 ms on 10.0.0.133 (executor driver) (113/208) +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO Executor: Finished task 113.0 in stage 10.0 (TID 130). 6563 bytes result sent to driver +26/04/01 08:48:01 INFO TaskSetManager: Starting task 121.0 in stage 10.0 (TID 138) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:01 INFO TaskSetManager: Finished task 113.0 in stage 10.0 (TID 130) in 7957 ms on 10.0.0.133 (executor driver) (114/208) +26/04/01 08:48:01 INFO Executor: Running task 121.0 in stage 10.0 (TID 138) +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO Executor: Finished task 114.0 in stage 10.0 (TID 131). 6563 bytes result sent to driver +26/04/01 08:48:01 INFO TaskSetManager: Starting task 122.0 in stage 10.0 (TID 139) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:01 INFO TaskSetManager: Finished task 114.0 in stage 10.0 (TID 131) in 7939 ms on 10.0.0.133 (executor driver) (115/208) +26/04/01 08:48:01 INFO Executor: Running task 122.0 in stage 10.0 (TID 139) +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO Executor: Finished task 115.0 in stage 10.0 (TID 132). 6563 bytes result sent to driver +26/04/01 08:48:01 INFO TaskSetManager: Starting task 123.0 in stage 10.0 (TID 140) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:01 INFO TaskSetManager: Finished task 115.0 in stage 10.0 (TID 132) in 7941 ms on 10.0.0.133 (executor driver) (116/208) +26/04/01 08:48:01 INFO Executor: Running task 123.0 in stage 10.0 (TID 140) +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO Executor: Finished task 116.0 in stage 10.0 (TID 133). 6563 bytes result sent to driver +26/04/01 08:48:01 INFO TaskSetManager: Starting task 124.0 in stage 10.0 (TID 141) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:01 INFO Executor: Running task 124.0 in stage 10.0 (TID 141) +26/04/01 08:48:01 INFO TaskSetManager: Finished task 116.0 in stage 10.0 (TID 133) in 7949 ms on 10.0.0.133 (executor driver) (117/208) +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO Executor: Finished task 117.0 in stage 10.0 (TID 134). 6563 bytes result sent to driver +26/04/01 08:48:01 INFO TaskSetManager: Starting task 125.0 in stage 10.0 (TID 142) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:01 INFO Executor: Running task 125.0 in stage 10.0 (TID 142) +26/04/01 08:48:01 INFO TaskSetManager: Finished task 117.0 in stage 10.0 (TID 134) in 7938 ms on 10.0.0.133 (executor driver) (118/208) +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO Executor: Finished task 118.0 in stage 10.0 (TID 135). 6563 bytes result sent to driver +26/04/01 08:48:01 INFO TaskSetManager: Starting task 126.0 in stage 10.0 (TID 143) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:48:01 INFO TaskSetManager: Finished task 118.0 in stage 10.0 (TID 135) in 7955 ms on 10.0.0.133 (executor driver) (119/208) +26/04/01 08:48:01 INFO Executor: Running task 126.0 in stage 10.0 (TID 143) +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO Executor: Finished task 119.0 in stage 10.0 (TID 136). 6563 bytes result sent to driver +26/04/01 08:48:01 INFO TaskSetManager: Starting task 127.0 in stage 10.0 (TID 144) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:01 INFO TaskSetManager: Finished task 119.0 in stage 10.0 (TID 136) in 7948 ms on 10.0.0.133 (executor driver) (120/208) +26/04/01 08:48:01 INFO Executor: Running task 127.0 in stage 10.0 (TID 144) +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO Executor: Finished task 120.0 in stage 10.0 (TID 137). 6606 bytes result sent to driver +26/04/01 08:48:09 INFO TaskSetManager: Starting task 128.0 in stage 10.0 (TID 145) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:09 INFO Executor: Running task 128.0 in stage 10.0 (TID 145) +26/04/01 08:48:09 INFO TaskSetManager: Finished task 120.0 in stage 10.0 (TID 137) in 8001 ms on 10.0.0.133 (executor driver) (121/208) +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO Executor: Finished task 121.0 in stage 10.0 (TID 138). 6606 bytes result sent to driver +26/04/01 08:48:09 INFO TaskSetManager: Starting task 129.0 in stage 10.0 (TID 146) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:09 INFO TaskSetManager: Finished task 121.0 in stage 10.0 (TID 138) in 7974 ms on 10.0.0.133 (executor driver) (122/208) +26/04/01 08:48:09 INFO Executor: Running task 129.0 in stage 10.0 (TID 146) +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO Executor: Finished task 122.0 in stage 10.0 (TID 139). 6606 bytes result sent to driver +26/04/01 08:48:09 INFO TaskSetManager: Starting task 130.0 in stage 10.0 (TID 147) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:09 INFO TaskSetManager: Finished task 122.0 in stage 10.0 (TID 139) in 7965 ms on 10.0.0.133 (executor driver) (123/208) +26/04/01 08:48:09 INFO Executor: Running task 130.0 in stage 10.0 (TID 147) +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO Executor: Finished task 123.0 in stage 10.0 (TID 140). 6606 bytes result sent to driver +26/04/01 08:48:09 INFO TaskSetManager: Starting task 131.0 in stage 10.0 (TID 148) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:09 INFO TaskSetManager: Finished task 123.0 in stage 10.0 (TID 140) in 7966 ms on 10.0.0.133 (executor driver) (124/208) +26/04/01 08:48:09 INFO Executor: Running task 131.0 in stage 10.0 (TID 148) +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO Executor: Finished task 124.0 in stage 10.0 (TID 141). 6606 bytes result sent to driver +26/04/01 08:48:09 INFO TaskSetManager: Starting task 132.0 in stage 10.0 (TID 149) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:48:09 INFO TaskSetManager: Finished task 124.0 in stage 10.0 (TID 141) in 7986 ms on 10.0.0.133 (executor driver) (125/208) +26/04/01 08:48:09 INFO Executor: Running task 132.0 in stage 10.0 (TID 149) +26/04/01 08:48:09 INFO Executor: Finished task 125.0 in stage 10.0 (TID 142). 6606 bytes result sent to driver +26/04/01 08:48:09 INFO TaskSetManager: Starting task 133.0 in stage 10.0 (TID 150) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:09 INFO TaskSetManager: Finished task 125.0 in stage 10.0 (TID 142) in 7975 ms on 10.0.0.133 (executor driver) (126/208) +26/04/01 08:48:09 INFO Executor: Running task 133.0 in stage 10.0 (TID 150) +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO Executor: Finished task 126.0 in stage 10.0 (TID 143). 6606 bytes result sent to driver +26/04/01 08:48:09 INFO TaskSetManager: Starting task 134.0 in stage 10.0 (TID 151) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:09 INFO Executor: Running task 134.0 in stage 10.0 (TID 151) +26/04/01 08:48:09 INFO TaskSetManager: Finished task 126.0 in stage 10.0 (TID 143) in 7953 ms on 10.0.0.133 (executor driver) (127/208) +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO Executor: Finished task 127.0 in stage 10.0 (TID 144). 6606 bytes result sent to driver +26/04/01 08:48:09 INFO TaskSetManager: Starting task 135.0 in stage 10.0 (TID 152) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:09 INFO TaskSetManager: Finished task 127.0 in stage 10.0 (TID 144) in 7971 ms on 10.0.0.133 (executor driver) (128/208) +26/04/01 08:48:09 INFO Executor: Running task 135.0 in stage 10.0 (TID 152) +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO Executor: Finished task 129.0 in stage 10.0 (TID 146). 6606 bytes result sent to driver +26/04/01 08:48:17 INFO TaskSetManager: Starting task 136.0 in stage 10.0 (TID 153) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:17 INFO TaskSetManager: Finished task 129.0 in stage 10.0 (TID 146) in 7965 ms on 10.0.0.133 (executor driver) (129/208) +26/04/01 08:48:17 INFO Executor: Running task 136.0 in stage 10.0 (TID 153) +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO Executor: Finished task 130.0 in stage 10.0 (TID 147). 6606 bytes result sent to driver +26/04/01 08:48:17 INFO TaskSetManager: Starting task 137.0 in stage 10.0 (TID 154) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:17 INFO TaskSetManager: Finished task 130.0 in stage 10.0 (TID 147) in 7955 ms on 10.0.0.133 (executor driver) (130/208) +26/04/01 08:48:17 INFO Executor: Running task 137.0 in stage 10.0 (TID 154) +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO Executor: Finished task 131.0 in stage 10.0 (TID 148). 6606 bytes result sent to driver +26/04/01 08:48:17 INFO TaskSetManager: Starting task 138.0 in stage 10.0 (TID 155) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:48:17 INFO TaskSetManager: Finished task 131.0 in stage 10.0 (TID 148) in 7948 ms on 10.0.0.133 (executor driver) (131/208) +26/04/01 08:48:17 INFO Executor: Running task 138.0 in stage 10.0 (TID 155) +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO Executor: Finished task 128.0 in stage 10.0 (TID 145). 6606 bytes result sent to driver +26/04/01 08:48:17 INFO TaskSetManager: Starting task 139.0 in stage 10.0 (TID 156) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:17 INFO TaskSetManager: Finished task 128.0 in stage 10.0 (TID 145) in 8138 ms on 10.0.0.133 (executor driver) (132/208) +26/04/01 08:48:17 INFO Executor: Running task 139.0 in stage 10.0 (TID 156) +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO Executor: Finished task 133.0 in stage 10.0 (TID 150). 6606 bytes result sent to driver +26/04/01 08:48:17 INFO TaskSetManager: Starting task 140.0 in stage 10.0 (TID 157) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:17 INFO TaskSetManager: Finished task 133.0 in stage 10.0 (TID 150) in 7968 ms on 10.0.0.133 (executor driver) (133/208) +26/04/01 08:48:17 INFO Executor: Running task 140.0 in stage 10.0 (TID 157) +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO Executor: Finished task 132.0 in stage 10.0 (TID 149). 6606 bytes result sent to driver +26/04/01 08:48:17 INFO TaskSetManager: Starting task 141.0 in stage 10.0 (TID 158) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:17 INFO Executor: Running task 141.0 in stage 10.0 (TID 158) +26/04/01 08:48:17 INFO TaskSetManager: Finished task 132.0 in stage 10.0 (TID 149) in 7979 ms on 10.0.0.133 (executor driver) (134/208) +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO Executor: Finished task 134.0 in stage 10.0 (TID 151). 6606 bytes result sent to driver +26/04/01 08:48:17 INFO TaskSetManager: Starting task 142.0 in stage 10.0 (TID 159) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:17 INFO TaskSetManager: Finished task 134.0 in stage 10.0 (TID 151) in 7969 ms on 10.0.0.133 (executor driver) (135/208) +26/04/01 08:48:17 INFO Executor: Running task 142.0 in stage 10.0 (TID 159) +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO Executor: Finished task 135.0 in stage 10.0 (TID 152). 6606 bytes result sent to driver +26/04/01 08:48:17 INFO TaskSetManager: Starting task 143.0 in stage 10.0 (TID 160) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:17 INFO TaskSetManager: Finished task 135.0 in stage 10.0 (TID 152) in 7970 ms on 10.0.0.133 (executor driver) (136/208) +26/04/01 08:48:17 INFO Executor: Running task 143.0 in stage 10.0 (TID 160) +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO Executor: Finished task 136.0 in stage 10.0 (TID 153). 6606 bytes result sent to driver +26/04/01 08:48:25 INFO TaskSetManager: Starting task 144.0 in stage 10.0 (TID 161) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:48:25 INFO TaskSetManager: Finished task 136.0 in stage 10.0 (TID 153) in 7954 ms on 10.0.0.133 (executor driver) (137/208) +26/04/01 08:48:25 INFO Executor: Running task 144.0 in stage 10.0 (TID 161) +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO Executor: Finished task 137.0 in stage 10.0 (TID 154). 6606 bytes result sent to driver +26/04/01 08:48:25 INFO TaskSetManager: Starting task 145.0 in stage 10.0 (TID 162) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:48:25 INFO Executor: Running task 145.0 in stage 10.0 (TID 162) +26/04/01 08:48:25 INFO TaskSetManager: Finished task 137.0 in stage 10.0 (TID 154) in 7904 ms on 10.0.0.133 (executor driver) (138/208) +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO Executor: Finished task 138.0 in stage 10.0 (TID 155). 6606 bytes result sent to driver +26/04/01 08:48:25 INFO Executor: Finished task 139.0 in stage 10.0 (TID 156). 6606 bytes result sent to driver +26/04/01 08:48:25 INFO TaskSetManager: Starting task 146.0 in stage 10.0 (TID 163) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:25 INFO Executor: Running task 146.0 in stage 10.0 (TID 163) +26/04/01 08:48:25 INFO TaskSetManager: Starting task 147.0 in stage 10.0 (TID 164) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:25 INFO TaskSetManager: Finished task 138.0 in stage 10.0 (TID 155) in 7886 ms on 10.0.0.133 (executor driver) (139/208) +26/04/01 08:48:25 INFO Executor: Running task 147.0 in stage 10.0 (TID 164) +26/04/01 08:48:25 INFO TaskSetManager: Finished task 139.0 in stage 10.0 (TID 156) in 7882 ms on 10.0.0.133 (executor driver) (140/208) +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO Executor: Finished task 140.0 in stage 10.0 (TID 157). 6606 bytes result sent to driver +26/04/01 08:48:25 INFO TaskSetManager: Starting task 148.0 in stage 10.0 (TID 165) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:25 INFO TaskSetManager: Finished task 140.0 in stage 10.0 (TID 157) in 7917 ms on 10.0.0.133 (executor driver) (141/208) +26/04/01 08:48:25 INFO Executor: Running task 148.0 in stage 10.0 (TID 165) +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO Executor: Finished task 141.0 in stage 10.0 (TID 158). 6606 bytes result sent to driver +26/04/01 08:48:25 INFO TaskSetManager: Starting task 149.0 in stage 10.0 (TID 166) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:25 INFO Executor: Running task 149.0 in stage 10.0 (TID 166) +26/04/01 08:48:25 INFO TaskSetManager: Finished task 141.0 in stage 10.0 (TID 158) in 7924 ms on 10.0.0.133 (executor driver) (142/208) +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO Executor: Finished task 142.0 in stage 10.0 (TID 159). 6606 bytes result sent to driver +26/04/01 08:48:25 INFO TaskSetManager: Starting task 150.0 in stage 10.0 (TID 167) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:48:25 INFO TaskSetManager: Finished task 142.0 in stage 10.0 (TID 159) in 7914 ms on 10.0.0.133 (executor driver) (143/208) +26/04/01 08:48:25 INFO Executor: Running task 150.0 in stage 10.0 (TID 167) +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO Executor: Finished task 143.0 in stage 10.0 (TID 160). 6606 bytes result sent to driver +26/04/01 08:48:25 INFO TaskSetManager: Starting task 151.0 in stage 10.0 (TID 168) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:25 INFO TaskSetManager: Finished task 143.0 in stage 10.0 (TID 160) in 7886 ms on 10.0.0.133 (executor driver) (144/208) +26/04/01 08:48:25 INFO Executor: Running task 151.0 in stage 10.0 (TID 168) +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO Executor: Finished task 144.0 in stage 10.0 (TID 161). 6606 bytes result sent to driver +26/04/01 08:48:33 INFO TaskSetManager: Starting task 152.0 in stage 10.0 (TID 169) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:33 INFO TaskSetManager: Finished task 144.0 in stage 10.0 (TID 161) in 7943 ms on 10.0.0.133 (executor driver) (145/208) +26/04/01 08:48:33 INFO Executor: Running task 152.0 in stage 10.0 (TID 169) +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO Executor: Finished task 145.0 in stage 10.0 (TID 162). 6606 bytes result sent to driver +26/04/01 08:48:33 INFO TaskSetManager: Starting task 153.0 in stage 10.0 (TID 170) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:33 INFO TaskSetManager: Finished task 145.0 in stage 10.0 (TID 162) in 7949 ms on 10.0.0.133 (executor driver) (146/208) +26/04/01 08:48:33 INFO Executor: Running task 153.0 in stage 10.0 (TID 170) +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO Executor: Finished task 147.0 in stage 10.0 (TID 164). 6606 bytes result sent to driver +26/04/01 08:48:33 INFO TaskSetManager: Starting task 154.0 in stage 10.0 (TID 171) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:33 INFO TaskSetManager: Finished task 147.0 in stage 10.0 (TID 164) in 7941 ms on 10.0.0.133 (executor driver) (147/208) +26/04/01 08:48:33 INFO Executor: Running task 154.0 in stage 10.0 (TID 171) +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO Executor: Finished task 146.0 in stage 10.0 (TID 163). 6606 bytes result sent to driver +26/04/01 08:48:33 INFO TaskSetManager: Starting task 155.0 in stage 10.0 (TID 172) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:33 INFO TaskSetManager: Finished task 146.0 in stage 10.0 (TID 163) in 7952 ms on 10.0.0.133 (executor driver) (148/208) +26/04/01 08:48:33 INFO Executor: Running task 155.0 in stage 10.0 (TID 172) +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO Executor: Finished task 148.0 in stage 10.0 (TID 165). 6606 bytes result sent to driver +26/04/01 08:48:33 INFO TaskSetManager: Starting task 156.0 in stage 10.0 (TID 173) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:48:33 INFO TaskSetManager: Finished task 148.0 in stage 10.0 (TID 165) in 7939 ms on 10.0.0.133 (executor driver) (149/208) +26/04/01 08:48:33 INFO Executor: Running task 156.0 in stage 10.0 (TID 173) +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO Executor: Finished task 149.0 in stage 10.0 (TID 166). 6606 bytes result sent to driver +26/04/01 08:48:33 INFO TaskSetManager: Starting task 157.0 in stage 10.0 (TID 174) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:33 INFO Executor: Running task 157.0 in stage 10.0 (TID 174) +26/04/01 08:48:33 INFO TaskSetManager: Finished task 149.0 in stage 10.0 (TID 166) in 7935 ms on 10.0.0.133 (executor driver) (150/208) +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO Executor: Finished task 150.0 in stage 10.0 (TID 167). 6606 bytes result sent to driver +26/04/01 08:48:33 INFO TaskSetManager: Starting task 158.0 in stage 10.0 (TID 175) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:33 INFO TaskSetManager: Finished task 150.0 in stage 10.0 (TID 167) in 7951 ms on 10.0.0.133 (executor driver) (151/208) +26/04/01 08:48:33 INFO Executor: Running task 158.0 in stage 10.0 (TID 175) +26/04/01 08:48:33 INFO Executor: Finished task 151.0 in stage 10.0 (TID 168). 6606 bytes result sent to driver +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO TaskSetManager: Starting task 159.0 in stage 10.0 (TID 176) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:33 INFO TaskSetManager: Finished task 151.0 in stage 10.0 (TID 168) in 7926 ms on 10.0.0.133 (executor driver) (152/208) +26/04/01 08:48:33 INFO Executor: Running task 159.0 in stage 10.0 (TID 176) +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO Executor: Finished task 152.0 in stage 10.0 (TID 169). 6563 bytes result sent to driver +26/04/01 08:48:41 INFO TaskSetManager: Starting task 160.0 in stage 10.0 (TID 177) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:41 INFO TaskSetManager: Finished task 152.0 in stage 10.0 (TID 169) in 8113 ms on 10.0.0.133 (executor driver) (153/208) +26/04/01 08:48:41 INFO Executor: Running task 160.0 in stage 10.0 (TID 177) +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO Executor: Finished task 154.0 in stage 10.0 (TID 171). 6563 bytes result sent to driver +26/04/01 08:48:41 INFO TaskSetManager: Starting task 161.0 in stage 10.0 (TID 178) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:41 INFO TaskSetManager: Finished task 154.0 in stage 10.0 (TID 171) in 8098 ms on 10.0.0.133 (executor driver) (154/208) +26/04/01 08:48:41 INFO Executor: Running task 161.0 in stage 10.0 (TID 178) +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO Executor: Finished task 155.0 in stage 10.0 (TID 172). 6563 bytes result sent to driver +26/04/01 08:48:41 INFO TaskSetManager: Starting task 162.0 in stage 10.0 (TID 179) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:48:41 INFO TaskSetManager: Finished task 155.0 in stage 10.0 (TID 172) in 8097 ms on 10.0.0.133 (executor driver) (155/208) +26/04/01 08:48:41 INFO Executor: Running task 162.0 in stage 10.0 (TID 179) +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO Executor: Finished task 153.0 in stage 10.0 (TID 170). 6563 bytes result sent to driver +26/04/01 08:48:41 INFO TaskSetManager: Starting task 163.0 in stage 10.0 (TID 180) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:48:41 INFO TaskSetManager: Finished task 153.0 in stage 10.0 (TID 170) in 8189 ms on 10.0.0.133 (executor driver) (156/208) +26/04/01 08:48:41 INFO Executor: Running task 163.0 in stage 10.0 (TID 180) +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO Executor: Finished task 157.0 in stage 10.0 (TID 174). 6563 bytes result sent to driver +26/04/01 08:48:41 INFO TaskSetManager: Starting task 164.0 in stage 10.0 (TID 181) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:41 INFO TaskSetManager: Finished task 157.0 in stage 10.0 (TID 174) in 8127 ms on 10.0.0.133 (executor driver) (157/208) +26/04/01 08:48:41 INFO Executor: Running task 164.0 in stage 10.0 (TID 181) +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO Executor: Finished task 156.0 in stage 10.0 (TID 173). 6563 bytes result sent to driver +26/04/01 08:48:41 INFO TaskSetManager: Starting task 165.0 in stage 10.0 (TID 182) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:41 INFO Executor: Running task 165.0 in stage 10.0 (TID 182) +26/04/01 08:48:41 INFO TaskSetManager: Finished task 156.0 in stage 10.0 (TID 173) in 8146 ms on 10.0.0.133 (executor driver) (158/208) +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO Executor: Finished task 158.0 in stage 10.0 (TID 175). 6563 bytes result sent to driver +26/04/01 08:48:41 INFO TaskSetManager: Starting task 166.0 in stage 10.0 (TID 183) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:41 INFO TaskSetManager: Finished task 158.0 in stage 10.0 (TID 175) in 8175 ms on 10.0.0.133 (executor driver) (159/208) +26/04/01 08:48:41 INFO Executor: Running task 166.0 in stage 10.0 (TID 183) +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO Executor: Finished task 159.0 in stage 10.0 (TID 176). 6563 bytes result sent to driver +26/04/01 08:48:41 INFO TaskSetManager: Starting task 167.0 in stage 10.0 (TID 184) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:41 INFO TaskSetManager: Finished task 159.0 in stage 10.0 (TID 176) in 8190 ms on 10.0.0.133 (executor driver) (160/208) +26/04/01 08:48:41 INFO Executor: Running task 167.0 in stage 10.0 (TID 184) +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO Executor: Finished task 160.0 in stage 10.0 (TID 177). 6606 bytes result sent to driver +26/04/01 08:48:49 INFO TaskSetManager: Starting task 168.0 in stage 10.0 (TID 185) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:48:49 INFO TaskSetManager: Finished task 160.0 in stage 10.0 (TID 177) in 7979 ms on 10.0.0.133 (executor driver) (161/208) +26/04/01 08:48:49 INFO Executor: Running task 168.0 in stage 10.0 (TID 185) +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO Executor: Finished task 162.0 in stage 10.0 (TID 179). 6606 bytes result sent to driver +26/04/01 08:48:49 INFO TaskSetManager: Starting task 169.0 in stage 10.0 (TID 186) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:49 INFO TaskSetManager: Finished task 162.0 in stage 10.0 (TID 179) in 7955 ms on 10.0.0.133 (executor driver) (162/208) +26/04/01 08:48:49 INFO Executor: Running task 169.0 in stage 10.0 (TID 186) +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO Executor: Finished task 161.0 in stage 10.0 (TID 178). 6606 bytes result sent to driver +26/04/01 08:48:49 INFO TaskSetManager: Starting task 170.0 in stage 10.0 (TID 187) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:49 INFO TaskSetManager: Finished task 161.0 in stage 10.0 (TID 178) in 7977 ms on 10.0.0.133 (executor driver) (163/208) +26/04/01 08:48:49 INFO Executor: Running task 170.0 in stage 10.0 (TID 187) +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO Executor: Finished task 163.0 in stage 10.0 (TID 180). 6606 bytes result sent to driver +26/04/01 08:48:49 INFO TaskSetManager: Starting task 171.0 in stage 10.0 (TID 188) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:49 INFO TaskSetManager: Finished task 163.0 in stage 10.0 (TID 180) in 7959 ms on 10.0.0.133 (executor driver) (164/208) +26/04/01 08:48:49 INFO Executor: Running task 171.0 in stage 10.0 (TID 188) +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO Executor: Finished task 165.0 in stage 10.0 (TID 182). 6606 bytes result sent to driver +26/04/01 08:48:49 INFO TaskSetManager: Starting task 172.0 in stage 10.0 (TID 189) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:49 INFO TaskSetManager: Finished task 165.0 in stage 10.0 (TID 182) in 7952 ms on 10.0.0.133 (executor driver) (165/208) +26/04/01 08:48:49 INFO Executor: Running task 172.0 in stage 10.0 (TID 189) +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO Executor: Finished task 164.0 in stage 10.0 (TID 181). 6606 bytes result sent to driver +26/04/01 08:48:49 INFO TaskSetManager: Starting task 173.0 in stage 10.0 (TID 190) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:49 INFO TaskSetManager: Finished task 164.0 in stage 10.0 (TID 181) in 7983 ms on 10.0.0.133 (executor driver) (166/208) +26/04/01 08:48:49 INFO Executor: Running task 173.0 in stage 10.0 (TID 190) +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO Executor: Finished task 166.0 in stage 10.0 (TID 183). 6606 bytes result sent to driver +26/04/01 08:48:49 INFO TaskSetManager: Starting task 174.0 in stage 10.0 (TID 191) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9267 bytes) +26/04/01 08:48:49 INFO TaskSetManager: Finished task 166.0 in stage 10.0 (TID 183) in 7961 ms on 10.0.0.133 (executor driver) (167/208) +26/04/01 08:48:49 INFO Executor: Running task 174.0 in stage 10.0 (TID 191) +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO Executor: Finished task 167.0 in stage 10.0 (TID 184). 6606 bytes result sent to driver +26/04/01 08:48:49 INFO TaskSetManager: Starting task 175.0 in stage 10.0 (TID 192) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9272 bytes) +26/04/01 08:48:49 INFO TaskSetManager: Finished task 167.0 in stage 10.0 (TID 184) in 7977 ms on 10.0.0.133 (executor driver) (168/208) +26/04/01 08:48:49 INFO Executor: Running task 175.0 in stage 10.0 (TID 192) +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO Executor: Finished task 168.0 in stage 10.0 (TID 185). 6606 bytes result sent to driver +26/04/01 08:48:57 INFO TaskSetManager: Starting task 176.0 in stage 10.0 (TID 193) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:57 INFO TaskSetManager: Finished task 168.0 in stage 10.0 (TID 185) in 8012 ms on 10.0.0.133 (executor driver) (169/208) +26/04/01 08:48:57 INFO Executor: Running task 176.0 in stage 10.0 (TID 193) +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO Executor: Finished task 170.0 in stage 10.0 (TID 187). 6606 bytes result sent to driver +26/04/01 08:48:57 INFO TaskSetManager: Starting task 177.0 in stage 10.0 (TID 194) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:57 INFO TaskSetManager: Finished task 170.0 in stage 10.0 (TID 187) in 8021 ms on 10.0.0.133 (executor driver) (170/208) +26/04/01 08:48:57 INFO Executor: Running task 177.0 in stage 10.0 (TID 194) +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO Executor: Finished task 169.0 in stage 10.0 (TID 186). 6606 bytes result sent to driver +26/04/01 08:48:57 INFO TaskSetManager: Starting task 178.0 in stage 10.0 (TID 195) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:57 INFO TaskSetManager: Finished task 169.0 in stage 10.0 (TID 186) in 8039 ms on 10.0.0.133 (executor driver) (171/208) +26/04/01 08:48:57 INFO Executor: Running task 178.0 in stage 10.0 (TID 195) +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO Executor: Finished task 171.0 in stage 10.0 (TID 188). 6606 bytes result sent to driver +26/04/01 08:48:57 INFO TaskSetManager: Starting task 179.0 in stage 10.0 (TID 196) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:57 INFO TaskSetManager: Finished task 171.0 in stage 10.0 (TID 188) in 8015 ms on 10.0.0.133 (executor driver) (172/208) +26/04/01 08:48:57 INFO Executor: Running task 179.0 in stage 10.0 (TID 196) +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO Executor: Finished task 173.0 in stage 10.0 (TID 190). 6606 bytes result sent to driver +26/04/01 08:48:57 INFO TaskSetManager: Starting task 180.0 in stage 10.0 (TID 197) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:48:57 INFO TaskSetManager: Finished task 173.0 in stage 10.0 (TID 190) in 8027 ms on 10.0.0.133 (executor driver) (173/208) +26/04/01 08:48:57 INFO Executor: Running task 180.0 in stage 10.0 (TID 197) +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO Executor: Finished task 172.0 in stage 10.0 (TID 189). 6606 bytes result sent to driver +26/04/01 08:48:57 INFO TaskSetManager: Starting task 181.0 in stage 10.0 (TID 198) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:48:57 INFO TaskSetManager: Finished task 172.0 in stage 10.0 (TID 189) in 8107 ms on 10.0.0.133 (executor driver) (174/208) +26/04/01 08:48:57 INFO Executor: Running task 181.0 in stage 10.0 (TID 198) +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO Executor: Finished task 174.0 in stage 10.0 (TID 191). 6606 bytes result sent to driver +26/04/01 08:48:57 INFO TaskSetManager: Starting task 182.0 in stage 10.0 (TID 199) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:57 INFO TaskSetManager: Finished task 174.0 in stage 10.0 (TID 191) in 8038 ms on 10.0.0.133 (executor driver) (175/208) +26/04/01 08:48:57 INFO Executor: Running task 182.0 in stage 10.0 (TID 199) +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO Executor: Finished task 175.0 in stage 10.0 (TID 192). 6606 bytes result sent to driver +26/04/01 08:48:57 INFO TaskSetManager: Starting task 183.0 in stage 10.0 (TID 200) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:48:57 INFO Executor: Running task 183.0 in stage 10.0 (TID 200) +26/04/01 08:48:57 INFO TaskSetManager: Finished task 175.0 in stage 10.0 (TID 192) in 8033 ms on 10.0.0.133 (executor driver) (176/208) +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO Executor: Finished task 176.0 in stage 10.0 (TID 193). 6606 bytes result sent to driver +26/04/01 08:49:05 INFO TaskSetManager: Starting task 184.0 in stage 10.0 (TID 201) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:49:05 INFO TaskSetManager: Finished task 176.0 in stage 10.0 (TID 193) in 7965 ms on 10.0.0.133 (executor driver) (177/208) +26/04/01 08:49:05 INFO Executor: Running task 184.0 in stage 10.0 (TID 201) +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO Executor: Finished task 178.0 in stage 10.0 (TID 195). 6606 bytes result sent to driver +26/04/01 08:49:05 INFO TaskSetManager: Starting task 185.0 in stage 10.0 (TID 202) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:49:05 INFO TaskSetManager: Finished task 178.0 in stage 10.0 (TID 195) in 7957 ms on 10.0.0.133 (executor driver) (178/208) +26/04/01 08:49:05 INFO Executor: Finished task 177.0 in stage 10.0 (TID 194). 6606 bytes result sent to driver +26/04/01 08:49:05 INFO Executor: Running task 185.0 in stage 10.0 (TID 202) +26/04/01 08:49:05 INFO TaskSetManager: Starting task 186.0 in stage 10.0 (TID 203) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:05 INFO TaskSetManager: Finished task 177.0 in stage 10.0 (TID 194) in 7963 ms on 10.0.0.133 (executor driver) (179/208) +26/04/01 08:49:05 INFO Executor: Running task 186.0 in stage 10.0 (TID 203) +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO Executor: Finished task 179.0 in stage 10.0 (TID 196). 6606 bytes result sent to driver +26/04/01 08:49:05 INFO TaskSetManager: Starting task 187.0 in stage 10.0 (TID 204) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9273 bytes) +26/04/01 08:49:05 INFO TaskSetManager: Finished task 179.0 in stage 10.0 (TID 196) in 7949 ms on 10.0.0.133 (executor driver) (180/208) +26/04/01 08:49:05 INFO Executor: Running task 187.0 in stage 10.0 (TID 204) +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO Executor: Finished task 180.0 in stage 10.0 (TID 197). 6606 bytes result sent to driver +26/04/01 08:49:05 INFO TaskSetManager: Starting task 188.0 in stage 10.0 (TID 205) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:49:05 INFO TaskSetManager: Finished task 180.0 in stage 10.0 (TID 197) in 7949 ms on 10.0.0.133 (executor driver) (181/208) +26/04/01 08:49:05 INFO Executor: Running task 188.0 in stage 10.0 (TID 205) +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO Executor: Finished task 181.0 in stage 10.0 (TID 198). 6606 bytes result sent to driver +26/04/01 08:49:05 INFO TaskSetManager: Starting task 189.0 in stage 10.0 (TID 206) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:49:05 INFO TaskSetManager: Finished task 181.0 in stage 10.0 (TID 198) in 7956 ms on 10.0.0.133 (executor driver) (182/208) +26/04/01 08:49:05 INFO Executor: Running task 189.0 in stage 10.0 (TID 206) +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO Executor: Finished task 182.0 in stage 10.0 (TID 199). 6606 bytes result sent to driver +26/04/01 08:49:05 INFO TaskSetManager: Starting task 190.0 in stage 10.0 (TID 207) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:49:05 INFO TaskSetManager: Finished task 182.0 in stage 10.0 (TID 199) in 7964 ms on 10.0.0.133 (executor driver) (183/208) +26/04/01 08:49:05 INFO Executor: Running task 190.0 in stage 10.0 (TID 207) +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO Executor: Finished task 183.0 in stage 10.0 (TID 200). 6606 bytes result sent to driver +26/04/01 08:49:05 INFO TaskSetManager: Starting task 191.0 in stage 10.0 (TID 208) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9274 bytes) +26/04/01 08:49:05 INFO Executor: Running task 191.0 in stage 10.0 (TID 208) +26/04/01 08:49:05 INFO TaskSetManager: Finished task 183.0 in stage 10.0 (TID 200) in 7952 ms on 10.0.0.133 (executor driver) (184/208) +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO Executor: Finished task 184.0 in stage 10.0 (TID 201). 6606 bytes result sent to driver +26/04/01 08:49:13 INFO TaskSetManager: Starting task 192.0 in stage 10.0 (TID 209) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:49:13 INFO TaskSetManager: Finished task 184.0 in stage 10.0 (TID 201) in 7954 ms on 10.0.0.133 (executor driver) (185/208) +26/04/01 08:49:13 INFO Executor: Running task 192.0 in stage 10.0 (TID 209) +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO Executor: Finished task 186.0 in stage 10.0 (TID 203). 6606 bytes result sent to driver +26/04/01 08:49:13 INFO TaskSetManager: Starting task 193.0 in stage 10.0 (TID 210) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9346 bytes) +26/04/01 08:49:13 INFO Executor: Running task 193.0 in stage 10.0 (TID 210) +26/04/01 08:49:13 INFO TaskSetManager: Finished task 186.0 in stage 10.0 (TID 203) in 7969 ms on 10.0.0.133 (executor driver) (186/208) +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO Executor: Finished task 185.0 in stage 10.0 (TID 202). 6606 bytes result sent to driver +26/04/01 08:49:13 INFO TaskSetManager: Starting task 194.0 in stage 10.0 (TID 211) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:49:13 INFO Executor: Running task 194.0 in stage 10.0 (TID 211) +26/04/01 08:49:13 INFO TaskSetManager: Finished task 185.0 in stage 10.0 (TID 202) in 7972 ms on 10.0.0.133 (executor driver) (187/208) +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO Executor: Finished task 187.0 in stage 10.0 (TID 204). 6606 bytes result sent to driver +26/04/01 08:49:13 INFO TaskSetManager: Starting task 195.0 in stage 10.0 (TID 212) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:49:13 INFO Executor: Running task 195.0 in stage 10.0 (TID 212) +26/04/01 08:49:13 INFO TaskSetManager: Finished task 187.0 in stage 10.0 (TID 204) in 7978 ms on 10.0.0.133 (executor driver) (188/208) +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO Executor: Finished task 188.0 in stage 10.0 (TID 205). 6606 bytes result sent to driver +26/04/01 08:49:13 INFO TaskSetManager: Starting task 196.0 in stage 10.0 (TID 213) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9346 bytes) +26/04/01 08:49:13 INFO TaskSetManager: Finished task 188.0 in stage 10.0 (TID 205) in 7952 ms on 10.0.0.133 (executor driver) (189/208) +26/04/01 08:49:13 INFO Executor: Running task 196.0 in stage 10.0 (TID 213) +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO Executor: Finished task 189.0 in stage 10.0 (TID 206). 6606 bytes result sent to driver +26/04/01 08:49:13 INFO TaskSetManager: Starting task 197.0 in stage 10.0 (TID 214) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9346 bytes) +26/04/01 08:49:13 INFO TaskSetManager: Finished task 189.0 in stage 10.0 (TID 206) in 7951 ms on 10.0.0.133 (executor driver) (190/208) +26/04/01 08:49:13 INFO Executor: Running task 197.0 in stage 10.0 (TID 214) +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO Executor: Finished task 190.0 in stage 10.0 (TID 207). 6606 bytes result sent to driver +26/04/01 08:49:13 INFO TaskSetManager: Starting task 198.0 in stage 10.0 (TID 215) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:49:13 INFO Executor: Running task 198.0 in stage 10.0 (TID 215) +26/04/01 08:49:13 INFO TaskSetManager: Finished task 190.0 in stage 10.0 (TID 207) in 7947 ms on 10.0.0.133 (executor driver) (191/208) +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO Executor: Finished task 191.0 in stage 10.0 (TID 208). 6606 bytes result sent to driver +26/04/01 08:49:13 INFO TaskSetManager: Starting task 199.0 in stage 10.0 (TID 216) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:49:13 INFO TaskSetManager: Finished task 191.0 in stage 10.0 (TID 208) in 7944 ms on 10.0.0.133 (executor driver) (192/208) +26/04/01 08:49:13 INFO Executor: Running task 199.0 in stage 10.0 (TID 216) +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO Executor: Finished task 192.0 in stage 10.0 (TID 209). 6563 bytes result sent to driver +26/04/01 08:49:18 INFO TaskSetManager: Starting task 200.0 in stage 10.0 (TID 217) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9345 bytes) +26/04/01 08:49:18 INFO TaskSetManager: Finished task 192.0 in stage 10.0 (TID 209) in 4962 ms on 10.0.0.133 (executor driver) (193/208) +26/04/01 08:49:18 INFO Executor: Running task 200.0 in stage 10.0 (TID 217) +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO Executor: Finished task 193.0 in stage 10.0 (TID 210). 6563 bytes result sent to driver +26/04/01 08:49:18 INFO TaskSetManager: Starting task 201.0 in stage 10.0 (TID 218) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9346 bytes) +26/04/01 08:49:18 INFO TaskSetManager: Finished task 193.0 in stage 10.0 (TID 210) in 4945 ms on 10.0.0.133 (executor driver) (194/208) +26/04/01 08:49:18 INFO Executor: Running task 201.0 in stage 10.0 (TID 218) +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO Executor: Finished task 194.0 in stage 10.0 (TID 211). 6563 bytes result sent to driver +26/04/01 08:49:18 INFO TaskSetManager: Starting task 202.0 in stage 10.0 (TID 219) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9346 bytes) +26/04/01 08:49:18 INFO TaskSetManager: Finished task 194.0 in stage 10.0 (TID 211) in 4962 ms on 10.0.0.133 (executor driver) (195/208) +26/04/01 08:49:18 INFO Executor: Running task 202.0 in stage 10.0 (TID 219) +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO Executor: Finished task 195.0 in stage 10.0 (TID 212). 6563 bytes result sent to driver +26/04/01 08:49:18 INFO TaskSetManager: Starting task 203.0 in stage 10.0 (TID 220) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:49:18 INFO TaskSetManager: Finished task 195.0 in stage 10.0 (TID 212) in 4938 ms on 10.0.0.133 (executor driver) (196/208) +26/04/01 08:49:18 INFO Executor: Running task 203.0 in stage 10.0 (TID 220) +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO Executor: Finished task 196.0 in stage 10.0 (TID 213). 6563 bytes result sent to driver +26/04/01 08:49:18 INFO TaskSetManager: Starting task 204.0 in stage 10.0 (TID 221) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:49:18 INFO Executor: Running task 204.0 in stage 10.0 (TID 221) +26/04/01 08:49:18 INFO TaskSetManager: Finished task 196.0 in stage 10.0 (TID 213) in 4962 ms on 10.0.0.133 (executor driver) (197/208) +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO Executor: Finished task 197.0 in stage 10.0 (TID 214). 6563 bytes result sent to driver +26/04/01 08:49:18 INFO TaskSetManager: Starting task 205.0 in stage 10.0 (TID 222) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9347 bytes) +26/04/01 08:49:18 INFO TaskSetManager: Finished task 197.0 in stage 10.0 (TID 214) in 4935 ms on 10.0.0.133 (executor driver) (198/208) +26/04/01 08:49:18 INFO Executor: Running task 205.0 in stage 10.0 (TID 222) +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO Executor: Finished task 198.0 in stage 10.0 (TID 215). 6563 bytes result sent to driver +26/04/01 08:49:18 INFO Executor: Finished task 199.0 in stage 10.0 (TID 216). 6563 bytes result sent to driver +26/04/01 08:49:18 INFO TaskSetManager: Starting task 206.0 in stage 10.0 (TID 223) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9346 bytes) +26/04/01 08:49:18 INFO TaskSetManager: Starting task 207.0 in stage 10.0 (TID 224) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9346 bytes) +26/04/01 08:49:18 INFO Executor: Running task 206.0 in stage 10.0 (TID 223) +26/04/01 08:49:18 INFO TaskSetManager: Finished task 198.0 in stage 10.0 (TID 215) in 4944 ms on 10.0.0.133 (executor driver) (199/208) +26/04/01 08:49:18 INFO Executor: Running task 207.0 in stage 10.0 (TID 224) +26/04/01 08:49:18 INFO TaskSetManager: Finished task 199.0 in stage 10.0 (TID 216) in 4925 ms on 10.0.0.133 (executor driver) (200/208) +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO Executor: Finished task 200.0 in stage 10.0 (TID 217). 6606 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 0.0 in stage 11.0 (TID 225) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9463 bytes) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 200.0 in stage 10.0 (TID 217) in 4964 ms on 10.0.0.133 (executor driver) (201/208) +26/04/01 08:49:23 INFO Executor: Running task 0.0 in stage 11.0 (TID 225) +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO Executor: Finished task 201.0 in stage 10.0 (TID 218). 6606 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 1.0 in stage 11.0 (TID 226) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9464 bytes) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 201.0 in stage 10.0 (TID 218) in 4966 ms on 10.0.0.133 (executor driver) (202/208) +26/04/01 08:49:23 INFO Executor: Running task 1.0 in stage 11.0 (TID 226) +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO Executor: Finished task 202.0 in stage 10.0 (TID 219). 6606 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 2.0 in stage 11.0 (TID 227) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9464 bytes) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 202.0 in stage 10.0 (TID 219) in 4976 ms on 10.0.0.133 (executor driver) (203/208) +26/04/01 08:49:23 INFO Executor: Running task 2.0 in stage 11.0 (TID 227) +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO Executor: Finished task 0.0 in stage 11.0 (TID 225). 6563 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 3.0 in stage 11.0 (TID 228) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9465 bytes) +26/04/01 08:49:23 INFO Executor: Running task 3.0 in stage 11.0 (TID 228) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 0.0 in stage 11.0 (TID 225) in 110 ms on 10.0.0.133 (executor driver) (1/8) +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO Executor: Finished task 203.0 in stage 10.0 (TID 220). 6606 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 4.0 in stage 11.0 (TID 229) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9465 bytes) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 203.0 in stage 10.0 (TID 220) in 4986 ms on 10.0.0.133 (executor driver) (204/208) +26/04/01 08:49:23 INFO Executor: Running task 4.0 in stage 11.0 (TID 229) +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO Executor: Finished task 1.0 in stage 11.0 (TID 226). 6563 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 5.0 in stage 11.0 (TID 230) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9464 bytes) +26/04/01 08:49:23 INFO Executor: Running task 5.0 in stage 11.0 (TID 230) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 1.0 in stage 11.0 (TID 226) in 110 ms on 10.0.0.133 (executor driver) (2/8) +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO Executor: Finished task 2.0 in stage 11.0 (TID 227). 6563 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 6.0 in stage 11.0 (TID 231) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9463 bytes) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 2.0 in stage 11.0 (TID 227) in 110 ms on 10.0.0.133 (executor driver) (3/8) +26/04/01 08:49:23 INFO Executor: Running task 6.0 in stage 11.0 (TID 231) +26/04/01 08:49:23 INFO Executor: Finished task 3.0 in stage 11.0 (TID 228). 6563 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 7.0 in stage 11.0 (TID 232) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9463 bytes) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 3.0 in stage 11.0 (TID 228) in 107 ms on 10.0.0.133 (executor driver) (4/8) +26/04/01 08:49:23 INFO Executor: Running task 7.0 in stage 11.0 (TID 232) +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO Executor: Finished task 204.0 in stage 10.0 (TID 221). 6606 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 233) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:23 INFO Executor: Running task 0.0 in stage 12.0 (TID 233) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 204.0 in stage 10.0 (TID 221) in 4956 ms on 10.0.0.133 (executor driver) (205/208) +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO Executor: Finished task 4.0 in stage 11.0 (TID 229). 6563 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 1.0 in stage 12.0 (TID 234) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 4.0 in stage 11.0 (TID 229) in 108 ms on 10.0.0.133 (executor driver) (5/8) +26/04/01 08:49:23 INFO Executor: Running task 1.0 in stage 12.0 (TID 234) +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO Executor: Finished task 5.0 in stage 11.0 (TID 230). 6563 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 2.0 in stage 12.0 (TID 235) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 5.0 in stage 11.0 (TID 230) in 108 ms on 10.0.0.133 (executor driver) (6/8) +26/04/01 08:49:23 INFO Executor: Running task 2.0 in stage 12.0 (TID 235) +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO Executor: Finished task 205.0 in stage 10.0 (TID 222). 6606 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 3.0 in stage 12.0 (TID 236) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 205.0 in stage 10.0 (TID 222) in 5005 ms on 10.0.0.133 (executor driver) (206/208) +26/04/01 08:49:23 INFO Executor: Running task 3.0 in stage 12.0 (TID 236) +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO Executor: Finished task 6.0 in stage 11.0 (TID 231). 6563 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 4.0 in stage 12.0 (TID 237) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 6.0 in stage 11.0 (TID 231) in 114 ms on 10.0.0.133 (executor driver) (7/8) +26/04/01 08:49:23 INFO Executor: Running task 4.0 in stage 12.0 (TID 237) +26/04/01 08:49:23 INFO Executor: Finished task 7.0 in stage 11.0 (TID 232). 6563 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 5.0 in stage 12.0 (TID 238) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 7.0 in stage 11.0 (TID 232) in 113 ms on 10.0.0.133 (executor driver) (8/8) +26/04/01 08:49:23 INFO Executor: Running task 5.0 in stage 12.0 (TID 238) +26/04/01 08:49:23 INFO TaskSchedulerImpl: Removed TaskSet 11.0, whose tasks have all completed, from pool +26/04/01 08:49:23 INFO DAGScheduler: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 203.485 s +26/04/01 08:49:23 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:49:23 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 12, ShuffleMapStage 13, ShuffleMapStage 10) +26/04/01 08:49:23 INFO DAGScheduler: waiting: Set() +26/04/01 08:49:23 INFO DAGScheduler: failed: Set() +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO Executor: Finished task 207.0 in stage 10.0 (TID 224). 6606 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 6.0 in stage 12.0 (TID 239) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 207.0 in stage 10.0 (TID 224) in 4961 ms on 10.0.0.133 (executor driver) (207/208) +26/04/01 08:49:23 INFO Executor: Running task 6.0 in stage 12.0 (TID 239) +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO Executor: Finished task 206.0 in stage 10.0 (TID 223). 6606 bytes result sent to driver +26/04/01 08:49:23 INFO TaskSetManager: Starting task 7.0 in stage 12.0 (TID 240) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:23 INFO TaskSetManager: Finished task 206.0 in stage 10.0 (TID 223) in 4966 ms on 10.0.0.133 (executor driver) (208/208) +26/04/01 08:49:23 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool +26/04/01 08:49:23 INFO Executor: Running task 7.0 in stage 12.0 (TID 240) +26/04/01 08:49:23 INFO DAGScheduler: ShuffleMapStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 203.602 s +26/04/01 08:49:23 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:49:23 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 12, ShuffleMapStage 13) +26/04/01 08:49:23 INFO DAGScheduler: waiting: Set() +26/04/01 08:49:23 INFO DAGScheduler: failed: Set() +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:26 INFO Executor: Finished task 1.0 in stage 12.0 (TID 234). 6563 bytes result sent to driver +26/04/01 08:49:26 INFO TaskSetManager: Starting task 8.0 in stage 12.0 (TID 241) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:26 INFO TaskSetManager: Finished task 1.0 in stage 12.0 (TID 234) in 3309 ms on 10.0.0.133 (executor driver) (1/37) +26/04/01 08:49:26 INFO Executor: Running task 8.0 in stage 12.0 (TID 241) +26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:26 INFO Executor: Finished task 0.0 in stage 12.0 (TID 233). 6563 bytes result sent to driver +26/04/01 08:49:26 INFO TaskSetManager: Starting task 9.0 in stage 12.0 (TID 242) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:26 INFO Executor: Running task 9.0 in stage 12.0 (TID 242) +26/04/01 08:49:26 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 233) in 3341 ms on 10.0.0.133 (executor driver) (2/37) +26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:26 INFO Executor: Finished task 2.0 in stage 12.0 (TID 235). 6563 bytes result sent to driver +26/04/01 08:49:26 INFO TaskSetManager: Starting task 10.0 in stage 12.0 (TID 243) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:26 INFO TaskSetManager: Finished task 2.0 in stage 12.0 (TID 235) in 3306 ms on 10.0.0.133 (executor driver) (3/37) +26/04/01 08:49:26 INFO Executor: Running task 10.0 in stage 12.0 (TID 243) +26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:26 INFO Executor: Finished task 3.0 in stage 12.0 (TID 236). 6563 bytes result sent to driver +26/04/01 08:49:26 INFO TaskSetManager: Starting task 11.0 in stage 12.0 (TID 244) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:26 INFO TaskSetManager: Finished task 3.0 in stage 12.0 (TID 236) in 3318 ms on 10.0.0.133 (executor driver) (4/37) +26/04/01 08:49:26 INFO Executor: Running task 11.0 in stage 12.0 (TID 244) +26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:26 INFO Executor: Finished task 4.0 in stage 12.0 (TID 237). 6563 bytes result sent to driver +26/04/01 08:49:26 INFO TaskSetManager: Starting task 12.0 in stage 12.0 (TID 245) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:26 INFO Executor: Running task 12.0 in stage 12.0 (TID 245) +26/04/01 08:49:26 INFO TaskSetManager: Finished task 4.0 in stage 12.0 (TID 237) in 3320 ms on 10.0.0.133 (executor driver) (5/37) +26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:26 INFO Executor: Finished task 5.0 in stage 12.0 (TID 238). 6606 bytes result sent to driver +26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:26 INFO TaskSetManager: Starting task 13.0 in stage 12.0 (TID 246) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:26 INFO TaskSetManager: Finished task 5.0 in stage 12.0 (TID 238) in 3323 ms on 10.0.0.133 (executor driver) (6/37) +26/04/01 08:49:26 INFO Executor: Running task 13.0 in stage 12.0 (TID 246) +26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:27 INFO Executor: Finished task 6.0 in stage 12.0 (TID 239). 6606 bytes result sent to driver +26/04/01 08:49:27 INFO TaskSetManager: Starting task 14.0 in stage 12.0 (TID 247) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:27 INFO TaskSetManager: Finished task 6.0 in stage 12.0 (TID 239) in 3309 ms on 10.0.0.133 (executor driver) (7/37) +26/04/01 08:49:27 INFO Executor: Running task 14.0 in stage 12.0 (TID 247) +26/04/01 08:49:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:27 INFO Executor: Finished task 7.0 in stage 12.0 (TID 240). 6606 bytes result sent to driver +26/04/01 08:49:27 INFO TaskSetManager: Starting task 15.0 in stage 12.0 (TID 248) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:27 INFO TaskSetManager: Finished task 7.0 in stage 12.0 (TID 240) in 3329 ms on 10.0.0.133 (executor driver) (8/37) +26/04/01 08:49:27 INFO Executor: Running task 15.0 in stage 12.0 (TID 248) +26/04/01 08:49:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO Executor: Finished task 8.0 in stage 12.0 (TID 241). 6606 bytes result sent to driver +26/04/01 08:49:30 INFO TaskSetManager: Starting task 16.0 in stage 12.0 (TID 249) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:30 INFO TaskSetManager: Finished task 8.0 in stage 12.0 (TID 241) in 3299 ms on 10.0.0.133 (executor driver) (9/37) +26/04/01 08:49:30 INFO Executor: Running task 16.0 in stage 12.0 (TID 249) +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO Executor: Finished task 9.0 in stage 12.0 (TID 242). 6606 bytes result sent to driver +26/04/01 08:49:30 INFO TaskSetManager: Starting task 17.0 in stage 12.0 (TID 250) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:30 INFO Executor: Running task 17.0 in stage 12.0 (TID 250) +26/04/01 08:49:30 INFO TaskSetManager: Finished task 9.0 in stage 12.0 (TID 242) in 3296 ms on 10.0.0.133 (executor driver) (10/37) +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO Executor: Finished task 11.0 in stage 12.0 (TID 244). 6606 bytes result sent to driver +26/04/01 08:49:30 INFO Executor: Finished task 10.0 in stage 12.0 (TID 243). 6606 bytes result sent to driver +26/04/01 08:49:30 INFO TaskSetManager: Starting task 18.0 in stage 12.0 (TID 251) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:30 INFO TaskSetManager: Finished task 11.0 in stage 12.0 (TID 244) in 3292 ms on 10.0.0.133 (executor driver) (11/37) +26/04/01 08:49:30 INFO Executor: Running task 18.0 in stage 12.0 (TID 251) +26/04/01 08:49:30 INFO TaskSetManager: Starting task 19.0 in stage 12.0 (TID 252) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:30 INFO TaskSetManager: Finished task 10.0 in stage 12.0 (TID 243) in 3316 ms on 10.0.0.133 (executor driver) (12/37) +26/04/01 08:49:30 INFO Executor: Running task 19.0 in stage 12.0 (TID 252) +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO Executor: Finished task 12.0 in stage 12.0 (TID 245). 6606 bytes result sent to driver +26/04/01 08:49:30 INFO TaskSetManager: Starting task 20.0 in stage 12.0 (TID 253) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:30 INFO TaskSetManager: Finished task 12.0 in stage 12.0 (TID 245) in 3300 ms on 10.0.0.133 (executor driver) (13/37) +26/04/01 08:49:30 INFO Executor: Running task 20.0 in stage 12.0 (TID 253) +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO Executor: Finished task 13.0 in stage 12.0 (TID 246). 6563 bytes result sent to driver +26/04/01 08:49:30 INFO TaskSetManager: Starting task 21.0 in stage 12.0 (TID 254) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:30 INFO TaskSetManager: Finished task 13.0 in stage 12.0 (TID 246) in 3307 ms on 10.0.0.133 (executor driver) (14/37) +26/04/01 08:49:30 INFO Executor: Running task 21.0 in stage 12.0 (TID 254) +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO Executor: Finished task 14.0 in stage 12.0 (TID 247). 6563 bytes result sent to driver +26/04/01 08:49:30 INFO TaskSetManager: Starting task 22.0 in stage 12.0 (TID 255) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:30 INFO TaskSetManager: Finished task 14.0 in stage 12.0 (TID 247) in 3310 ms on 10.0.0.133 (executor driver) (15/37) +26/04/01 08:49:30 INFO Executor: Running task 22.0 in stage 12.0 (TID 255) +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO Executor: Finished task 15.0 in stage 12.0 (TID 248). 6563 bytes result sent to driver +26/04/01 08:49:30 INFO TaskSetManager: Starting task 23.0 in stage 12.0 (TID 256) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:30 INFO Executor: Running task 23.0 in stage 12.0 (TID 256) +26/04/01 08:49:30 INFO TaskSetManager: Finished task 15.0 in stage 12.0 (TID 248) in 3301 ms on 10.0.0.133 (executor driver) (16/37) +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO Executor: Finished task 16.0 in stage 12.0 (TID 249). 6606 bytes result sent to driver +26/04/01 08:49:33 INFO TaskSetManager: Starting task 24.0 in stage 12.0 (TID 257) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:33 INFO TaskSetManager: Finished task 16.0 in stage 12.0 (TID 249) in 3303 ms on 10.0.0.133 (executor driver) (17/37) +26/04/01 08:49:33 INFO Executor: Running task 24.0 in stage 12.0 (TID 257) +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO Executor: Finished task 17.0 in stage 12.0 (TID 250). 6606 bytes result sent to driver +26/04/01 08:49:33 INFO TaskSetManager: Starting task 25.0 in stage 12.0 (TID 258) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:33 INFO TaskSetManager: Finished task 17.0 in stage 12.0 (TID 250) in 3297 ms on 10.0.0.133 (executor driver) (18/37) +26/04/01 08:49:33 INFO Executor: Running task 25.0 in stage 12.0 (TID 258) +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO Executor: Finished task 18.0 in stage 12.0 (TID 251). 6606 bytes result sent to driver +26/04/01 08:49:33 INFO Executor: Finished task 19.0 in stage 12.0 (TID 252). 6606 bytes result sent to driver +26/04/01 08:49:33 INFO TaskSetManager: Starting task 26.0 in stage 12.0 (TID 259) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:33 INFO Executor: Running task 26.0 in stage 12.0 (TID 259) +26/04/01 08:49:33 INFO TaskSetManager: Starting task 27.0 in stage 12.0 (TID 260) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:33 INFO TaskSetManager: Finished task 18.0 in stage 12.0 (TID 251) in 3305 ms on 10.0.0.133 (executor driver) (19/37) +26/04/01 08:49:33 INFO Executor: Running task 27.0 in stage 12.0 (TID 260) +26/04/01 08:49:33 INFO TaskSetManager: Finished task 19.0 in stage 12.0 (TID 252) in 3304 ms on 10.0.0.133 (executor driver) (20/37) +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO Executor: Finished task 20.0 in stage 12.0 (TID 253). 6606 bytes result sent to driver +26/04/01 08:49:33 INFO TaskSetManager: Starting task 28.0 in stage 12.0 (TID 261) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:33 INFO TaskSetManager: Finished task 20.0 in stage 12.0 (TID 253) in 3302 ms on 10.0.0.133 (executor driver) (21/37) +26/04/01 08:49:33 INFO Executor: Running task 28.0 in stage 12.0 (TID 261) +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO Executor: Finished task 21.0 in stage 12.0 (TID 254). 6606 bytes result sent to driver +26/04/01 08:49:33 INFO TaskSetManager: Starting task 29.0 in stage 12.0 (TID 262) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:33 INFO TaskSetManager: Finished task 21.0 in stage 12.0 (TID 254) in 3299 ms on 10.0.0.133 (executor driver) (22/37) +26/04/01 08:49:33 INFO Executor: Running task 29.0 in stage 12.0 (TID 262) +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO Executor: Finished task 22.0 in stage 12.0 (TID 255). 6606 bytes result sent to driver +26/04/01 08:49:33 INFO TaskSetManager: Starting task 30.0 in stage 12.0 (TID 263) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:33 INFO TaskSetManager: Finished task 22.0 in stage 12.0 (TID 255) in 3313 ms on 10.0.0.133 (executor driver) (23/37) +26/04/01 08:49:33 INFO Executor: Running task 30.0 in stage 12.0 (TID 263) +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO Executor: Finished task 23.0 in stage 12.0 (TID 256). 6606 bytes result sent to driver +26/04/01 08:49:33 INFO TaskSetManager: Starting task 31.0 in stage 12.0 (TID 264) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:33 INFO Executor: Running task 31.0 in stage 12.0 (TID 264) +26/04/01 08:49:33 INFO TaskSetManager: Finished task 23.0 in stage 12.0 (TID 256) in 3310 ms on 10.0.0.133 (executor driver) (24/37) +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO Executor: Finished task 24.0 in stage 12.0 (TID 257). 6563 bytes result sent to driver +26/04/01 08:49:36 INFO TaskSetManager: Starting task 32.0 in stage 12.0 (TID 265) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9694 bytes) +26/04/01 08:49:36 INFO TaskSetManager: Finished task 24.0 in stage 12.0 (TID 257) in 3300 ms on 10.0.0.133 (executor driver) (25/37) +26/04/01 08:49:36 INFO Executor: Running task 32.0 in stage 12.0 (TID 265) +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO Executor: Finished task 25.0 in stage 12.0 (TID 258). 6563 bytes result sent to driver +26/04/01 08:49:36 INFO TaskSetManager: Starting task 33.0 in stage 12.0 (TID 266) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9695 bytes) +26/04/01 08:49:36 INFO TaskSetManager: Finished task 25.0 in stage 12.0 (TID 258) in 3314 ms on 10.0.0.133 (executor driver) (26/37) +26/04/01 08:49:36 INFO Executor: Running task 33.0 in stage 12.0 (TID 266) +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO Executor: Finished task 27.0 in stage 12.0 (TID 260). 6563 bytes result sent to driver +26/04/01 08:49:36 INFO TaskSetManager: Starting task 34.0 in stage 12.0 (TID 267) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9691 bytes) +26/04/01 08:49:36 INFO TaskSetManager: Finished task 27.0 in stage 12.0 (TID 260) in 3300 ms on 10.0.0.133 (executor driver) (27/37) +26/04/01 08:49:36 INFO Executor: Running task 34.0 in stage 12.0 (TID 267) +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO Executor: Finished task 26.0 in stage 12.0 (TID 259). 6563 bytes result sent to driver +26/04/01 08:49:36 INFO TaskSetManager: Starting task 35.0 in stage 12.0 (TID 268) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9694 bytes) +26/04/01 08:49:36 INFO Executor: Running task 35.0 in stage 12.0 (TID 268) +26/04/01 08:49:36 INFO TaskSetManager: Finished task 26.0 in stage 12.0 (TID 259) in 3304 ms on 10.0.0.133 (executor driver) (28/37) +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO Executor: Finished task 28.0 in stage 12.0 (TID 261). 6563 bytes result sent to driver +26/04/01 08:49:36 INFO TaskSetManager: Starting task 36.0 in stage 12.0 (TID 269) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9482 bytes) +26/04/01 08:49:36 INFO TaskSetManager: Finished task 28.0 in stage 12.0 (TID 261) in 3310 ms on 10.0.0.133 (executor driver) (29/37) +26/04/01 08:49:36 INFO Executor: Running task 36.0 in stage 12.0 (TID 269) +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO Executor: Finished task 29.0 in stage 12.0 (TID 262). 6563 bytes result sent to driver +26/04/01 08:49:36 INFO TaskSetManager: Starting task 0.0 in stage 13.0 (TID 270) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:36 INFO TaskSetManager: Finished task 29.0 in stage 12.0 (TID 262) in 3312 ms on 10.0.0.133 (executor driver) (30/37) +26/04/01 08:49:36 INFO Executor: Running task 0.0 in stage 13.0 (TID 270) +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO Executor: Finished task 30.0 in stage 12.0 (TID 263). 6563 bytes result sent to driver +26/04/01 08:49:36 INFO TaskSetManager: Starting task 1.0 in stage 13.0 (TID 271) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:49:36 INFO Executor: Running task 1.0 in stage 13.0 (TID 271) +26/04/01 08:49:36 INFO TaskSetManager: Finished task 30.0 in stage 12.0 (TID 263) in 3304 ms on 10.0.0.133 (executor driver) (31/37) +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO Executor: Finished task 31.0 in stage 12.0 (TID 264). 6563 bytes result sent to driver +26/04/01 08:49:36 INFO TaskSetManager: Starting task 2.0 in stage 13.0 (TID 272) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:36 INFO Executor: Running task 2.0 in stage 13.0 (TID 272) +26/04/01 08:49:36 INFO TaskSetManager: Finished task 31.0 in stage 12.0 (TID 264) in 3306 ms on 10.0.0.133 (executor driver) (32/37) +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:38 INFO Executor: Finished task 36.0 in stage 12.0 (TID 269). 6606 bytes result sent to driver +26/04/01 08:49:38 INFO TaskSetManager: Starting task 3.0 in stage 13.0 (TID 273) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:38 INFO TaskSetManager: Finished task 36.0 in stage 12.0 (TID 269) in 1154 ms on 10.0.0.133 (executor driver) (33/37) +26/04/01 08:49:38 INFO Executor: Running task 3.0 in stage 13.0 (TID 273) +26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:38 INFO Executor: Finished task 32.0 in stage 12.0 (TID 265). 6606 bytes result sent to driver +26/04/01 08:49:38 INFO TaskSetManager: Starting task 4.0 in stage 13.0 (TID 274) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:38 INFO Executor: Running task 4.0 in stage 13.0 (TID 274) +26/04/01 08:49:38 INFO TaskSetManager: Finished task 32.0 in stage 12.0 (TID 265) in 2024 ms on 10.0.0.133 (executor driver) (34/37) +26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:38 INFO Executor: Finished task 33.0 in stage 12.0 (TID 266). 6606 bytes result sent to driver +26/04/01 08:49:38 INFO TaskSetManager: Starting task 5.0 in stage 13.0 (TID 275) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:49:38 INFO TaskSetManager: Finished task 33.0 in stage 12.0 (TID 266) in 2026 ms on 10.0.0.133 (executor driver) (35/37) +26/04/01 08:49:38 INFO Executor: Running task 5.0 in stage 13.0 (TID 275) +26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:38 INFO Executor: Finished task 35.0 in stage 12.0 (TID 268). 6606 bytes result sent to driver +26/04/01 08:49:38 INFO TaskSetManager: Starting task 6.0 in stage 13.0 (TID 276) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:38 INFO Executor: Running task 6.0 in stage 13.0 (TID 276) +26/04/01 08:49:38 INFO TaskSetManager: Finished task 35.0 in stage 12.0 (TID 268) in 2027 ms on 10.0.0.133 (executor driver) (36/37) +26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:38 INFO Executor: Finished task 34.0 in stage 12.0 (TID 267). 6606 bytes result sent to driver +26/04/01 08:49:38 INFO TaskSetManager: Starting task 7.0 in stage 13.0 (TID 277) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:38 INFO TaskSetManager: Finished task 34.0 in stage 12.0 (TID 267) in 2032 ms on 10.0.0.133 (executor driver) (37/37) +26/04/01 08:49:38 INFO Executor: Running task 7.0 in stage 13.0 (TID 277) +26/04/01 08:49:38 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool +26/04/01 08:49:38 INFO DAGScheduler: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 218.692 s +26/04/01 08:49:38 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:49:38 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 13) +26/04/01 08:49:38 INFO DAGScheduler: waiting: Set() +26/04/01 08:49:38 INFO DAGScheduler: failed: Set() +26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:39 INFO Executor: Finished task 0.0 in stage 13.0 (TID 270). 6606 bytes result sent to driver +26/04/01 08:49:39 INFO TaskSetManager: Starting task 8.0 in stage 13.0 (TID 278) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:49:39 INFO Executor: Running task 8.0 in stage 13.0 (TID 278) +26/04/01 08:49:39 INFO TaskSetManager: Finished task 0.0 in stage 13.0 (TID 270) in 2250 ms on 10.0.0.133 (executor driver) (1/64) +26/04/01 08:49:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:39 INFO Executor: Finished task 1.0 in stage 13.0 (TID 271). 6606 bytes result sent to driver +26/04/01 08:49:39 INFO TaskSetManager: Starting task 9.0 in stage 13.0 (TID 279) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:39 INFO TaskSetManager: Finished task 1.0 in stage 13.0 (TID 271) in 2233 ms on 10.0.0.133 (executor driver) (2/64) +26/04/01 08:49:39 INFO Executor: Running task 9.0 in stage 13.0 (TID 279) +26/04/01 08:49:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:39 INFO Executor: Finished task 2.0 in stage 13.0 (TID 272). 6606 bytes result sent to driver +26/04/01 08:49:39 INFO TaskSetManager: Starting task 10.0 in stage 13.0 (TID 280) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:49:39 INFO TaskSetManager: Finished task 2.0 in stage 13.0 (TID 272) in 2238 ms on 10.0.0.133 (executor driver) (3/64) +26/04/01 08:49:39 INFO Executor: Running task 10.0 in stage 13.0 (TID 280) +26/04/01 08:49:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:40 INFO Executor: Finished task 3.0 in stage 13.0 (TID 273). 6563 bytes result sent to driver +26/04/01 08:49:40 INFO TaskSetManager: Starting task 11.0 in stage 13.0 (TID 281) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:40 INFO Executor: Running task 11.0 in stage 13.0 (TID 281) +26/04/01 08:49:40 INFO TaskSetManager: Finished task 3.0 in stage 13.0 (TID 273) in 2214 ms on 10.0.0.133 (executor driver) (4/64) +26/04/01 08:49:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:40 INFO Executor: Finished task 4.0 in stage 13.0 (TID 274). 6563 bytes result sent to driver +26/04/01 08:49:40 INFO TaskSetManager: Starting task 12.0 in stage 13.0 (TID 282) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:40 INFO Executor: Running task 12.0 in stage 13.0 (TID 282) +26/04/01 08:49:40 INFO TaskSetManager: Finished task 4.0 in stage 13.0 (TID 274) in 2240 ms on 10.0.0.133 (executor driver) (5/64) +26/04/01 08:49:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:41 INFO Executor: Finished task 5.0 in stage 13.0 (TID 275). 6563 bytes result sent to driver +26/04/01 08:49:41 INFO TaskSetManager: Starting task 13.0 in stage 13.0 (TID 283) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:41 INFO Executor: Running task 13.0 in stage 13.0 (TID 283) +26/04/01 08:49:41 INFO TaskSetManager: Finished task 5.0 in stage 13.0 (TID 275) in 2243 ms on 10.0.0.133 (executor driver) (6/64) +26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:41 INFO Executor: Finished task 6.0 in stage 13.0 (TID 276). 6563 bytes result sent to driver +26/04/01 08:49:41 INFO TaskSetManager: Starting task 14.0 in stage 13.0 (TID 284) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:41 INFO TaskSetManager: Finished task 6.0 in stage 13.0 (TID 276) in 2246 ms on 10.0.0.133 (executor driver) (7/64) +26/04/01 08:49:41 INFO Executor: Running task 14.0 in stage 13.0 (TID 284) +26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:41 INFO Executor: Finished task 7.0 in stage 13.0 (TID 277). 6563 bytes result sent to driver +26/04/01 08:49:41 INFO TaskSetManager: Starting task 15.0 in stage 13.0 (TID 285) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:41 INFO TaskSetManager: Finished task 7.0 in stage 13.0 (TID 277) in 2253 ms on 10.0.0.133 (executor driver) (8/64) +26/04/01 08:49:41 INFO Executor: Running task 15.0 in stage 13.0 (TID 285) +26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:41 INFO Executor: Finished task 8.0 in stage 13.0 (TID 278). 6606 bytes result sent to driver +26/04/01 08:49:41 INFO TaskSetManager: Starting task 16.0 in stage 13.0 (TID 286) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:49:41 INFO TaskSetManager: Finished task 8.0 in stage 13.0 (TID 278) in 2245 ms on 10.0.0.133 (executor driver) (9/64) +26/04/01 08:49:41 INFO Executor: Running task 16.0 in stage 13.0 (TID 286) +26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:41 INFO Executor: Finished task 9.0 in stage 13.0 (TID 279). 6606 bytes result sent to driver +26/04/01 08:49:41 INFO TaskSetManager: Starting task 17.0 in stage 13.0 (TID 287) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:41 INFO TaskSetManager: Finished task 9.0 in stage 13.0 (TID 279) in 2246 ms on 10.0.0.133 (executor driver) (10/64) +26/04/01 08:49:41 INFO Executor: Running task 17.0 in stage 13.0 (TID 287) +26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:41 INFO Executor: Finished task 10.0 in stage 13.0 (TID 280). 6606 bytes result sent to driver +26/04/01 08:49:41 INFO TaskSetManager: Starting task 18.0 in stage 13.0 (TID 288) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:49:41 INFO TaskSetManager: Finished task 10.0 in stage 13.0 (TID 280) in 2236 ms on 10.0.0.133 (executor driver) (11/64) +26/04/01 08:49:41 INFO Executor: Running task 18.0 in stage 13.0 (TID 288) +26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:42 INFO Executor: Finished task 11.0 in stage 13.0 (TID 281). 6606 bytes result sent to driver +26/04/01 08:49:42 INFO TaskSetManager: Starting task 19.0 in stage 13.0 (TID 289) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:42 INFO Executor: Running task 19.0 in stage 13.0 (TID 289) +26/04/01 08:49:42 INFO TaskSetManager: Finished task 11.0 in stage 13.0 (TID 281) in 2220 ms on 10.0.0.133 (executor driver) (12/64) +26/04/01 08:49:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:43 INFO Executor: Finished task 12.0 in stage 13.0 (TID 282). 6606 bytes result sent to driver +26/04/01 08:49:43 INFO TaskSetManager: Starting task 20.0 in stage 13.0 (TID 290) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:43 INFO TaskSetManager: Finished task 12.0 in stage 13.0 (TID 282) in 2252 ms on 10.0.0.133 (executor driver) (13/64) +26/04/01 08:49:43 INFO Executor: Running task 20.0 in stage 13.0 (TID 290) +26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:43 INFO Executor: Finished task 13.0 in stage 13.0 (TID 283). 6606 bytes result sent to driver +26/04/01 08:49:43 INFO TaskSetManager: Starting task 21.0 in stage 13.0 (TID 291) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:49:43 INFO TaskSetManager: Finished task 13.0 in stage 13.0 (TID 283) in 2250 ms on 10.0.0.133 (executor driver) (14/64) +26/04/01 08:49:43 INFO Executor: Running task 21.0 in stage 13.0 (TID 291) +26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:43 INFO Executor: Finished task 14.0 in stage 13.0 (TID 284). 6606 bytes result sent to driver +26/04/01 08:49:43 INFO TaskSetManager: Starting task 22.0 in stage 13.0 (TID 292) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:43 INFO TaskSetManager: Finished task 14.0 in stage 13.0 (TID 284) in 2261 ms on 10.0.0.133 (executor driver) (15/64) +26/04/01 08:49:43 INFO Executor: Running task 22.0 in stage 13.0 (TID 292) +26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:43 INFO Executor: Finished task 15.0 in stage 13.0 (TID 285). 6606 bytes result sent to driver +26/04/01 08:49:43 INFO TaskSetManager: Starting task 23.0 in stage 13.0 (TID 293) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:43 INFO Executor: Running task 23.0 in stage 13.0 (TID 293) +26/04/01 08:49:43 INFO TaskSetManager: Finished task 15.0 in stage 13.0 (TID 285) in 2264 ms on 10.0.0.133 (executor driver) (16/64) +26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:43 INFO Executor: Finished task 16.0 in stage 13.0 (TID 286). 6606 bytes result sent to driver +26/04/01 08:49:43 INFO TaskSetManager: Starting task 24.0 in stage 13.0 (TID 294) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:49:43 INFO TaskSetManager: Finished task 16.0 in stage 13.0 (TID 286) in 2258 ms on 10.0.0.133 (executor driver) (17/64) +26/04/01 08:49:43 INFO Executor: Running task 24.0 in stage 13.0 (TID 294) +26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:43 INFO Executor: Finished task 17.0 in stage 13.0 (TID 287). 6563 bytes result sent to driver +26/04/01 08:49:43 INFO TaskSetManager: Starting task 25.0 in stage 13.0 (TID 295) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:43 INFO TaskSetManager: Finished task 17.0 in stage 13.0 (TID 287) in 2265 ms on 10.0.0.133 (executor driver) (18/64) +26/04/01 08:49:43 INFO Executor: Running task 25.0 in stage 13.0 (TID 295) +26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:43 INFO Executor: Finished task 18.0 in stage 13.0 (TID 288). 6563 bytes result sent to driver +26/04/01 08:49:43 INFO TaskSetManager: Starting task 26.0 in stage 13.0 (TID 296) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:43 INFO TaskSetManager: Finished task 18.0 in stage 13.0 (TID 288) in 2279 ms on 10.0.0.133 (executor driver) (19/64) +26/04/01 08:49:43 INFO Executor: Running task 26.0 in stage 13.0 (TID 296) +26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:44 INFO Executor: Finished task 19.0 in stage 13.0 (TID 289). 6563 bytes result sent to driver +26/04/01 08:49:44 INFO TaskSetManager: Starting task 27.0 in stage 13.0 (TID 297) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:44 INFO TaskSetManager: Finished task 19.0 in stage 13.0 (TID 289) in 2234 ms on 10.0.0.133 (executor driver) (20/64) +26/04/01 08:49:44 INFO Executor: Running task 27.0 in stage 13.0 (TID 297) +26/04/01 08:49:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:45 INFO Executor: Finished task 20.0 in stage 13.0 (TID 290). 6563 bytes result sent to driver +26/04/01 08:49:45 INFO TaskSetManager: Starting task 28.0 in stage 13.0 (TID 298) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:45 INFO Executor: Running task 28.0 in stage 13.0 (TID 298) +26/04/01 08:49:45 INFO TaskSetManager: Finished task 20.0 in stage 13.0 (TID 290) in 2258 ms on 10.0.0.133 (executor driver) (21/64) +26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:45 INFO Executor: Finished task 21.0 in stage 13.0 (TID 291). 6563 bytes result sent to driver +26/04/01 08:49:45 INFO TaskSetManager: Starting task 29.0 in stage 13.0 (TID 299) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:45 INFO TaskSetManager: Finished task 21.0 in stage 13.0 (TID 291) in 2247 ms on 10.0.0.133 (executor driver) (22/64) +26/04/01 08:49:45 INFO Executor: Running task 29.0 in stage 13.0 (TID 299) +26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:45 INFO Executor: Finished task 22.0 in stage 13.0 (TID 292). 6563 bytes result sent to driver +26/04/01 08:49:45 INFO TaskSetManager: Starting task 30.0 in stage 13.0 (TID 300) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9264 bytes) +26/04/01 08:49:45 INFO TaskSetManager: Finished task 22.0 in stage 13.0 (TID 292) in 2260 ms on 10.0.0.133 (executor driver) (23/64) +26/04/01 08:49:45 INFO Executor: Running task 30.0 in stage 13.0 (TID 300) +26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:45 INFO Executor: Finished task 23.0 in stage 13.0 (TID 293). 6563 bytes result sent to driver +26/04/01 08:49:45 INFO TaskSetManager: Starting task 31.0 in stage 13.0 (TID 301) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9263 bytes) +26/04/01 08:49:45 INFO TaskSetManager: Finished task 23.0 in stage 13.0 (TID 293) in 2256 ms on 10.0.0.133 (executor driver) (24/64) +26/04/01 08:49:45 INFO Executor: Running task 31.0 in stage 13.0 (TID 301) +26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:45 INFO Executor: Finished task 24.0 in stage 13.0 (TID 294). 6606 bytes result sent to driver +26/04/01 08:49:45 INFO TaskSetManager: Starting task 32.0 in stage 13.0 (TID 302) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:45 INFO TaskSetManager: Finished task 24.0 in stage 13.0 (TID 294) in 2250 ms on 10.0.0.133 (executor driver) (25/64) +26/04/01 08:49:45 INFO Executor: Running task 32.0 in stage 13.0 (TID 302) +26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:45 INFO Executor: Finished task 25.0 in stage 13.0 (TID 295). 6606 bytes result sent to driver +26/04/01 08:49:45 INFO TaskSetManager: Starting task 33.0 in stage 13.0 (TID 303) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:45 INFO TaskSetManager: Finished task 25.0 in stage 13.0 (TID 295) in 2255 ms on 10.0.0.133 (executor driver) (26/64) +26/04/01 08:49:45 INFO Executor: Running task 33.0 in stage 13.0 (TID 303) +26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:45 INFO Executor: Finished task 26.0 in stage 13.0 (TID 296). 6606 bytes result sent to driver +26/04/01 08:49:45 INFO TaskSetManager: Starting task 34.0 in stage 13.0 (TID 304) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:45 INFO TaskSetManager: Finished task 26.0 in stage 13.0 (TID 296) in 2251 ms on 10.0.0.133 (executor driver) (27/64) +26/04/01 08:49:45 INFO Executor: Running task 34.0 in stage 13.0 (TID 304) +26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:46 INFO Executor: Finished task 27.0 in stage 13.0 (TID 297). 6606 bytes result sent to driver +26/04/01 08:49:46 INFO TaskSetManager: Starting task 35.0 in stage 13.0 (TID 305) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:46 INFO TaskSetManager: Finished task 27.0 in stage 13.0 (TID 297) in 2226 ms on 10.0.0.133 (executor driver) (28/64) +26/04/01 08:49:46 INFO Executor: Running task 35.0 in stage 13.0 (TID 305) +26/04/01 08:49:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:47 INFO Executor: Finished task 32.0 in stage 13.0 (TID 302). 6563 bytes result sent to driver +26/04/01 08:49:47 INFO TaskSetManager: Starting task 36.0 in stage 13.0 (TID 306) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:47 INFO TaskSetManager: Finished task 32.0 in stage 13.0 (TID 302) in 1587 ms on 10.0.0.133 (executor driver) (29/64) +26/04/01 08:49:47 INFO Executor: Running task 36.0 in stage 13.0 (TID 306) +26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:47 INFO Executor: Finished task 33.0 in stage 13.0 (TID 303). 6563 bytes result sent to driver +26/04/01 08:49:47 INFO TaskSetManager: Starting task 37.0 in stage 13.0 (TID 307) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:47 INFO TaskSetManager: Finished task 33.0 in stage 13.0 (TID 303) in 1575 ms on 10.0.0.133 (executor driver) (30/64) +26/04/01 08:49:47 INFO Executor: Running task 37.0 in stage 13.0 (TID 307) +26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:47 INFO Executor: Finished task 34.0 in stage 13.0 (TID 304). 6563 bytes result sent to driver +26/04/01 08:49:47 INFO TaskSetManager: Starting task 38.0 in stage 13.0 (TID 308) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:47 INFO TaskSetManager: Finished task 34.0 in stage 13.0 (TID 304) in 1605 ms on 10.0.0.133 (executor driver) (31/64) +26/04/01 08:49:47 INFO Executor: Running task 38.0 in stage 13.0 (TID 308) +26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:47 INFO Executor: Finished task 28.0 in stage 13.0 (TID 298). 6606 bytes result sent to driver +26/04/01 08:49:47 INFO TaskSetManager: Starting task 39.0 in stage 13.0 (TID 309) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:47 INFO Executor: Running task 39.0 in stage 13.0 (TID 309) +26/04/01 08:49:47 INFO TaskSetManager: Finished task 28.0 in stage 13.0 (TID 298) in 2252 ms on 10.0.0.133 (executor driver) (32/64) +26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:47 INFO Executor: Finished task 29.0 in stage 13.0 (TID 299). 6606 bytes result sent to driver +26/04/01 08:49:47 INFO TaskSetManager: Starting task 40.0 in stage 13.0 (TID 310) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:47 INFO TaskSetManager: Finished task 29.0 in stage 13.0 (TID 299) in 2265 ms on 10.0.0.133 (executor driver) (33/64) +26/04/01 08:49:47 INFO Executor: Running task 40.0 in stage 13.0 (TID 310) +26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:47 INFO Executor: Finished task 30.0 in stage 13.0 (TID 300). 6606 bytes result sent to driver +26/04/01 08:49:47 INFO TaskSetManager: Starting task 41.0 in stage 13.0 (TID 311) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:47 INFO TaskSetManager: Finished task 30.0 in stage 13.0 (TID 300) in 2268 ms on 10.0.0.133 (executor driver) (34/64) +26/04/01 08:49:47 INFO Executor: Running task 41.0 in stage 13.0 (TID 311) +26/04/01 08:49:47 INFO Executor: Finished task 31.0 in stage 13.0 (TID 301). 6606 bytes result sent to driver +26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:47 INFO TaskSetManager: Starting task 42.0 in stage 13.0 (TID 312) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:47 INFO TaskSetManager: Finished task 31.0 in stage 13.0 (TID 301) in 2263 ms on 10.0.0.133 (executor driver) (35/64) +26/04/01 08:49:47 INFO Executor: Running task 42.0 in stage 13.0 (TID 312) +26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:48 INFO Executor: Finished task 35.0 in stage 13.0 (TID 305). 6563 bytes result sent to driver +26/04/01 08:49:48 INFO TaskSetManager: Starting task 43.0 in stage 13.0 (TID 313) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:48 INFO TaskSetManager: Finished task 35.0 in stage 13.0 (TID 305) in 1575 ms on 10.0.0.133 (executor driver) (36/64) +26/04/01 08:49:48 INFO Executor: Running task 43.0 in stage 13.0 (TID 313) +26/04/01 08:49:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:49 INFO Executor: Finished task 36.0 in stage 13.0 (TID 306). 6563 bytes result sent to driver +26/04/01 08:49:49 INFO TaskSetManager: Starting task 44.0 in stage 13.0 (TID 314) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:49 INFO TaskSetManager: Finished task 36.0 in stage 13.0 (TID 306) in 1569 ms on 10.0.0.133 (executor driver) (37/64) +26/04/01 08:49:49 INFO Executor: Running task 44.0 in stage 13.0 (TID 314) +26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:49 INFO Executor: Finished task 37.0 in stage 13.0 (TID 307). 6563 bytes result sent to driver +26/04/01 08:49:49 INFO TaskSetManager: Starting task 45.0 in stage 13.0 (TID 315) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:49 INFO Executor: Running task 45.0 in stage 13.0 (TID 315) +26/04/01 08:49:49 INFO TaskSetManager: Finished task 37.0 in stage 13.0 (TID 307) in 1562 ms on 10.0.0.133 (executor driver) (38/64) +26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:49 INFO Executor: Finished task 38.0 in stage 13.0 (TID 308). 6563 bytes result sent to driver +26/04/01 08:49:49 INFO TaskSetManager: Starting task 46.0 in stage 13.0 (TID 316) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:49 INFO Executor: Running task 46.0 in stage 13.0 (TID 316) +26/04/01 08:49:49 INFO TaskSetManager: Finished task 38.0 in stage 13.0 (TID 308) in 1564 ms on 10.0.0.133 (executor driver) (39/64) +26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:49 INFO Executor: Finished task 39.0 in stage 13.0 (TID 309). 6563 bytes result sent to driver +26/04/01 08:49:49 INFO TaskSetManager: Starting task 47.0 in stage 13.0 (TID 317) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:49 INFO TaskSetManager: Finished task 39.0 in stage 13.0 (TID 309) in 1571 ms on 10.0.0.133 (executor driver) (40/64) +26/04/01 08:49:49 INFO Executor: Running task 47.0 in stage 13.0 (TID 317) +26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:49 INFO Executor: Finished task 40.0 in stage 13.0 (TID 310). 6563 bytes result sent to driver +26/04/01 08:49:49 INFO TaskSetManager: Starting task 48.0 in stage 13.0 (TID 318) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:49 INFO TaskSetManager: Finished task 40.0 in stage 13.0 (TID 310) in 1570 ms on 10.0.0.133 (executor driver) (41/64) +26/04/01 08:49:49 INFO Executor: Running task 48.0 in stage 13.0 (TID 318) +26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:49 INFO Executor: Finished task 41.0 in stage 13.0 (TID 311). 6563 bytes result sent to driver +26/04/01 08:49:49 INFO TaskSetManager: Starting task 49.0 in stage 13.0 (TID 319) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:49 INFO TaskSetManager: Finished task 41.0 in stage 13.0 (TID 311) in 1570 ms on 10.0.0.133 (executor driver) (42/64) +26/04/01 08:49:49 INFO Executor: Running task 49.0 in stage 13.0 (TID 319) +26/04/01 08:49:49 INFO Executor: Finished task 42.0 in stage 13.0 (TID 312). 6563 bytes result sent to driver +26/04/01 08:49:49 INFO TaskSetManager: Starting task 50.0 in stage 13.0 (TID 320) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:49 INFO TaskSetManager: Finished task 42.0 in stage 13.0 (TID 312) in 1569 ms on 10.0.0.133 (executor driver) (43/64) +26/04/01 08:49:49 INFO Executor: Running task 50.0 in stage 13.0 (TID 320) +26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:50 INFO Executor: Finished task 43.0 in stage 13.0 (TID 313). 6563 bytes result sent to driver +26/04/01 08:49:50 INFO TaskSetManager: Starting task 51.0 in stage 13.0 (TID 321) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:50 INFO TaskSetManager: Finished task 43.0 in stage 13.0 (TID 313) in 1547 ms on 10.0.0.133 (executor driver) (44/64) +26/04/01 08:49:50 INFO Executor: Running task 51.0 in stage 13.0 (TID 321) +26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:50 INFO Executor: Finished task 44.0 in stage 13.0 (TID 314). 6563 bytes result sent to driver +26/04/01 08:49:50 INFO TaskSetManager: Starting task 52.0 in stage 13.0 (TID 322) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:50 INFO Executor: Running task 52.0 in stage 13.0 (TID 322) +26/04/01 08:49:50 INFO TaskSetManager: Finished task 44.0 in stage 13.0 (TID 314) in 1552 ms on 10.0.0.133 (executor driver) (45/64) +26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:50 INFO Executor: Finished task 45.0 in stage 13.0 (TID 315). 6563 bytes result sent to driver +26/04/01 08:49:50 INFO TaskSetManager: Starting task 53.0 in stage 13.0 (TID 323) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:50 INFO TaskSetManager: Finished task 45.0 in stage 13.0 (TID 315) in 1560 ms on 10.0.0.133 (executor driver) (46/64) +26/04/01 08:49:50 INFO Executor: Running task 53.0 in stage 13.0 (TID 323) +26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:50 INFO Executor: Finished task 46.0 in stage 13.0 (TID 316). 6563 bytes result sent to driver +26/04/01 08:49:50 INFO TaskSetManager: Starting task 54.0 in stage 13.0 (TID 324) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:50 INFO TaskSetManager: Finished task 46.0 in stage 13.0 (TID 316) in 1556 ms on 10.0.0.133 (executor driver) (47/64) +26/04/01 08:49:50 INFO Executor: Running task 54.0 in stage 13.0 (TID 324) +26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:50 INFO Executor: Finished task 47.0 in stage 13.0 (TID 317). 6563 bytes result sent to driver +26/04/01 08:49:50 INFO TaskSetManager: Starting task 55.0 in stage 13.0 (TID 325) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:50 INFO TaskSetManager: Finished task 47.0 in stage 13.0 (TID 317) in 1564 ms on 10.0.0.133 (executor driver) (48/64) +26/04/01 08:49:50 INFO Executor: Running task 55.0 in stage 13.0 (TID 325) +26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:50 INFO Executor: Finished task 48.0 in stage 13.0 (TID 318). 6563 bytes result sent to driver +26/04/01 08:49:50 INFO TaskSetManager: Starting task 56.0 in stage 13.0 (TID 326) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:50 INFO TaskSetManager: Finished task 48.0 in stage 13.0 (TID 318) in 1562 ms on 10.0.0.133 (executor driver) (49/64) +26/04/01 08:49:50 INFO Executor: Running task 56.0 in stage 13.0 (TID 326) +26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:51 INFO Executor: Finished task 49.0 in stage 13.0 (TID 319). 6563 bytes result sent to driver +26/04/01 08:49:51 INFO TaskSetManager: Starting task 57.0 in stage 13.0 (TID 327) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:51 INFO TaskSetManager: Finished task 49.0 in stage 13.0 (TID 319) in 1564 ms on 10.0.0.133 (executor driver) (50/64) +26/04/01 08:49:51 INFO Executor: Running task 57.0 in stage 13.0 (TID 327) +26/04/01 08:49:51 INFO Executor: Finished task 50.0 in stage 13.0 (TID 320). 6563 bytes result sent to driver +26/04/01 08:49:51 INFO TaskSetManager: Starting task 58.0 in stage 13.0 (TID 328) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:51 INFO TaskSetManager: Finished task 50.0 in stage 13.0 (TID 320) in 1564 ms on 10.0.0.133 (executor driver) (51/64) +26/04/01 08:49:51 INFO Executor: Running task 58.0 in stage 13.0 (TID 328) +26/04/01 08:49:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:51 INFO Executor: Finished task 51.0 in stage 13.0 (TID 321). 6563 bytes result sent to driver +26/04/01 08:49:51 INFO TaskSetManager: Starting task 59.0 in stage 13.0 (TID 329) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:51 INFO TaskSetManager: Finished task 51.0 in stage 13.0 (TID 321) in 1541 ms on 10.0.0.133 (executor driver) (52/64) +26/04/01 08:49:51 INFO Executor: Running task 59.0 in stage 13.0 (TID 329) +26/04/01 08:49:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:52 INFO Executor: Finished task 52.0 in stage 13.0 (TID 322). 6563 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 60.0 in stage 13.0 (TID 330) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 52.0 in stage 13.0 (TID 322) in 1556 ms on 10.0.0.133 (executor driver) (53/64) +26/04/01 08:49:52 INFO Executor: Running task 60.0 in stage 13.0 (TID 330) +26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:52 INFO Executor: Finished task 53.0 in stage 13.0 (TID 323). 6563 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 61.0 in stage 13.0 (TID 331) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9269 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 53.0 in stage 13.0 (TID 323) in 1567 ms on 10.0.0.133 (executor driver) (54/64) +26/04/01 08:49:52 INFO Executor: Running task 61.0 in stage 13.0 (TID 331) +26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:52 INFO Executor: Finished task 54.0 in stage 13.0 (TID 324). 6563 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 62.0 in stage 13.0 (TID 332) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 54.0 in stage 13.0 (TID 324) in 1570 ms on 10.0.0.133 (executor driver) (55/64) +26/04/01 08:49:52 INFO Executor: Running task 62.0 in stage 13.0 (TID 332) +26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:52 INFO Executor: Finished task 55.0 in stage 13.0 (TID 325). 6563 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 63.0 in stage 13.0 (TID 333) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9268 bytes) +26/04/01 08:49:52 INFO Executor: Running task 63.0 in stage 13.0 (TID 333) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 55.0 in stage 13.0 (TID 325) in 1563 ms on 10.0.0.133 (executor driver) (56/64) +26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:52 INFO Executor: Finished task 56.0 in stage 13.0 (TID 326). 6563 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 334) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 0.0 in stage 15.0 (TID 334) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 56.0 in stage 13.0 (TID 326) in 1567 ms on 10.0.0.133 (executor driver) (57/64) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:49:52 INFO Executor: Finished task 0.0 in stage 15.0 (TID 334). 33895 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 1.0 in stage 15.0 (TID 335) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 1.0 in stage 15.0 (TID 335) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 334) in 28 ms on 10.0.0.133 (executor driver) (1/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 1.0 in stage 15.0 (TID 335). 34070 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 2.0 in stage 15.0 (TID 336) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 2.0 in stage 15.0 (TID 336) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 1.0 in stage 15.0 (TID 335) in 7 ms on 10.0.0.133 (executor driver) (2/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 2.0 in stage 15.0 (TID 336). 33801 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 3.0 in stage 15.0 (TID 337) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 3.0 in stage 15.0 (TID 337) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 2.0 in stage 15.0 (TID 336) in 5 ms on 10.0.0.133 (executor driver) (3/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 3.0 in stage 15.0 (TID 337). 34328 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 4.0 in stage 15.0 (TID 338) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 4.0 in stage 15.0 (TID 338) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 3.0 in stage 15.0 (TID 337) in 5 ms on 10.0.0.133 (executor driver) (4/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 4.0 in stage 15.0 (TID 338). 34473 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 5.0 in stage 15.0 (TID 339) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 5.0 in stage 15.0 (TID 339) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 4.0 in stage 15.0 (TID 338) in 7 ms on 10.0.0.133 (executor driver) (5/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 5.0 in stage 15.0 (TID 339). 34156 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 6.0 in stage 15.0 (TID 340) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 6.0 in stage 15.0 (TID 340) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 5.0 in stage 15.0 (TID 339) in 4 ms on 10.0.0.133 (executor driver) (6/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 6.0 in stage 15.0 (TID 340). 33938 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 7.0 in stage 15.0 (TID 341) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 7.0 in stage 15.0 (TID 341) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 6.0 in stage 15.0 (TID 340) in 5 ms on 10.0.0.133 (executor driver) (7/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 7.0 in stage 15.0 (TID 341). 34062 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 8.0 in stage 15.0 (TID 342) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 8.0 in stage 15.0 (TID 342) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 7.0 in stage 15.0 (TID 341) in 5 ms on 10.0.0.133 (executor driver) (8/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 8.0 in stage 15.0 (TID 342). 33972 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 9.0 in stage 15.0 (TID 343) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 9.0 in stage 15.0 (TID 343) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 8.0 in stage 15.0 (TID 342) in 6 ms on 10.0.0.133 (executor driver) (9/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 9.0 in stage 15.0 (TID 343). 34979 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 10.0 in stage 15.0 (TID 344) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 10.0 in stage 15.0 (TID 344) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 9.0 in stage 15.0 (TID 343) in 6 ms on 10.0.0.133 (executor driver) (10/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 10.0 in stage 15.0 (TID 344). 34438 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 11.0 in stage 15.0 (TID 345) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 11.0 in stage 15.0 (TID 345) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 10.0 in stage 15.0 (TID 344) in 6 ms on 10.0.0.133 (executor driver) (11/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 11.0 in stage 15.0 (TID 345). 33620 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 12.0 in stage 15.0 (TID 346) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Finished task 57.0 in stage 13.0 (TID 327). 6606 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Running task 12.0 in stage 15.0 (TID 346) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 13.0 in stage 15.0 (TID 347) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 11.0 in stage 15.0 (TID 345) in 5 ms on 10.0.0.133 (executor driver) (12/200) +26/04/01 08:49:52 INFO Executor: Running task 13.0 in stage 15.0 (TID 347) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 57.0 in stage 13.0 (TID 327) in 1566 ms on 10.0.0.133 (executor driver) (58/64) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 58.0 in stage 13.0 (TID 328). 6606 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 14.0 in stage 15.0 (TID 348) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 14.0 in stage 15.0 (TID 348) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 58.0 in stage 13.0 (TID 328) in 1570 ms on 10.0.0.133 (executor driver) (59/64) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 12.0 in stage 15.0 (TID 346). 34084 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 15.0 in stage 15.0 (TID 349) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Finished task 13.0 in stage 15.0 (TID 347). 34520 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Running task 15.0 in stage 15.0 (TID 349) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 16.0 in stage 15.0 (TID 350) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 12.0 in stage 15.0 (TID 346) in 5 ms on 10.0.0.133 (executor driver) (13/200) +26/04/01 08:49:52 INFO Executor: Running task 16.0 in stage 15.0 (TID 350) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 13.0 in stage 15.0 (TID 347) in 5 ms on 10.0.0.133 (executor driver) (14/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 14.0 in stage 15.0 (TID 348). 34658 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 17.0 in stage 15.0 (TID 351) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 17.0 in stage 15.0 (TID 351) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 14.0 in stage 15.0 (TID 348) in 5 ms on 10.0.0.133 (executor driver) (15/200) +26/04/01 08:49:52 INFO Executor: Finished task 15.0 in stage 15.0 (TID 349). 34244 bytes result sent to driver +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO TaskSetManager: Starting task 18.0 in stage 15.0 (TID 352) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 18.0 in stage 15.0 (TID 352) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 15.0 in stage 15.0 (TID 349) in 5 ms on 10.0.0.133 (executor driver) (16/200) +26/04/01 08:49:52 INFO Executor: Finished task 16.0 in stage 15.0 (TID 350). 33628 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 19.0 in stage 15.0 (TID 353) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 19.0 in stage 15.0 (TID 353) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 16.0 in stage 15.0 (TID 350) in 6 ms on 10.0.0.133 (executor driver) (17/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 17.0 in stage 15.0 (TID 351). 34240 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 20.0 in stage 15.0 (TID 354) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 20.0 in stage 15.0 (TID 354) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 17.0 in stage 15.0 (TID 351) in 5 ms on 10.0.0.133 (executor driver) (18/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 18.0 in stage 15.0 (TID 352). 34217 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 19.0 in stage 15.0 (TID 353). 35085 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 21.0 in stage 15.0 (TID 355) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 21.0 in stage 15.0 (TID 355) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 22.0 in stage 15.0 (TID 356) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 18.0 in stage 15.0 (TID 352) in 6 ms on 10.0.0.133 (executor driver) (19/200) +26/04/01 08:49:52 INFO Executor: Running task 22.0 in stage 15.0 (TID 356) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 19.0 in stage 15.0 (TID 353) in 6 ms on 10.0.0.133 (executor driver) (20/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms +26/04/01 08:49:52 INFO Executor: Finished task 20.0 in stage 15.0 (TID 354). 34602 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 23.0 in stage 15.0 (TID 357) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 20.0 in stage 15.0 (TID 354) in 5 ms on 10.0.0.133 (executor driver) (21/200) +26/04/01 08:49:52 INFO Executor: Running task 23.0 in stage 15.0 (TID 357) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 22.0 in stage 15.0 (TID 356). 34615 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 21.0 in stage 15.0 (TID 355). 34867 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 24.0 in stage 15.0 (TID 358) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 24.0 in stage 15.0 (TID 358) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 25.0 in stage 15.0 (TID 359) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 22.0 in stage 15.0 (TID 356) in 5 ms on 10.0.0.133 (executor driver) (22/200) +26/04/01 08:49:52 INFO Executor: Running task 25.0 in stage 15.0 (TID 359) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 21.0 in stage 15.0 (TID 355) in 6 ms on 10.0.0.133 (executor driver) (23/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 23.0 in stage 15.0 (TID 357). 34082 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 26.0 in stage 15.0 (TID 360) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 26.0 in stage 15.0 (TID 360) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 23.0 in stage 15.0 (TID 357) in 6 ms on 10.0.0.133 (executor driver) (24/200) +26/04/01 08:49:52 INFO Executor: Finished task 25.0 in stage 15.0 (TID 359). 34085 bytes result sent to driver +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO TaskSetManager: Starting task 27.0 in stage 15.0 (TID 361) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 27.0 in stage 15.0 (TID 361) +26/04/01 08:49:52 INFO Executor: Finished task 24.0 in stage 15.0 (TID 358). 34683 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Finished task 25.0 in stage 15.0 (TID 359) in 5 ms on 10.0.0.133 (executor driver) (25/200) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 28.0 in stage 15.0 (TID 362) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 28.0 in stage 15.0 (TID 362) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 24.0 in stage 15.0 (TID 358) in 5 ms on 10.0.0.133 (executor driver) (26/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 26.0 in stage 15.0 (TID 360). 33825 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 29.0 in stage 15.0 (TID 363) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 29.0 in stage 15.0 (TID 363) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 26.0 in stage 15.0 (TID 360) in 4 ms on 10.0.0.133 (executor driver) (27/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 27.0 in stage 15.0 (TID 361). 34141 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 30.0 in stage 15.0 (TID 364) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Finished task 28.0 in stage 15.0 (TID 362). 34850 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Running task 30.0 in stage 15.0 (TID 364) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 31.0 in stage 15.0 (TID 365) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 27.0 in stage 15.0 (TID 361) in 5 ms on 10.0.0.133 (executor driver) (28/200) +26/04/01 08:49:52 INFO Executor: Running task 31.0 in stage 15.0 (TID 365) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 28.0 in stage 15.0 (TID 362) in 5 ms on 10.0.0.133 (executor driver) (29/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 29.0 in stage 15.0 (TID 363). 33967 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 32.0 in stage 15.0 (TID 366) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Finished task 30.0 in stage 15.0 (TID 364). 33979 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 31.0 in stage 15.0 (TID 365). 34381 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Running task 32.0 in stage 15.0 (TID 366) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 29.0 in stage 15.0 (TID 363) in 7 ms on 10.0.0.133 (executor driver) (30/200) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 33.0 in stage 15.0 (TID 367) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 33.0 in stage 15.0 (TID 367) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 34.0 in stage 15.0 (TID 368) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 30.0 in stage 15.0 (TID 364) in 6 ms on 10.0.0.133 (executor driver) (31/200) +26/04/01 08:49:52 INFO Executor: Running task 34.0 in stage 15.0 (TID 368) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 31.0 in stage 15.0 (TID 365) in 6 ms on 10.0.0.133 (executor driver) (32/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 32.0 in stage 15.0 (TID 366). 34649 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 34.0 in stage 15.0 (TID 368). 34290 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 33.0 in stage 15.0 (TID 367). 34079 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 35.0 in stage 15.0 (TID 369) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 35.0 in stage 15.0 (TID 369) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 36.0 in stage 15.0 (TID 370) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 37.0 in stage 15.0 (TID 371) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 36.0 in stage 15.0 (TID 370) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 34.0 in stage 15.0 (TID 368) in 6 ms on 10.0.0.133 (executor driver) (33/200) +26/04/01 08:49:52 INFO Executor: Running task 37.0 in stage 15.0 (TID 371) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 33.0 in stage 15.0 (TID 367) in 6 ms on 10.0.0.133 (executor driver) (34/200) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 32.0 in stage 15.0 (TID 366) in 6 ms on 10.0.0.133 (executor driver) (35/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 35.0 in stage 15.0 (TID 369). 34960 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 36.0 in stage 15.0 (TID 370). 34927 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 37.0 in stage 15.0 (TID 371). 34242 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 38.0 in stage 15.0 (TID 372) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 38.0 in stage 15.0 (TID 372) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 39.0 in stage 15.0 (TID 373) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 35.0 in stage 15.0 (TID 369) in 5 ms on 10.0.0.133 (executor driver) (36/200) +26/04/01 08:49:52 INFO Executor: Running task 39.0 in stage 15.0 (TID 373) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 40.0 in stage 15.0 (TID 374) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 37.0 in stage 15.0 (TID 371) in 5 ms on 10.0.0.133 (executor driver) (37/200) +26/04/01 08:49:52 INFO Executor: Running task 40.0 in stage 15.0 (TID 374) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 36.0 in stage 15.0 (TID 370) in 5 ms on 10.0.0.133 (executor driver) (38/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 38.0 in stage 15.0 (TID 372). 33723 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 39.0 in stage 15.0 (TID 373). 33734 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 41.0 in stage 15.0 (TID 375) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 41.0 in stage 15.0 (TID 375) +26/04/01 08:49:52 INFO Executor: Finished task 40.0 in stage 15.0 (TID 374). 34627 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 42.0 in stage 15.0 (TID 376) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 39.0 in stage 15.0 (TID 373) in 6 ms on 10.0.0.133 (executor driver) (39/200) +26/04/01 08:49:52 INFO Executor: Running task 42.0 in stage 15.0 (TID 376) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 43.0 in stage 15.0 (TID 377) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO TaskSetManager: Finished task 38.0 in stage 15.0 (TID 372) in 7 ms on 10.0.0.133 (executor driver) (40/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO TaskSetManager: Finished task 40.0 in stage 15.0 (TID 374) in 6 ms on 10.0.0.133 (executor driver) (41/200) +26/04/01 08:49:52 INFO Executor: Running task 43.0 in stage 15.0 (TID 377) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 41.0 in stage 15.0 (TID 375). 34443 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 44.0 in stage 15.0 (TID 378) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 41.0 in stage 15.0 (TID 375) in 4 ms on 10.0.0.133 (executor driver) (42/200) +26/04/01 08:49:52 INFO Executor: Running task 44.0 in stage 15.0 (TID 378) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 42.0 in stage 15.0 (TID 376). 34213 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 43.0 in stage 15.0 (TID 377). 33552 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 45.0 in stage 15.0 (TID 379) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 45.0 in stage 15.0 (TID 379) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 46.0 in stage 15.0 (TID 380) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 42.0 in stage 15.0 (TID 376) in 5 ms on 10.0.0.133 (executor driver) (43/200) +26/04/01 08:49:52 INFO Executor: Running task 46.0 in stage 15.0 (TID 380) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 43.0 in stage 15.0 (TID 377) in 5 ms on 10.0.0.133 (executor driver) (44/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 44.0 in stage 15.0 (TID 378). 33901 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 47.0 in stage 15.0 (TID 381) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 47.0 in stage 15.0 (TID 381) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 44.0 in stage 15.0 (TID 378) in 5 ms on 10.0.0.133 (executor driver) (45/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 45.0 in stage 15.0 (TID 379). 34036 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 48.0 in stage 15.0 (TID 382) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 48.0 in stage 15.0 (TID 382) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 45.0 in stage 15.0 (TID 379) in 5 ms on 10.0.0.133 (executor driver) (46/200) +26/04/01 08:49:52 INFO Executor: Finished task 46.0 in stage 15.0 (TID 380). 34072 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 49.0 in stage 15.0 (TID 383) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 49.0 in stage 15.0 (TID 383) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 46.0 in stage 15.0 (TID 380) in 5 ms on 10.0.0.133 (executor driver) (47/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 47.0 in stage 15.0 (TID 381). 34470 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 50.0 in stage 15.0 (TID 384) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 50.0 in stage 15.0 (TID 384) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 47.0 in stage 15.0 (TID 381) in 4 ms on 10.0.0.133 (executor driver) (48/200) +26/04/01 08:49:52 INFO Executor: Finished task 49.0 in stage 15.0 (TID 383). 33847 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 48.0 in stage 15.0 (TID 382). 33975 bytes result sent to driver +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO TaskSetManager: Starting task 51.0 in stage 15.0 (TID 385) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 51.0 in stage 15.0 (TID 385) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 52.0 in stage 15.0 (TID 386) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 52.0 in stage 15.0 (TID 386) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 49.0 in stage 15.0 (TID 383) in 6 ms on 10.0.0.133 (executor driver) (49/200) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 48.0 in stage 15.0 (TID 382) in 7 ms on 10.0.0.133 (executor driver) (50/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 50.0 in stage 15.0 (TID 384). 34504 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 53.0 in stage 15.0 (TID 387) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 53.0 in stage 15.0 (TID 387) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 50.0 in stage 15.0 (TID 384) in 7 ms on 10.0.0.133 (executor driver) (51/200) +26/04/01 08:49:52 INFO Executor: Finished task 52.0 in stage 15.0 (TID 386). 34119 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 51.0 in stage 15.0 (TID 385). 34243 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 54.0 in stage 15.0 (TID 388) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO Executor: Running task 54.0 in stage 15.0 (TID 388) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 55.0 in stage 15.0 (TID 389) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO TaskSetManager: Finished task 52.0 in stage 15.0 (TID 386) in 6 ms on 10.0.0.133 (executor driver) (52/200) +26/04/01 08:49:52 INFO Executor: Running task 55.0 in stage 15.0 (TID 389) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 51.0 in stage 15.0 (TID 385) in 6 ms on 10.0.0.133 (executor driver) (53/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 53.0 in stage 15.0 (TID 387). 34449 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 56.0 in stage 15.0 (TID 390) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 56.0 in stage 15.0 (TID 390) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 53.0 in stage 15.0 (TID 387) in 6 ms on 10.0.0.133 (executor driver) (54/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 54.0 in stage 15.0 (TID 388). 34702 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 55.0 in stage 15.0 (TID 389). 34244 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 57.0 in stage 15.0 (TID 391) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 57.0 in stage 15.0 (TID 391) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 58.0 in stage 15.0 (TID 392) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 54.0 in stage 15.0 (TID 388) in 6 ms on 10.0.0.133 (executor driver) (55/200) +26/04/01 08:49:52 INFO Executor: Running task 58.0 in stage 15.0 (TID 392) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 55.0 in stage 15.0 (TID 389) in 6 ms on 10.0.0.133 (executor driver) (56/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 56.0 in stage 15.0 (TID 390). 34901 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 59.0 in stage 15.0 (TID 393) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 59.0 in stage 15.0 (TID 393) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 56.0 in stage 15.0 (TID 390) in 6 ms on 10.0.0.133 (executor driver) (57/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 57.0 in stage 15.0 (TID 391). 34181 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 58.0 in stage 15.0 (TID 392). 34726 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 60.0 in stage 15.0 (TID 394) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 60.0 in stage 15.0 (TID 394) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 61.0 in stage 15.0 (TID 395) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 57.0 in stage 15.0 (TID 391) in 7 ms on 10.0.0.133 (executor driver) (58/200) +26/04/01 08:49:52 INFO Executor: Running task 61.0 in stage 15.0 (TID 395) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 58.0 in stage 15.0 (TID 392) in 6 ms on 10.0.0.133 (executor driver) (59/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 59.0 in stage 15.0 (TID 393). 34502 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 62.0 in stage 15.0 (TID 396) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 59.0 in stage 15.0 (TID 393) in 6 ms on 10.0.0.133 (executor driver) (60/200) +26/04/01 08:49:52 INFO Executor: Running task 62.0 in stage 15.0 (TID 396) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 61.0 in stage 15.0 (TID 395). 33977 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 63.0 in stage 15.0 (TID 397) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 61.0 in stage 15.0 (TID 395) in 6 ms on 10.0.0.133 (executor driver) (61/200) +26/04/01 08:49:52 INFO Executor: Finished task 60.0 in stage 15.0 (TID 394). 34580 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Running task 63.0 in stage 15.0 (TID 397) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 64.0 in stage 15.0 (TID 398) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 64.0 in stage 15.0 (TID 398) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 60.0 in stage 15.0 (TID 394) in 6 ms on 10.0.0.133 (executor driver) (62/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 62.0 in stage 15.0 (TID 396). 34073 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 65.0 in stage 15.0 (TID 399) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 65.0 in stage 15.0 (TID 399) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 62.0 in stage 15.0 (TID 396) in 5 ms on 10.0.0.133 (executor driver) (63/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 63.0 in stage 15.0 (TID 397). 33522 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 66.0 in stage 15.0 (TID 400) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 66.0 in stage 15.0 (TID 400) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 63.0 in stage 15.0 (TID 397) in 6 ms on 10.0.0.133 (executor driver) (64/200) +26/04/01 08:49:52 INFO Executor: Finished task 64.0 in stage 15.0 (TID 398). 33929 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 67.0 in stage 15.0 (TID 401) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 67.0 in stage 15.0 (TID 401) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 64.0 in stage 15.0 (TID 398) in 6 ms on 10.0.0.133 (executor driver) (65/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 65.0 in stage 15.0 (TID 399). 34066 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 68.0 in stage 15.0 (TID 402) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 65.0 in stage 15.0 (TID 399) in 6 ms on 10.0.0.133 (executor driver) (66/200) +26/04/01 08:49:52 INFO Executor: Running task 68.0 in stage 15.0 (TID 402) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 67.0 in stage 15.0 (TID 401). 34324 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 66.0 in stage 15.0 (TID 400). 34407 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 69.0 in stage 15.0 (TID 403) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 69.0 in stage 15.0 (TID 403) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 70.0 in stage 15.0 (TID 404) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 67.0 in stage 15.0 (TID 401) in 5 ms on 10.0.0.133 (executor driver) (67/200) +26/04/01 08:49:52 INFO Executor: Running task 70.0 in stage 15.0 (TID 404) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 66.0 in stage 15.0 (TID 400) in 5 ms on 10.0.0.133 (executor driver) (68/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 68.0 in stage 15.0 (TID 402). 33789 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 71.0 in stage 15.0 (TID 405) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 71.0 in stage 15.0 (TID 405) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 68.0 in stage 15.0 (TID 402) in 5 ms on 10.0.0.133 (executor driver) (69/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 70.0 in stage 15.0 (TID 404). 34912 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 72.0 in stage 15.0 (TID 406) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 72.0 in stage 15.0 (TID 406) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 70.0 in stage 15.0 (TID 404) in 5 ms on 10.0.0.133 (executor driver) (70/200) +26/04/01 08:49:52 INFO Executor: Finished task 69.0 in stage 15.0 (TID 403). 33810 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 73.0 in stage 15.0 (TID 407) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 73.0 in stage 15.0 (TID 407) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 69.0 in stage 15.0 (TID 403) in 5 ms on 10.0.0.133 (executor driver) (71/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 71.0 in stage 15.0 (TID 405). 34265 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 74.0 in stage 15.0 (TID 408) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 74.0 in stage 15.0 (TID 408) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 71.0 in stage 15.0 (TID 405) in 5 ms on 10.0.0.133 (executor driver) (72/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 72.0 in stage 15.0 (TID 406). 34272 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 75.0 in stage 15.0 (TID 409) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 75.0 in stage 15.0 (TID 409) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 72.0 in stage 15.0 (TID 406) in 6 ms on 10.0.0.133 (executor driver) (73/200) +26/04/01 08:49:52 INFO Executor: Finished task 73.0 in stage 15.0 (TID 407). 34476 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 76.0 in stage 15.0 (TID 410) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 76.0 in stage 15.0 (TID 410) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 73.0 in stage 15.0 (TID 407) in 5 ms on 10.0.0.133 (executor driver) (74/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 74.0 in stage 15.0 (TID 408). 33908 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 77.0 in stage 15.0 (TID 411) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 77.0 in stage 15.0 (TID 411) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 74.0 in stage 15.0 (TID 408) in 4 ms on 10.0.0.133 (executor driver) (75/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 75.0 in stage 15.0 (TID 409). 34266 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 78.0 in stage 15.0 (TID 412) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Finished task 76.0 in stage 15.0 (TID 410). 34196 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Running task 78.0 in stage 15.0 (TID 412) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 79.0 in stage 15.0 (TID 413) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 75.0 in stage 15.0 (TID 409) in 4 ms on 10.0.0.133 (executor driver) (76/200) +26/04/01 08:49:52 INFO Executor: Running task 79.0 in stage 15.0 (TID 413) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 76.0 in stage 15.0 (TID 410) in 4 ms on 10.0.0.133 (executor driver) (77/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 77.0 in stage 15.0 (TID 411). 34812 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 80.0 in stage 15.0 (TID 414) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 80.0 in stage 15.0 (TID 414) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 77.0 in stage 15.0 (TID 411) in 5 ms on 10.0.0.133 (executor driver) (78/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.6 KiB) non-empty blocks including 8 (31.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 78.0 in stage 15.0 (TID 412). 34141 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 81.0 in stage 15.0 (TID 415) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Finished task 79.0 in stage 15.0 (TID 413). 34085 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Running task 81.0 in stage 15.0 (TID 415) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 82.0 in stage 15.0 (TID 416) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 78.0 in stage 15.0 (TID 412) in 5 ms on 10.0.0.133 (executor driver) (79/200) +26/04/01 08:49:52 INFO Executor: Running task 82.0 in stage 15.0 (TID 416) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 79.0 in stage 15.0 (TID 413) in 5 ms on 10.0.0.133 (executor driver) (80/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 80.0 in stage 15.0 (TID 414). 34593 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 83.0 in stage 15.0 (TID 417) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 83.0 in stage 15.0 (TID 417) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 80.0 in stage 15.0 (TID 414) in 4 ms on 10.0.0.133 (executor driver) (81/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 81.0 in stage 15.0 (TID 415). 33935 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 84.0 in stage 15.0 (TID 418) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Finished task 82.0 in stage 15.0 (TID 416). 34466 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Running task 84.0 in stage 15.0 (TID 418) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 85.0 in stage 15.0 (TID 419) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 81.0 in stage 15.0 (TID 415) in 5 ms on 10.0.0.133 (executor driver) (82/200) +26/04/01 08:49:52 INFO Executor: Running task 85.0 in stage 15.0 (TID 419) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 82.0 in stage 15.0 (TID 416) in 4 ms on 10.0.0.133 (executor driver) (83/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 83.0 in stage 15.0 (TID 417). 34273 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 86.0 in stage 15.0 (TID 420) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 86.0 in stage 15.0 (TID 420) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 83.0 in stage 15.0 (TID 417) in 5 ms on 10.0.0.133 (executor driver) (84/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 84.0 in stage 15.0 (TID 418). 33511 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 85.0 in stage 15.0 (TID 419). 34506 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 87.0 in stage 15.0 (TID 421) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 87.0 in stage 15.0 (TID 421) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 88.0 in stage 15.0 (TID 422) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 84.0 in stage 15.0 (TID 418) in 5 ms on 10.0.0.133 (executor driver) (85/200) +26/04/01 08:49:52 INFO Executor: Running task 88.0 in stage 15.0 (TID 422) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 85.0 in stage 15.0 (TID 419) in 5 ms on 10.0.0.133 (executor driver) (86/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 86.0 in stage 15.0 (TID 420). 34439 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 89.0 in stage 15.0 (TID 423) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 89.0 in stage 15.0 (TID 423) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 86.0 in stage 15.0 (TID 420) in 6 ms on 10.0.0.133 (executor driver) (87/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO Executor: Finished task 88.0 in stage 15.0 (TID 422). 34788 bytes result sent to driver +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 87.0 in stage 15.0 (TID 421). 33774 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 90.0 in stage 15.0 (TID 424) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 90.0 in stage 15.0 (TID 424) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 91.0 in stage 15.0 (TID 425) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 88.0 in stage 15.0 (TID 422) in 5 ms on 10.0.0.133 (executor driver) (88/200) +26/04/01 08:49:52 INFO Executor: Running task 91.0 in stage 15.0 (TID 425) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 87.0 in stage 15.0 (TID 421) in 5 ms on 10.0.0.133 (executor driver) (89/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 89.0 in stage 15.0 (TID 423). 33884 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 92.0 in stage 15.0 (TID 426) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 92.0 in stage 15.0 (TID 426) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 89.0 in stage 15.0 (TID 423) in 5 ms on 10.0.0.133 (executor driver) (90/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 90.0 in stage 15.0 (TID 424). 33756 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 91.0 in stage 15.0 (TID 425). 33835 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 93.0 in stage 15.0 (TID 427) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 93.0 in stage 15.0 (TID 427) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 94.0 in stage 15.0 (TID 428) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 90.0 in stage 15.0 (TID 424) in 5 ms on 10.0.0.133 (executor driver) (91/200) +26/04/01 08:49:52 INFO Executor: Running task 94.0 in stage 15.0 (TID 428) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 91.0 in stage 15.0 (TID 425) in 5 ms on 10.0.0.133 (executor driver) (92/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 92.0 in stage 15.0 (TID 426). 33996 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 95.0 in stage 15.0 (TID 429) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 95.0 in stage 15.0 (TID 429) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 92.0 in stage 15.0 (TID 426) in 6 ms on 10.0.0.133 (executor driver) (93/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 93.0 in stage 15.0 (TID 427). 34038 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 94.0 in stage 15.0 (TID 428). 34594 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 96.0 in stage 15.0 (TID 430) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 96.0 in stage 15.0 (TID 430) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 97.0 in stage 15.0 (TID 431) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 93.0 in stage 15.0 (TID 427) in 6 ms on 10.0.0.133 (executor driver) (94/200) +26/04/01 08:49:52 INFO Executor: Running task 97.0 in stage 15.0 (TID 431) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 94.0 in stage 15.0 (TID 428) in 7 ms on 10.0.0.133 (executor driver) (95/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 95.0 in stage 15.0 (TID 429). 33891 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 98.0 in stage 15.0 (TID 432) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 98.0 in stage 15.0 (TID 432) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 95.0 in stage 15.0 (TID 429) in 4 ms on 10.0.0.133 (executor driver) (96/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 97.0 in stage 15.0 (TID 431). 33883 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 96.0 in stage 15.0 (TID 430). 34205 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 99.0 in stage 15.0 (TID 433) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 99.0 in stage 15.0 (TID 433) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 100.0 in stage 15.0 (TID 434) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 97.0 in stage 15.0 (TID 431) in 4 ms on 10.0.0.133 (executor driver) (97/200) +26/04/01 08:49:52 INFO Executor: Running task 100.0 in stage 15.0 (TID 434) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 96.0 in stage 15.0 (TID 430) in 5 ms on 10.0.0.133 (executor driver) (98/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 98.0 in stage 15.0 (TID 432). 34607 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 101.0 in stage 15.0 (TID 435) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 101.0 in stage 15.0 (TID 435) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 98.0 in stage 15.0 (TID 432) in 4 ms on 10.0.0.133 (executor driver) (99/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 99.0 in stage 15.0 (TID 433). 34229 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 100.0 in stage 15.0 (TID 434). 34078 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 102.0 in stage 15.0 (TID 436) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 102.0 in stage 15.0 (TID 436) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 103.0 in stage 15.0 (TID 437) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 99.0 in stage 15.0 (TID 433) in 5 ms on 10.0.0.133 (executor driver) (100/200) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 100.0 in stage 15.0 (TID 434) in 5 ms on 10.0.0.133 (executor driver) (101/200) +26/04/01 08:49:52 INFO Executor: Running task 103.0 in stage 15.0 (TID 437) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 101.0 in stage 15.0 (TID 435). 33625 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 104.0 in stage 15.0 (TID 438) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 104.0 in stage 15.0 (TID 438) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 101.0 in stage 15.0 (TID 435) in 4 ms on 10.0.0.133 (executor driver) (102/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 102.0 in stage 15.0 (TID 436). 34103 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 105.0 in stage 15.0 (TID 439) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 105.0 in stage 15.0 (TID 439) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 102.0 in stage 15.0 (TID 436) in 4 ms on 10.0.0.133 (executor driver) (103/200) +26/04/01 08:49:52 INFO Executor: Finished task 103.0 in stage 15.0 (TID 437). 34407 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 106.0 in stage 15.0 (TID 440) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 106.0 in stage 15.0 (TID 440) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 103.0 in stage 15.0 (TID 437) in 5 ms on 10.0.0.133 (executor driver) (104/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 104.0 in stage 15.0 (TID 438). 34155 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 107.0 in stage 15.0 (TID 441) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 107.0 in stage 15.0 (TID 441) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 104.0 in stage 15.0 (TID 438) in 4 ms on 10.0.0.133 (executor driver) (105/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 105.0 in stage 15.0 (TID 439). 33868 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 108.0 in stage 15.0 (TID 442) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 108.0 in stage 15.0 (TID 442) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 105.0 in stage 15.0 (TID 439) in 4 ms on 10.0.0.133 (executor driver) (106/200) +26/04/01 08:49:52 INFO Executor: Finished task 106.0 in stage 15.0 (TID 440). 33851 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 109.0 in stage 15.0 (TID 443) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 109.0 in stage 15.0 (TID 443) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 106.0 in stage 15.0 (TID 440) in 4 ms on 10.0.0.133 (executor driver) (107/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 107.0 in stage 15.0 (TID 441). 33898 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 110.0 in stage 15.0 (TID 444) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 110.0 in stage 15.0 (TID 444) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 107.0 in stage 15.0 (TID 441) in 4 ms on 10.0.0.133 (executor driver) (108/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 108.0 in stage 15.0 (TID 442). 34303 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 109.0 in stage 15.0 (TID 443). 34424 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 111.0 in stage 15.0 (TID 445) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 111.0 in stage 15.0 (TID 445) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 112.0 in stage 15.0 (TID 446) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 108.0 in stage 15.0 (TID 442) in 5 ms on 10.0.0.133 (executor driver) (109/200) +26/04/01 08:49:52 INFO Executor: Running task 112.0 in stage 15.0 (TID 446) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 109.0 in stage 15.0 (TID 443) in 4 ms on 10.0.0.133 (executor driver) (110/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 110.0 in stage 15.0 (TID 444). 34141 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 113.0 in stage 15.0 (TID 447) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 113.0 in stage 15.0 (TID 447) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 110.0 in stage 15.0 (TID 444) in 3 ms on 10.0.0.133 (executor driver) (111/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 112.0 in stage 15.0 (TID 446). 34551 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 111.0 in stage 15.0 (TID 445). 34432 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 114.0 in stage 15.0 (TID 448) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 114.0 in stage 15.0 (TID 448) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 115.0 in stage 15.0 (TID 449) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 112.0 in stage 15.0 (TID 446) in 3 ms on 10.0.0.133 (executor driver) (112/200) +26/04/01 08:49:52 INFO Executor: Running task 115.0 in stage 15.0 (TID 449) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 111.0 in stage 15.0 (TID 445) in 3 ms on 10.0.0.133 (executor driver) (113/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 113.0 in stage 15.0 (TID 447). 34235 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 116.0 in stage 15.0 (TID 450) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 116.0 in stage 15.0 (TID 450) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 113.0 in stage 15.0 (TID 447) in 4 ms on 10.0.0.133 (executor driver) (114/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 114.0 in stage 15.0 (TID 448). 34058 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 117.0 in stage 15.0 (TID 451) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 117.0 in stage 15.0 (TID 451) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 114.0 in stage 15.0 (TID 448) in 4 ms on 10.0.0.133 (executor driver) (115/200) +26/04/01 08:49:52 INFO Executor: Finished task 115.0 in stage 15.0 (TID 449). 33531 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 118.0 in stage 15.0 (TID 452) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 118.0 in stage 15.0 (TID 452) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO TaskSetManager: Finished task 115.0 in stage 15.0 (TID 449) in 5 ms on 10.0.0.133 (executor driver) (116/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 116.0 in stage 15.0 (TID 450). 34105 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 119.0 in stage 15.0 (TID 453) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO Executor: Running task 119.0 in stage 15.0 (TID 453) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO TaskSetManager: Finished task 116.0 in stage 15.0 (TID 450) in 4 ms on 10.0.0.133 (executor driver) (117/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 117.0 in stage 15.0 (TID 451). 34192 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 120.0 in stage 15.0 (TID 454) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 117.0 in stage 15.0 (TID 451) in 3 ms on 10.0.0.133 (executor driver) (118/200) +26/04/01 08:49:52 INFO Executor: Running task 120.0 in stage 15.0 (TID 454) +26/04/01 08:49:52 INFO Executor: Finished task 118.0 in stage 15.0 (TID 452). 34647 bytes result sent to driver +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO TaskSetManager: Starting task 121.0 in stage 15.0 (TID 455) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 121.0 in stage 15.0 (TID 455) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 118.0 in stage 15.0 (TID 452) in 4 ms on 10.0.0.133 (executor driver) (119/200) +26/04/01 08:49:52 INFO Executor: Finished task 119.0 in stage 15.0 (TID 453). 33642 bytes result sent to driver +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO TaskSetManager: Starting task 122.0 in stage 15.0 (TID 456) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 122.0 in stage 15.0 (TID 456) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 119.0 in stage 15.0 (TID 453) in 4 ms on 10.0.0.133 (executor driver) (120/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 120.0 in stage 15.0 (TID 454). 33991 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 123.0 in stage 15.0 (TID 457) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 123.0 in stage 15.0 (TID 457) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 120.0 in stage 15.0 (TID 454) in 4 ms on 10.0.0.133 (executor driver) (121/200) +26/04/01 08:49:52 INFO Executor: Finished task 121.0 in stage 15.0 (TID 455). 34107 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 124.0 in stage 15.0 (TID 458) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO Executor: Running task 124.0 in stage 15.0 (TID 458) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 121.0 in stage 15.0 (TID 455) in 4 ms on 10.0.0.133 (executor driver) (122/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 122.0 in stage 15.0 (TID 456). 34345 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 125.0 in stage 15.0 (TID 459) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 125.0 in stage 15.0 (TID 459) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 122.0 in stage 15.0 (TID 456) in 4 ms on 10.0.0.133 (executor driver) (123/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 123.0 in stage 15.0 (TID 457). 34123 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 124.0 in stage 15.0 (TID 458). 34152 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 126.0 in stage 15.0 (TID 460) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 126.0 in stage 15.0 (TID 460) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 127.0 in stage 15.0 (TID 461) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 123.0 in stage 15.0 (TID 457) in 5 ms on 10.0.0.133 (executor driver) (124/200) +26/04/01 08:49:52 INFO Executor: Running task 127.0 in stage 15.0 (TID 461) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 124.0 in stage 15.0 (TID 458) in 4 ms on 10.0.0.133 (executor driver) (125/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 125.0 in stage 15.0 (TID 459). 34554 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 128.0 in stage 15.0 (TID 462) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 128.0 in stage 15.0 (TID 462) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 125.0 in stage 15.0 (TID 459) in 4 ms on 10.0.0.133 (executor driver) (126/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 126.0 in stage 15.0 (TID 460). 33922 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 129.0 in stage 15.0 (TID 463) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 129.0 in stage 15.0 (TID 463) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 126.0 in stage 15.0 (TID 460) in 4 ms on 10.0.0.133 (executor driver) (127/200) +26/04/01 08:49:52 INFO Executor: Finished task 127.0 in stage 15.0 (TID 461). 34650 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 130.0 in stage 15.0 (TID 464) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 127.0 in stage 15.0 (TID 461) in 5 ms on 10.0.0.133 (executor driver) (128/200) +26/04/01 08:49:52 INFO Executor: Running task 130.0 in stage 15.0 (TID 464) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 128.0 in stage 15.0 (TID 462). 34349 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 131.0 in stage 15.0 (TID 465) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 131.0 in stage 15.0 (TID 465) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.6 KiB) non-empty blocks including 8 (31.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO TaskSetManager: Finished task 128.0 in stage 15.0 (TID 462) in 5 ms on 10.0.0.133 (executor driver) (129/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 129.0 in stage 15.0 (TID 463). 34012 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 132.0 in stage 15.0 (TID 466) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 132.0 in stage 15.0 (TID 466) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 129.0 in stage 15.0 (TID 463) in 4 ms on 10.0.0.133 (executor driver) (130/200) +26/04/01 08:49:52 INFO Executor: Finished task 130.0 in stage 15.0 (TID 464). 35090 bytes result sent to driver +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO TaskSetManager: Starting task 133.0 in stage 15.0 (TID 467) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Running task 133.0 in stage 15.0 (TID 467) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 130.0 in stage 15.0 (TID 464) in 4 ms on 10.0.0.133 (executor driver) (131/200) +26/04/01 08:49:52 INFO Executor: Finished task 131.0 in stage 15.0 (TID 465). 33973 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 134.0 in stage 15.0 (TID 468) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 134.0 in stage 15.0 (TID 468) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 131.0 in stage 15.0 (TID 465) in 5 ms on 10.0.0.133 (executor driver) (132/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 132.0 in stage 15.0 (TID 466). 34084 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 135.0 in stage 15.0 (TID 469) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 135.0 in stage 15.0 (TID 469) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 132.0 in stage 15.0 (TID 466) in 7 ms on 10.0.0.133 (executor driver) (133/200) +26/04/01 08:49:52 INFO Executor: Finished task 133.0 in stage 15.0 (TID 467). 33807 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 136.0 in stage 15.0 (TID 470) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 136.0 in stage 15.0 (TID 470) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 133.0 in stage 15.0 (TID 467) in 7 ms on 10.0.0.133 (executor driver) (134/200) +26/04/01 08:49:52 INFO Executor: Finished task 134.0 in stage 15.0 (TID 468). 33755 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 137.0 in stage 15.0 (TID 471) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Running task 137.0 in stage 15.0 (TID 471) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 134.0 in stage 15.0 (TID 468) in 7 ms on 10.0.0.133 (executor driver) (135/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 135.0 in stage 15.0 (TID 469). 34404 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 138.0 in stage 15.0 (TID 472) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 138.0 in stage 15.0 (TID 472) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 135.0 in stage 15.0 (TID 469) in 4 ms on 10.0.0.133 (executor driver) (136/200) +26/04/01 08:49:52 INFO Executor: Finished task 136.0 in stage 15.0 (TID 470). 34430 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 137.0 in stage 15.0 (TID 471). 34756 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 139.0 in stage 15.0 (TID 473) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Running task 139.0 in stage 15.0 (TID 473) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 140.0 in stage 15.0 (TID 474) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 136.0 in stage 15.0 (TID 470) in 4 ms on 10.0.0.133 (executor driver) (137/200) +26/04/01 08:49:52 INFO Executor: Running task 140.0 in stage 15.0 (TID 474) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 137.0 in stage 15.0 (TID 471) in 4 ms on 10.0.0.133 (executor driver) (138/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 138.0 in stage 15.0 (TID 472). 33693 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 141.0 in stage 15.0 (TID 475) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 141.0 in stage 15.0 (TID 475) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 138.0 in stage 15.0 (TID 472) in 4 ms on 10.0.0.133 (executor driver) (139/200) +26/04/01 08:49:52 INFO Executor: Finished task 140.0 in stage 15.0 (TID 474). 33890 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 142.0 in stage 15.0 (TID 476) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Finished task 139.0 in stage 15.0 (TID 473). 34378 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Running task 142.0 in stage 15.0 (TID 476) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 143.0 in stage 15.0 (TID 477) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 140.0 in stage 15.0 (TID 474) in 3 ms on 10.0.0.133 (executor driver) (140/200) +26/04/01 08:49:52 INFO Executor: Running task 143.0 in stage 15.0 (TID 477) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO TaskSetManager: Finished task 139.0 in stage 15.0 (TID 473) in 3 ms on 10.0.0.133 (executor driver) (141/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 141.0 in stage 15.0 (TID 475). 34328 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 144.0 in stage 15.0 (TID 478) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 144.0 in stage 15.0 (TID 478) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 141.0 in stage 15.0 (TID 475) in 4 ms on 10.0.0.133 (executor driver) (142/200) +26/04/01 08:49:52 INFO Executor: Finished task 142.0 in stage 15.0 (TID 476). 34951 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 143.0 in stage 15.0 (TID 477). 34375 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 145.0 in stage 15.0 (TID 479) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Running task 145.0 in stage 15.0 (TID 479) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 146.0 in stage 15.0 (TID 480) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 142.0 in stage 15.0 (TID 476) in 4 ms on 10.0.0.133 (executor driver) (143/200) +26/04/01 08:49:52 INFO Executor: Running task 146.0 in stage 15.0 (TID 480) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 143.0 in stage 15.0 (TID 477) in 4 ms on 10.0.0.133 (executor driver) (144/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 144.0 in stage 15.0 (TID 478). 33655 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 147.0 in stage 15.0 (TID 481) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 147.0 in stage 15.0 (TID 481) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 144.0 in stage 15.0 (TID 478) in 5 ms on 10.0.0.133 (executor driver) (145/200) +26/04/01 08:49:52 INFO Executor: Finished task 145.0 in stage 15.0 (TID 479). 34189 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 148.0 in stage 15.0 (TID 482) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 148.0 in stage 15.0 (TID 482) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 145.0 in stage 15.0 (TID 479) in 4 ms on 10.0.0.133 (executor driver) (146/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 146.0 in stage 15.0 (TID 480). 33726 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 149.0 in stage 15.0 (TID 483) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 149.0 in stage 15.0 (TID 483) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 146.0 in stage 15.0 (TID 480) in 4 ms on 10.0.0.133 (executor driver) (147/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 147.0 in stage 15.0 (TID 481). 33597 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 150.0 in stage 15.0 (TID 484) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Finished task 148.0 in stage 15.0 (TID 482). 34244 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Running task 150.0 in stage 15.0 (TID 484) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 151.0 in stage 15.0 (TID 485) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 147.0 in stage 15.0 (TID 481) in 5 ms on 10.0.0.133 (executor driver) (148/200) +26/04/01 08:49:52 INFO Executor: Running task 151.0 in stage 15.0 (TID 485) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 148.0 in stage 15.0 (TID 482) in 4 ms on 10.0.0.133 (executor driver) (149/200) +26/04/01 08:49:52 INFO Executor: Finished task 149.0 in stage 15.0 (TID 483). 34799 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 152.0 in stage 15.0 (TID 486) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 152.0 in stage 15.0 (TID 486) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 149.0 in stage 15.0 (TID 483) in 4 ms on 10.0.0.133 (executor driver) (150/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 151.0 in stage 15.0 (TID 485). 34392 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 153.0 in stage 15.0 (TID 487) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 153.0 in stage 15.0 (TID 487) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 151.0 in stage 15.0 (TID 485) in 4 ms on 10.0.0.133 (executor driver) (151/200) +26/04/01 08:49:52 INFO Executor: Finished task 152.0 in stage 15.0 (TID 486). 34441 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 154.0 in stage 15.0 (TID 488) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 154.0 in stage 15.0 (TID 488) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 152.0 in stage 15.0 (TID 486) in 4 ms on 10.0.0.133 (executor driver) (152/200) +26/04/01 08:49:52 INFO Executor: Finished task 150.0 in stage 15.0 (TID 484). 34597 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 155.0 in stage 15.0 (TID 489) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 155.0 in stage 15.0 (TID 489) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 150.0 in stage 15.0 (TID 484) in 5 ms on 10.0.0.133 (executor driver) (153/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 153.0 in stage 15.0 (TID 487). 34127 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 156.0 in stage 15.0 (TID 490) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 156.0 in stage 15.0 (TID 490) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 153.0 in stage 15.0 (TID 487) in 4 ms on 10.0.0.133 (executor driver) (154/200) +26/04/01 08:49:52 INFO Executor: Finished task 154.0 in stage 15.0 (TID 488). 33726 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 155.0 in stage 15.0 (TID 489). 33738 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 157.0 in stage 15.0 (TID 491) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 157.0 in stage 15.0 (TID 491) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 158.0 in stage 15.0 (TID 492) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 158.0 in stage 15.0 (TID 492) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 154.0 in stage 15.0 (TID 488) in 4 ms on 10.0.0.133 (executor driver) (155/200) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 155.0 in stage 15.0 (TID 489) in 4 ms on 10.0.0.133 (executor driver) (156/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 157.0 in stage 15.0 (TID 491). 33697 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 159.0 in stage 15.0 (TID 493) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Finished task 156.0 in stage 15.0 (TID 490). 34636 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 158.0 in stage 15.0 (TID 492). 34759 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Running task 159.0 in stage 15.0 (TID 493) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 160.0 in stage 15.0 (TID 494) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 157.0 in stage 15.0 (TID 491) in 4 ms on 10.0.0.133 (executor driver) (157/200) +26/04/01 08:49:52 INFO Executor: Running task 160.0 in stage 15.0 (TID 494) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 161.0 in stage 15.0 (TID 495) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 156.0 in stage 15.0 (TID 490) in 4 ms on 10.0.0.133 (executor driver) (158/200) +26/04/01 08:49:52 INFO Executor: Running task 161.0 in stage 15.0 (TID 495) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 158.0 in stage 15.0 (TID 492) in 4 ms on 10.0.0.133 (executor driver) (159/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 159.0 in stage 15.0 (TID 493). 34519 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 161.0 in stage 15.0 (TID 495). 33835 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 162.0 in stage 15.0 (TID 496) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 162.0 in stage 15.0 (TID 496) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 163.0 in stage 15.0 (TID 497) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 159.0 in stage 15.0 (TID 493) in 5 ms on 10.0.0.133 (executor driver) (160/200) +26/04/01 08:49:52 INFO Executor: Running task 163.0 in stage 15.0 (TID 497) +26/04/01 08:49:52 INFO Executor: Finished task 160.0 in stage 15.0 (TID 494). 34867 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Finished task 161.0 in stage 15.0 (TID 495) in 5 ms on 10.0.0.133 (executor driver) (161/200) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 164.0 in stage 15.0 (TID 498) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 164.0 in stage 15.0 (TID 498) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 160.0 in stage 15.0 (TID 494) in 5 ms on 10.0.0.133 (executor driver) (162/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 162.0 in stage 15.0 (TID 496). 34295 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 165.0 in stage 15.0 (TID 499) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 162.0 in stage 15.0 (TID 496) in 4 ms on 10.0.0.133 (executor driver) (163/200) +26/04/01 08:49:52 INFO Executor: Running task 165.0 in stage 15.0 (TID 499) +26/04/01 08:49:52 INFO Executor: Finished task 164.0 in stage 15.0 (TID 498). 33851 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 163.0 in stage 15.0 (TID 497). 34082 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 166.0 in stage 15.0 (TID 500) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 166.0 in stage 15.0 (TID 500) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 167.0 in stage 15.0 (TID 501) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 164.0 in stage 15.0 (TID 498) in 4 ms on 10.0.0.133 (executor driver) (164/200) +26/04/01 08:49:52 INFO Executor: Running task 167.0 in stage 15.0 (TID 501) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 163.0 in stage 15.0 (TID 497) in 4 ms on 10.0.0.133 (executor driver) (165/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 165.0 in stage 15.0 (TID 499). 34396 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 168.0 in stage 15.0 (TID 502) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 168.0 in stage 15.0 (TID 502) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 165.0 in stage 15.0 (TID 499) in 4 ms on 10.0.0.133 (executor driver) (166/200) +26/04/01 08:49:52 INFO Executor: Finished task 167.0 in stage 15.0 (TID 501). 33887 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 166.0 in stage 15.0 (TID 500). 34848 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 169.0 in stage 15.0 (TID 503) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 169.0 in stage 15.0 (TID 503) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 170.0 in stage 15.0 (TID 504) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 167.0 in stage 15.0 (TID 501) in 4 ms on 10.0.0.133 (executor driver) (167/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO Executor: Running task 170.0 in stage 15.0 (TID 504) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO TaskSetManager: Finished task 166.0 in stage 15.0 (TID 500) in 4 ms on 10.0.0.133 (executor driver) (168/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 168.0 in stage 15.0 (TID 502). 33588 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 171.0 in stage 15.0 (TID 505) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 171.0 in stage 15.0 (TID 505) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 168.0 in stage 15.0 (TID 502) in 4 ms on 10.0.0.133 (executor driver) (169/200) +26/04/01 08:49:52 INFO Executor: Finished task 169.0 in stage 15.0 (TID 503). 34471 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 170.0 in stage 15.0 (TID 504). 34026 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 172.0 in stage 15.0 (TID 506) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Running task 172.0 in stage 15.0 (TID 506) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 173.0 in stage 15.0 (TID 507) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 169.0 in stage 15.0 (TID 503) in 4 ms on 10.0.0.133 (executor driver) (170/200) +26/04/01 08:49:52 INFO Executor: Running task 173.0 in stage 15.0 (TID 507) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 170.0 in stage 15.0 (TID 504) in 4 ms on 10.0.0.133 (executor driver) (171/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 171.0 in stage 15.0 (TID 505). 34099 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 174.0 in stage 15.0 (TID 508) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 174.0 in stage 15.0 (TID 508) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 171.0 in stage 15.0 (TID 505) in 4 ms on 10.0.0.133 (executor driver) (172/200) +26/04/01 08:49:52 INFO Executor: Finished task 172.0 in stage 15.0 (TID 506). 33919 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 173.0 in stage 15.0 (TID 507). 34566 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 175.0 in stage 15.0 (TID 509) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 175.0 in stage 15.0 (TID 509) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO TaskSetManager: Starting task 176.0 in stage 15.0 (TID 510) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO TaskSetManager: Finished task 172.0 in stage 15.0 (TID 506) in 4 ms on 10.0.0.133 (executor driver) (173/200) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 173.0 in stage 15.0 (TID 507) in 4 ms on 10.0.0.133 (executor driver) (174/200) +26/04/01 08:49:52 INFO Executor: Running task 176.0 in stage 15.0 (TID 510) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 174.0 in stage 15.0 (TID 508). 33575 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 177.0 in stage 15.0 (TID 511) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 174.0 in stage 15.0 (TID 508) in 7 ms on 10.0.0.133 (executor driver) (175/200) +26/04/01 08:49:52 INFO Executor: Finished task 175.0 in stage 15.0 (TID 509). 34385 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 178.0 in stage 15.0 (TID 512) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 175.0 in stage 15.0 (TID 509) in 6 ms on 10.0.0.133 (executor driver) (176/200) +26/04/01 08:49:52 INFO Executor: Running task 177.0 in stage 15.0 (TID 511) +26/04/01 08:49:52 INFO Executor: Running task 178.0 in stage 15.0 (TID 512) +26/04/01 08:49:52 INFO Executor: Finished task 176.0 in stage 15.0 (TID 510). 33718 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 179.0 in stage 15.0 (TID 513) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 179.0 in stage 15.0 (TID 513) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 176.0 in stage 15.0 (TID 510) in 7 ms on 10.0.0.133 (executor driver) (177/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 177.0 in stage 15.0 (TID 511). 34399 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 180.0 in stage 15.0 (TID 514) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Finished task 178.0 in stage 15.0 (TID 512). 34255 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 179.0 in stage 15.0 (TID 513). 34030 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Finished task 177.0 in stage 15.0 (TID 511) in 5 ms on 10.0.0.133 (executor driver) (178/200) +26/04/01 08:49:52 INFO Executor: Running task 180.0 in stage 15.0 (TID 514) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 181.0 in stage 15.0 (TID 515) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 181.0 in stage 15.0 (TID 515) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 182.0 in stage 15.0 (TID 516) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 178.0 in stage 15.0 (TID 512) in 5 ms on 10.0.0.133 (executor driver) (179/200) +26/04/01 08:49:52 INFO Executor: Running task 182.0 in stage 15.0 (TID 516) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 179.0 in stage 15.0 (TID 513) in 5 ms on 10.0.0.133 (executor driver) (180/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 180.0 in stage 15.0 (TID 514). 34531 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 182.0 in stage 15.0 (TID 516). 34252 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 183.0 in stage 15.0 (TID 517) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 183.0 in stage 15.0 (TID 517) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 184.0 in stage 15.0 (TID 518) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 180.0 in stage 15.0 (TID 514) in 4 ms on 10.0.0.133 (executor driver) (181/200) +26/04/01 08:49:52 INFO Executor: Running task 184.0 in stage 15.0 (TID 518) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 182.0 in stage 15.0 (TID 516) in 4 ms on 10.0.0.133 (executor driver) (182/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.6 KiB) non-empty blocks including 8 (31.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 181.0 in stage 15.0 (TID 515). 34196 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 185.0 in stage 15.0 (TID 519) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 185.0 in stage 15.0 (TID 519) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 181.0 in stage 15.0 (TID 515) in 5 ms on 10.0.0.133 (executor driver) (183/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 183.0 in stage 15.0 (TID 517). 34150 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 184.0 in stage 15.0 (TID 518). 34956 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 186.0 in stage 15.0 (TID 520) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 183.0 in stage 15.0 (TID 517) in 4 ms on 10.0.0.133 (executor driver) (184/200) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 187.0 in stage 15.0 (TID 521) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 186.0 in stage 15.0 (TID 520) +26/04/01 08:49:52 INFO Executor: Running task 187.0 in stage 15.0 (TID 521) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 184.0 in stage 15.0 (TID 518) in 4 ms on 10.0.0.133 (executor driver) (185/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO Executor: Finished task 185.0 in stage 15.0 (TID 519). 34506 bytes result sent to driver +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO TaskSetManager: Starting task 188.0 in stage 15.0 (TID 522) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO Executor: Running task 188.0 in stage 15.0 (TID 522) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO TaskSetManager: Finished task 185.0 in stage 15.0 (TID 519) in 4 ms on 10.0.0.133 (executor driver) (186/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 187.0 in stage 15.0 (TID 521). 34650 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 189.0 in stage 15.0 (TID 523) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 189.0 in stage 15.0 (TID 523) +26/04/01 08:49:52 INFO Executor: Finished task 186.0 in stage 15.0 (TID 520). 33801 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Finished task 187.0 in stage 15.0 (TID 521) in 5 ms on 10.0.0.133 (executor driver) (187/200) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 190.0 in stage 15.0 (TID 524) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Finished task 188.0 in stage 15.0 (TID 522). 34848 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Running task 190.0 in stage 15.0 (TID 524) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 191.0 in stage 15.0 (TID 525) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 186.0 in stage 15.0 (TID 520) in 5 ms on 10.0.0.133 (executor driver) (188/200) +26/04/01 08:49:52 INFO Executor: Running task 191.0 in stage 15.0 (TID 525) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 188.0 in stage 15.0 (TID 522) in 4 ms on 10.0.0.133 (executor driver) (189/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 189.0 in stage 15.0 (TID 523). 34333 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 192.0 in stage 15.0 (TID 526) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 192.0 in stage 15.0 (TID 526) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 189.0 in stage 15.0 (TID 523) in 5 ms on 10.0.0.133 (executor driver) (190/200) +26/04/01 08:49:52 INFO Executor: Finished task 191.0 in stage 15.0 (TID 525). 34559 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 193.0 in stage 15.0 (TID 527) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Finished task 190.0 in stage 15.0 (TID 524). 34397 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Running task 193.0 in stage 15.0 (TID 527) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 194.0 in stage 15.0 (TID 528) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 191.0 in stage 15.0 (TID 525) in 4 ms on 10.0.0.133 (executor driver) (191/200) +26/04/01 08:49:52 INFO Executor: Running task 194.0 in stage 15.0 (TID 528) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 190.0 in stage 15.0 (TID 524) in 4 ms on 10.0.0.133 (executor driver) (192/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 192.0 in stage 15.0 (TID 526). 33410 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 195.0 in stage 15.0 (TID 529) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 195.0 in stage 15.0 (TID 529) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 192.0 in stage 15.0 (TID 526) in 4 ms on 10.0.0.133 (executor driver) (193/200) +26/04/01 08:49:52 INFO Executor: Finished task 194.0 in stage 15.0 (TID 528). 34348 bytes result sent to driver +26/04/01 08:49:52 INFO Executor: Finished task 193.0 in stage 15.0 (TID 527). 34499 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 196.0 in stage 15.0 (TID 530) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Running task 196.0 in stage 15.0 (TID 530) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 197.0 in stage 15.0 (TID 531) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 194.0 in stage 15.0 (TID 528) in 4 ms on 10.0.0.133 (executor driver) (194/200) +26/04/01 08:49:52 INFO Executor: Running task 197.0 in stage 15.0 (TID 531) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 193.0 in stage 15.0 (TID 527) in 4 ms on 10.0.0.133 (executor driver) (195/200) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO Executor: Finished task 195.0 in stage 15.0 (TID 529). 34725 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 198.0 in stage 15.0 (TID 532) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 198.0 in stage 15.0 (TID 532) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 195.0 in stage 15.0 (TID 529) in 4 ms on 10.0.0.133 (executor driver) (196/200) +26/04/01 08:49:52 INFO Executor: Finished task 196.0 in stage 15.0 (TID 530). 34709 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Starting task 199.0 in stage 15.0 (TID 533) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9215 bytes) +26/04/01 08:49:52 INFO Executor: Running task 199.0 in stage 15.0 (TID 533) +26/04/01 08:49:52 INFO TaskSetManager: Finished task 196.0 in stage 15.0 (TID 530) in 3 ms on 10.0.0.133 (executor driver) (197/200) +26/04/01 08:49:52 INFO Executor: Finished task 197.0 in stage 15.0 (TID 531). 34348 bytes result sent to driver +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO TaskSetManager: Finished task 197.0 in stage 15.0 (TID 531) in 4 ms on 10.0.0.133 (executor driver) (198/200) +26/04/01 08:49:52 INFO Executor: Finished task 199.0 in stage 15.0 (TID 533). 34766 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Finished task 199.0 in stage 15.0 (TID 533) in 4 ms on 10.0.0.133 (executor driver) (199/200) +26/04/01 08:49:52 INFO Executor: Finished task 198.0 in stage 15.0 (TID 532). 34063 bytes result sent to driver +26/04/01 08:49:52 INFO TaskSetManager: Finished task 198.0 in stage 15.0 (TID 532) in 4 ms on 10.0.0.133 (executor driver) (200/200) +26/04/01 08:49:52 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool +26/04/01 08:49:52 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 230.196 s +26/04/01 08:49:52 INFO DAGScheduler: Job 14 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:49:52 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished +26/04/01 08:49:52 INFO DAGScheduler: Job 14 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 230.211398 s +26/04/01 08:49:52 INFO Utils: Coalesced 1600 broadcast batches into 1 (1088125 rows) +26/04/01 08:49:52 INFO MemoryStore: Block broadcast_22 stored as values in memory (estimated size 5.2 MiB, free 12.4 GiB) +26/04/01 08:49:52 INFO MemoryStore: Block broadcast_22_piece0 stored as bytes in memory (estimated size 4.0 MiB, free 12.4 GiB) +26/04/01 08:49:52 INFO BlockManagerInfo: Added broadcast_22_piece0 in memory on 10.0.0.133:59333 (size: 4.0 MiB, free: 12.6 GiB) +26/04/01 08:49:52 INFO MemoryStore: Block broadcast_22_piece1 stored as bytes in memory (estimated size 357.8 KiB, free 12.4 GiB) +26/04/01 08:49:52 INFO BlockManagerInfo: Added broadcast_22_piece1 in memory on 10.0.0.133:59333 (size: 357.8 KiB, free: 12.6 GiB) +26/04/01 08:49:52 INFO SparkContext: Created broadcast 22 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:49:52 INFO ShufflePartitionsUtil: For shuffle(1), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 08:49:52 INFO DAGScheduler: Registering RDD 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 5 +26/04/01 08:49:52 INFO DAGScheduler: Got map stage job 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions +26/04/01 08:49:52 INFO DAGScheduler: Final stage: ShuffleMapStage 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:49:52 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 16) +26/04/01 08:49:52 INFO DAGScheduler: Missing parents: List() +26/04/01 08:49:52 INFO DAGScheduler: Submitting ShuffleMapStage 17 (MapPartitionsRDD[39] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:49:52 INFO MemoryStore: Block broadcast_23 stored as values in memory (estimated size 21.7 KiB, free 12.4 GiB) +26/04/01 08:49:52 INFO MemoryStore: Block broadcast_23_piece0 stored as bytes in memory (estimated size 11.0 KiB, free 12.4 GiB) +26/04/01 08:49:52 INFO BlockManagerInfo: Added broadcast_23_piece0 in memory on 10.0.0.133:59333 (size: 11.0 KiB, free: 12.6 GiB) +26/04/01 08:49:52 INFO SparkContext: Created broadcast 23 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:49:52 INFO DAGScheduler: Submitting 200 missing tasks from ShuffleMapStage 17 (MapPartitionsRDD[39] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:49:52 INFO TaskSchedulerImpl: Adding task set 17.0 with 200 tasks resource profile 0 +26/04/01 08:49:52 INFO TaskSetManager: Starting task 0.0 in stage 17.0 (TID 534) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 1.0 in stage 17.0 (TID 535) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:52 INFO TaskSetManager: Starting task 2.0 in stage 17.0 (TID 536) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:52 INFO Executor: Running task 0.0 in stage 17.0 (TID 534) +26/04/01 08:49:52 INFO Executor: Running task 2.0 in stage 17.0 (TID 536) +26/04/01 08:49:52 INFO Executor: Running task 1.0 in stage 17.0 (TID 535) +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:53 INFO Executor: Finished task 59.0 in stage 13.0 (TID 329). 6606 bytes result sent to driver +26/04/01 08:49:53 INFO TaskSetManager: Starting task 3.0 in stage 17.0 (TID 537) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:53 INFO TaskSetManager: Finished task 59.0 in stage 13.0 (TID 329) in 1583 ms on 10.0.0.133 (executor driver) (60/64) +26/04/01 08:49:53 INFO Executor: Running task 3.0 in stage 17.0 (TID 537) +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:53 INFO Executor: Finished task 60.0 in stage 13.0 (TID 330). 6606 bytes result sent to driver +26/04/01 08:49:53 INFO TaskSetManager: Starting task 4.0 in stage 17.0 (TID 538) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:53 INFO TaskSetManager: Finished task 60.0 in stage 13.0 (TID 330) in 1589 ms on 10.0.0.133 (executor driver) (61/64) +26/04/01 08:49:53 INFO Executor: Running task 4.0 in stage 17.0 (TID 538) +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:53 INFO Executor: Finished task 61.0 in stage 13.0 (TID 331). 6606 bytes result sent to driver +26/04/01 08:49:53 INFO TaskSetManager: Starting task 5.0 in stage 17.0 (TID 539) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:53 INFO Executor: Running task 5.0 in stage 17.0 (TID 539) +26/04/01 08:49:53 INFO TaskSetManager: Finished task 61.0 in stage 13.0 (TID 331) in 1581 ms on 10.0.0.133 (executor driver) (62/64) +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:53 INFO Executor: Finished task 62.0 in stage 13.0 (TID 332). 6606 bytes result sent to driver +26/04/01 08:49:53 INFO TaskSetManager: Starting task 6.0 in stage 17.0 (TID 540) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:53 INFO Executor: Running task 6.0 in stage 17.0 (TID 540) +26/04/01 08:49:53 INFO TaskSetManager: Finished task 62.0 in stage 13.0 (TID 332) in 1578 ms on 10.0.0.133 (executor driver) (63/64) +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:54 INFO Executor: Finished task 63.0 in stage 13.0 (TID 333). 6606 bytes result sent to driver +26/04/01 08:49:54 INFO TaskSetManager: Starting task 7.0 in stage 17.0 (TID 541) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:54 INFO TaskSetManager: Finished task 63.0 in stage 13.0 (TID 333) in 1582 ms on 10.0.0.133 (executor driver) (64/64) +26/04/01 08:49:54 INFO Executor: Running task 7.0 in stage 17.0 (TID 541) +26/04/01 08:49:54 INFO TaskSchedulerImpl: Removed TaskSet 13.0, whose tasks have all completed, from pool +26/04/01 08:49:54 INFO DAGScheduler: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 233.877 s +26/04/01 08:49:54 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:49:54 INFO DAGScheduler: running: Set(ShuffleMapStage 17) +26/04/01 08:49:54 INFO DAGScheduler: waiting: Set() +26/04/01 08:49:54 INFO DAGScheduler: failed: Set() +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:54 INFO Executor: Finished task 0.0 in stage 17.0 (TID 534). 6401 bytes result sent to driver +26/04/01 08:49:54 INFO TaskSetManager: Starting task 8.0 in stage 17.0 (TID 542) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:54 INFO Executor: Running task 8.0 in stage 17.0 (TID 542) +26/04/01 08:49:54 INFO TaskSetManager: Finished task 0.0 in stage 17.0 (TID 534) in 2021 ms on 10.0.0.133 (executor driver) (1/200) +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:54 INFO Executor: Finished task 1.0 in stage 17.0 (TID 535). 6401 bytes result sent to driver +26/04/01 08:49:54 INFO TaskSetManager: Starting task 9.0 in stage 17.0 (TID 543) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:54 INFO TaskSetManager: Finished task 1.0 in stage 17.0 (TID 535) in 2033 ms on 10.0.0.133 (executor driver) (2/200) +26/04/01 08:49:54 INFO Executor: Running task 9.0 in stage 17.0 (TID 543) +26/04/01 08:49:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:54 INFO Executor: Finished task 2.0 in stage 17.0 (TID 536). 6401 bytes result sent to driver +26/04/01 08:49:54 INFO TaskSetManager: Starting task 10.0 in stage 17.0 (TID 544) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:54 INFO Executor: Running task 10.0 in stage 17.0 (TID 544) +26/04/01 08:49:54 INFO TaskSetManager: Finished task 2.0 in stage 17.0 (TID 536) in 2035 ms on 10.0.0.133 (executor driver) (3/200) +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:55 INFO Executor: Finished task 3.0 in stage 17.0 (TID 537). 6401 bytes result sent to driver +26/04/01 08:49:55 INFO TaskSetManager: Starting task 11.0 in stage 17.0 (TID 545) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:55 INFO TaskSetManager: Finished task 3.0 in stage 17.0 (TID 537) in 1950 ms on 10.0.0.133 (executor driver) (4/200) +26/04/01 08:49:55 INFO Executor: Running task 11.0 in stage 17.0 (TID 545) +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:55 INFO Executor: Finished task 4.0 in stage 17.0 (TID 538). 6401 bytes result sent to driver +26/04/01 08:49:55 INFO TaskSetManager: Starting task 12.0 in stage 17.0 (TID 546) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:55 INFO Executor: Running task 12.0 in stage 17.0 (TID 546) +26/04/01 08:49:55 INFO TaskSetManager: Finished task 4.0 in stage 17.0 (TID 538) in 1864 ms on 10.0.0.133 (executor driver) (5/200) +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:55 INFO Executor: Finished task 5.0 in stage 17.0 (TID 539). 6401 bytes result sent to driver +26/04/01 08:49:55 INFO TaskSetManager: Starting task 13.0 in stage 17.0 (TID 547) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:55 INFO Executor: Running task 13.0 in stage 17.0 (TID 547) +26/04/01 08:49:55 INFO TaskSetManager: Finished task 5.0 in stage 17.0 (TID 539) in 1863 ms on 10.0.0.133 (executor driver) (6/200) +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:55 INFO Executor: Finished task 6.0 in stage 17.0 (TID 540). 6401 bytes result sent to driver +26/04/01 08:49:55 INFO TaskSetManager: Starting task 14.0 in stage 17.0 (TID 548) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:55 INFO TaskSetManager: Finished task 6.0 in stage 17.0 (TID 540) in 1854 ms on 10.0.0.133 (executor driver) (7/200) +26/04/01 08:49:55 INFO Executor: Running task 14.0 in stage 17.0 (TID 548) +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:55 INFO Executor: Finished task 7.0 in stage 17.0 (TID 541). 6401 bytes result sent to driver +26/04/01 08:49:55 INFO TaskSetManager: Starting task 15.0 in stage 17.0 (TID 549) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:55 INFO Executor: Running task 15.0 in stage 17.0 (TID 549) +26/04/01 08:49:55 INFO TaskSetManager: Finished task 7.0 in stage 17.0 (TID 541) in 1863 ms on 10.0.0.133 (executor driver) (8/200) +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:56 INFO Executor: Finished task 8.0 in stage 17.0 (TID 542). 6401 bytes result sent to driver +26/04/01 08:49:56 INFO TaskSetManager: Starting task 16.0 in stage 17.0 (TID 550) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:56 INFO Executor: Running task 16.0 in stage 17.0 (TID 550) +26/04/01 08:49:56 INFO TaskSetManager: Finished task 8.0 in stage 17.0 (TID 542) in 1831 ms on 10.0.0.133 (executor driver) (9/200) +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:56 INFO Executor: Finished task 10.0 in stage 17.0 (TID 544). 6401 bytes result sent to driver +26/04/01 08:49:56 INFO TaskSetManager: Starting task 17.0 in stage 17.0 (TID 551) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:56 INFO TaskSetManager: Finished task 10.0 in stage 17.0 (TID 544) in 1839 ms on 10.0.0.133 (executor driver) (10/200) +26/04/01 08:49:56 INFO Executor: Running task 17.0 in stage 17.0 (TID 551) +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:56 INFO Executor: Finished task 9.0 in stage 17.0 (TID 543). 6401 bytes result sent to driver +26/04/01 08:49:56 INFO TaskSetManager: Starting task 18.0 in stage 17.0 (TID 552) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:56 INFO Executor: Running task 18.0 in stage 17.0 (TID 552) +26/04/01 08:49:56 INFO TaskSetManager: Finished task 9.0 in stage 17.0 (TID 543) in 1854 ms on 10.0.0.133 (executor driver) (11/200) +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:56 INFO Executor: Finished task 11.0 in stage 17.0 (TID 545). 6401 bytes result sent to driver +26/04/01 08:49:56 INFO TaskSetManager: Starting task 19.0 in stage 17.0 (TID 553) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:56 INFO TaskSetManager: Finished task 11.0 in stage 17.0 (TID 545) in 1840 ms on 10.0.0.133 (executor driver) (12/200) +26/04/01 08:49:56 INFO Executor: Running task 19.0 in stage 17.0 (TID 553) +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:57 INFO Executor: Finished task 12.0 in stage 17.0 (TID 546). 6401 bytes result sent to driver +26/04/01 08:49:57 INFO TaskSetManager: Starting task 20.0 in stage 17.0 (TID 554) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:57 INFO TaskSetManager: Finished task 12.0 in stage 17.0 (TID 546) in 1841 ms on 10.0.0.133 (executor driver) (13/200) +26/04/01 08:49:57 INFO Executor: Running task 20.0 in stage 17.0 (TID 554) +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:57 INFO Executor: Finished task 13.0 in stage 17.0 (TID 547). 6401 bytes result sent to driver +26/04/01 08:49:57 INFO TaskSetManager: Starting task 21.0 in stage 17.0 (TID 555) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:57 INFO Executor: Running task 21.0 in stage 17.0 (TID 555) +26/04/01 08:49:57 INFO TaskSetManager: Finished task 13.0 in stage 17.0 (TID 547) in 1911 ms on 10.0.0.133 (executor driver) (14/200) +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:57 INFO Executor: Finished task 14.0 in stage 17.0 (TID 548). 6401 bytes result sent to driver +26/04/01 08:49:57 INFO TaskSetManager: Starting task 22.0 in stage 17.0 (TID 556) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:57 INFO Executor: Running task 22.0 in stage 17.0 (TID 556) +26/04/01 08:49:57 INFO TaskSetManager: Finished task 14.0 in stage 17.0 (TID 548) in 1926 ms on 10.0.0.133 (executor driver) (15/200) +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (76.4 MiB) non-empty blocks including 208 (76.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (76.4 MiB) non-empty blocks including 208 (76.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:57 INFO Executor: Finished task 15.0 in stage 17.0 (TID 549). 6401 bytes result sent to driver +26/04/01 08:49:57 INFO TaskSetManager: Starting task 23.0 in stage 17.0 (TID 557) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:57 INFO TaskSetManager: Finished task 15.0 in stage 17.0 (TID 549) in 1851 ms on 10.0.0.133 (executor driver) (16/200) +26/04/01 08:49:57 INFO Executor: Running task 23.0 in stage 17.0 (TID 557) +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:58 INFO Executor: Finished task 16.0 in stage 17.0 (TID 550). 6401 bytes result sent to driver +26/04/01 08:49:58 INFO TaskSetManager: Starting task 24.0 in stage 17.0 (TID 558) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:58 INFO TaskSetManager: Finished task 16.0 in stage 17.0 (TID 550) in 1843 ms on 10.0.0.133 (executor driver) (17/200) +26/04/01 08:49:58 INFO Executor: Running task 24.0 in stage 17.0 (TID 558) +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:58 INFO Executor: Finished task 17.0 in stage 17.0 (TID 551). 6401 bytes result sent to driver +26/04/01 08:49:58 INFO TaskSetManager: Starting task 25.0 in stage 17.0 (TID 559) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:58 INFO TaskSetManager: Finished task 17.0 in stage 17.0 (TID 551) in 1847 ms on 10.0.0.133 (executor driver) (18/200) +26/04/01 08:49:58 INFO Executor: Running task 25.0 in stage 17.0 (TID 559) +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:58 INFO Executor: Finished task 18.0 in stage 17.0 (TID 552). 6401 bytes result sent to driver +26/04/01 08:49:58 INFO TaskSetManager: Starting task 26.0 in stage 17.0 (TID 560) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:58 INFO TaskSetManager: Finished task 18.0 in stage 17.0 (TID 552) in 1853 ms on 10.0.0.133 (executor driver) (19/200) +26/04/01 08:49:58 INFO Executor: Running task 26.0 in stage 17.0 (TID 560) +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:58 INFO Executor: Finished task 19.0 in stage 17.0 (TID 553). 6401 bytes result sent to driver +26/04/01 08:49:58 INFO TaskSetManager: Starting task 27.0 in stage 17.0 (TID 561) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:58 INFO Executor: Running task 27.0 in stage 17.0 (TID 561) +26/04/01 08:49:58 INFO TaskSetManager: Finished task 19.0 in stage 17.0 (TID 553) in 1860 ms on 10.0.0.133 (executor driver) (20/200) +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:59 INFO Executor: Finished task 20.0 in stage 17.0 (TID 554). 6401 bytes result sent to driver +26/04/01 08:49:59 INFO TaskSetManager: Starting task 28.0 in stage 17.0 (TID 562) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:59 INFO Executor: Running task 28.0 in stage 17.0 (TID 562) +26/04/01 08:49:59 INFO TaskSetManager: Finished task 20.0 in stage 17.0 (TID 554) in 1847 ms on 10.0.0.133 (executor driver) (21/200) +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:59 INFO Executor: Finished task 21.0 in stage 17.0 (TID 555). 6401 bytes result sent to driver +26/04/01 08:49:59 INFO TaskSetManager: Starting task 29.0 in stage 17.0 (TID 563) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:59 INFO TaskSetManager: Finished task 21.0 in stage 17.0 (TID 555) in 1860 ms on 10.0.0.133 (executor driver) (22/200) +26/04/01 08:49:59 INFO Executor: Running task 29.0 in stage 17.0 (TID 563) +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:59 INFO Executor: Finished task 22.0 in stage 17.0 (TID 556). 6401 bytes result sent to driver +26/04/01 08:49:59 INFO TaskSetManager: Starting task 30.0 in stage 17.0 (TID 564) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:59 INFO Executor: Running task 30.0 in stage 17.0 (TID 564) +26/04/01 08:49:59 INFO TaskSetManager: Finished task 22.0 in stage 17.0 (TID 556) in 1856 ms on 10.0.0.133 (executor driver) (23/200) +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:59 INFO Executor: Finished task 23.0 in stage 17.0 (TID 557). 6401 bytes result sent to driver +26/04/01 08:49:59 INFO TaskSetManager: Starting task 31.0 in stage 17.0 (TID 565) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9624 bytes) +26/04/01 08:49:59 INFO Executor: Running task 31.0 in stage 17.0 (TID 565) +26/04/01 08:49:59 INFO TaskSetManager: Finished task 23.0 in stage 17.0 (TID 557) in 1855 ms on 10.0.0.133 (executor driver) (24/200) +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:00 INFO Executor: Finished task 24.0 in stage 17.0 (TID 558). 6401 bytes result sent to driver +26/04/01 08:50:00 INFO TaskSetManager: Starting task 32.0 in stage 17.0 (TID 566) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:00 INFO TaskSetManager: Finished task 24.0 in stage 17.0 (TID 558) in 1828 ms on 10.0.0.133 (executor driver) (25/200) +26/04/01 08:50:00 INFO Executor: Running task 32.0 in stage 17.0 (TID 566) +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:00 INFO Executor: Finished task 26.0 in stage 17.0 (TID 560). 6401 bytes result sent to driver +26/04/01 08:50:00 INFO TaskSetManager: Starting task 33.0 in stage 17.0 (TID 567) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:00 INFO Executor: Running task 33.0 in stage 17.0 (TID 567) +26/04/01 08:50:00 INFO TaskSetManager: Finished task 26.0 in stage 17.0 (TID 560) in 1802 ms on 10.0.0.133 (executor driver) (26/200) +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:00 INFO Executor: Finished task 25.0 in stage 17.0 (TID 559). 6401 bytes result sent to driver +26/04/01 08:50:00 INFO TaskSetManager: Starting task 34.0 in stage 17.0 (TID 568) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:00 INFO Executor: Running task 34.0 in stage 17.0 (TID 568) +26/04/01 08:50:00 INFO TaskSetManager: Finished task 25.0 in stage 17.0 (TID 559) in 1839 ms on 10.0.0.133 (executor driver) (27/200) +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:00 INFO Executor: Finished task 27.0 in stage 17.0 (TID 561). 6401 bytes result sent to driver +26/04/01 08:50:00 INFO TaskSetManager: Starting task 35.0 in stage 17.0 (TID 569) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:00 INFO Executor: Running task 35.0 in stage 17.0 (TID 569) +26/04/01 08:50:00 INFO TaskSetManager: Finished task 27.0 in stage 17.0 (TID 561) in 1845 ms on 10.0.0.133 (executor driver) (28/200) +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:01 INFO Executor: Finished task 28.0 in stage 17.0 (TID 562). 7132 bytes result sent to driver +26/04/01 08:50:01 INFO TaskSetManager: Starting task 36.0 in stage 17.0 (TID 570) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:01 INFO TaskSetManager: Finished task 28.0 in stage 17.0 (TID 562) in 1840 ms on 10.0.0.133 (executor driver) (29/200) +26/04/01 08:50:01 INFO Executor: Running task 36.0 in stage 17.0 (TID 570) +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:01 INFO Executor: Finished task 29.0 in stage 17.0 (TID 563). 7132 bytes result sent to driver +26/04/01 08:50:01 INFO TaskSetManager: Starting task 37.0 in stage 17.0 (TID 571) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:01 INFO Executor: Running task 37.0 in stage 17.0 (TID 571) +26/04/01 08:50:01 INFO TaskSetManager: Finished task 29.0 in stage 17.0 (TID 563) in 1835 ms on 10.0.0.133 (executor driver) (30/200) +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:01 INFO Executor: Finished task 30.0 in stage 17.0 (TID 564). 7132 bytes result sent to driver +26/04/01 08:50:01 INFO TaskSetManager: Starting task 38.0 in stage 17.0 (TID 572) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:01 INFO TaskSetManager: Finished task 30.0 in stage 17.0 (TID 564) in 1834 ms on 10.0.0.133 (executor driver) (31/200) +26/04/01 08:50:01 INFO Executor: Running task 38.0 in stage 17.0 (TID 572) +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:01 INFO Executor: Finished task 31.0 in stage 17.0 (TID 565). 7132 bytes result sent to driver +26/04/01 08:50:01 INFO TaskSetManager: Starting task 39.0 in stage 17.0 (TID 573) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:01 INFO Executor: Running task 39.0 in stage 17.0 (TID 573) +26/04/01 08:50:01 INFO TaskSetManager: Finished task 31.0 in stage 17.0 (TID 565) in 1851 ms on 10.0.0.133 (executor driver) (32/200) +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:02 INFO Executor: Finished task 32.0 in stage 17.0 (TID 566). 7132 bytes result sent to driver +26/04/01 08:50:02 INFO TaskSetManager: Starting task 40.0 in stage 17.0 (TID 574) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:02 INFO TaskSetManager: Finished task 32.0 in stage 17.0 (TID 566) in 1857 ms on 10.0.0.133 (executor driver) (33/200) +26/04/01 08:50:02 INFO Executor: Running task 40.0 in stage 17.0 (TID 574) +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:02 INFO Executor: Finished task 33.0 in stage 17.0 (TID 567). 7132 bytes result sent to driver +26/04/01 08:50:02 INFO TaskSetManager: Starting task 41.0 in stage 17.0 (TID 575) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:02 INFO Executor: Running task 41.0 in stage 17.0 (TID 575) +26/04/01 08:50:02 INFO TaskSetManager: Finished task 33.0 in stage 17.0 (TID 567) in 1850 ms on 10.0.0.133 (executor driver) (34/200) +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:02 INFO Executor: Finished task 34.0 in stage 17.0 (TID 568). 7132 bytes result sent to driver +26/04/01 08:50:02 INFO TaskSetManager: Starting task 42.0 in stage 17.0 (TID 576) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:02 INFO TaskSetManager: Finished task 34.0 in stage 17.0 (TID 568) in 1853 ms on 10.0.0.133 (executor driver) (35/200) +26/04/01 08:50:02 INFO Executor: Running task 42.0 in stage 17.0 (TID 576) +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:02 INFO Executor: Finished task 35.0 in stage 17.0 (TID 569). 7132 bytes result sent to driver +26/04/01 08:50:02 INFO TaskSetManager: Starting task 43.0 in stage 17.0 (TID 577) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:02 INFO TaskSetManager: Finished task 35.0 in stage 17.0 (TID 569) in 1909 ms on 10.0.0.133 (executor driver) (36/200) +26/04/01 08:50:02 INFO Executor: Running task 43.0 in stage 17.0 (TID 577) +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:02 INFO Executor: Finished task 36.0 in stage 17.0 (TID 570). 6401 bytes result sent to driver +26/04/01 08:50:02 INFO TaskSetManager: Starting task 44.0 in stage 17.0 (TID 578) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:02 INFO TaskSetManager: Finished task 36.0 in stage 17.0 (TID 570) in 1876 ms on 10.0.0.133 (executor driver) (37/200) +26/04/01 08:50:02 INFO Executor: Running task 44.0 in stage 17.0 (TID 578) +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:03 INFO Executor: Finished task 37.0 in stage 17.0 (TID 571). 6401 bytes result sent to driver +26/04/01 08:50:03 INFO TaskSetManager: Starting task 45.0 in stage 17.0 (TID 579) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:03 INFO TaskSetManager: Finished task 37.0 in stage 17.0 (TID 571) in 1861 ms on 10.0.0.133 (executor driver) (38/200) +26/04/01 08:50:03 INFO Executor: Running task 45.0 in stage 17.0 (TID 579) +26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:03 INFO Executor: Finished task 38.0 in stage 17.0 (TID 572). 6401 bytes result sent to driver +26/04/01 08:50:03 INFO TaskSetManager: Starting task 46.0 in stage 17.0 (TID 580) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:03 INFO TaskSetManager: Finished task 38.0 in stage 17.0 (TID 572) in 1836 ms on 10.0.0.133 (executor driver) (39/200) +26/04/01 08:50:03 INFO Executor: Running task 46.0 in stage 17.0 (TID 580) +26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:03 INFO Executor: Finished task 39.0 in stage 17.0 (TID 573). 6487 bytes result sent to driver +26/04/01 08:50:03 INFO TaskSetManager: Starting task 47.0 in stage 17.0 (TID 581) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:03 INFO Executor: Running task 47.0 in stage 17.0 (TID 581) +26/04/01 08:50:03 INFO TaskSetManager: Finished task 39.0 in stage 17.0 (TID 573) in 1845 ms on 10.0.0.133 (executor driver) (40/200) +26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:04 INFO Executor: Finished task 40.0 in stage 17.0 (TID 574). 6444 bytes result sent to driver +26/04/01 08:50:04 INFO Executor: Finished task 41.0 in stage 17.0 (TID 575). 6444 bytes result sent to driver +26/04/01 08:50:04 INFO TaskSetManager: Starting task 48.0 in stage 17.0 (TID 582) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:04 INFO TaskSetManager: Finished task 40.0 in stage 17.0 (TID 574) in 1855 ms on 10.0.0.133 (executor driver) (41/200) +26/04/01 08:50:04 INFO Executor: Running task 48.0 in stage 17.0 (TID 582) +26/04/01 08:50:04 INFO TaskSetManager: Starting task 49.0 in stage 17.0 (TID 583) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:04 INFO Executor: Running task 49.0 in stage 17.0 (TID 583) +26/04/01 08:50:04 INFO TaskSetManager: Finished task 41.0 in stage 17.0 (TID 575) in 1844 ms on 10.0.0.133 (executor driver) (42/200) +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:04 INFO Executor: Finished task 42.0 in stage 17.0 (TID 576). 6444 bytes result sent to driver +26/04/01 08:50:04 INFO TaskSetManager: Starting task 50.0 in stage 17.0 (TID 584) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:04 INFO TaskSetManager: Finished task 42.0 in stage 17.0 (TID 576) in 1840 ms on 10.0.0.133 (executor driver) (43/200) +26/04/01 08:50:04 INFO Executor: Running task 50.0 in stage 17.0 (TID 584) +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:04 INFO Executor: Finished task 43.0 in stage 17.0 (TID 577). 6444 bytes result sent to driver +26/04/01 08:50:04 INFO TaskSetManager: Starting task 51.0 in stage 17.0 (TID 585) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:04 INFO TaskSetManager: Finished task 43.0 in stage 17.0 (TID 577) in 1845 ms on 10.0.0.133 (executor driver) (44/200) +26/04/01 08:50:04 INFO Executor: Running task 51.0 in stage 17.0 (TID 585) +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:04 INFO Executor: Finished task 44.0 in stage 17.0 (TID 578). 6444 bytes result sent to driver +26/04/01 08:50:04 INFO TaskSetManager: Starting task 52.0 in stage 17.0 (TID 586) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:04 INFO TaskSetManager: Finished task 44.0 in stage 17.0 (TID 578) in 1869 ms on 10.0.0.133 (executor driver) (45/200) +26/04/01 08:50:04 INFO Executor: Running task 52.0 in stage 17.0 (TID 586) +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:04 INFO Executor: Finished task 45.0 in stage 17.0 (TID 579). 6444 bytes result sent to driver +26/04/01 08:50:04 INFO TaskSetManager: Starting task 53.0 in stage 17.0 (TID 587) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:04 INFO TaskSetManager: Finished task 45.0 in stage 17.0 (TID 579) in 1857 ms on 10.0.0.133 (executor driver) (46/200) +26/04/01 08:50:04 INFO Executor: Running task 53.0 in stage 17.0 (TID 587) +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:04 INFO Executor: Finished task 46.0 in stage 17.0 (TID 580). 6444 bytes result sent to driver +26/04/01 08:50:04 INFO TaskSetManager: Starting task 54.0 in stage 17.0 (TID 588) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:04 INFO Executor: Running task 54.0 in stage 17.0 (TID 588) +26/04/01 08:50:04 INFO TaskSetManager: Finished task 46.0 in stage 17.0 (TID 580) in 1837 ms on 10.0.0.133 (executor driver) (47/200) +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:05 INFO Executor: Finished task 47.0 in stage 17.0 (TID 581). 6401 bytes result sent to driver +26/04/01 08:50:05 INFO TaskSetManager: Starting task 55.0 in stage 17.0 (TID 589) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:05 INFO TaskSetManager: Finished task 47.0 in stage 17.0 (TID 581) in 1852 ms on 10.0.0.133 (executor driver) (48/200) +26/04/01 08:50:05 INFO Executor: Running task 55.0 in stage 17.0 (TID 589) +26/04/01 08:50:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:06 INFO Executor: Finished task 48.0 in stage 17.0 (TID 582). 6401 bytes result sent to driver +26/04/01 08:50:06 INFO TaskSetManager: Starting task 56.0 in stage 17.0 (TID 590) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:06 INFO Executor: Running task 56.0 in stage 17.0 (TID 590) +26/04/01 08:50:06 INFO TaskSetManager: Finished task 48.0 in stage 17.0 (TID 582) in 1836 ms on 10.0.0.133 (executor driver) (49/200) +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:06 INFO Executor: Finished task 49.0 in stage 17.0 (TID 583). 6401 bytes result sent to driver +26/04/01 08:50:06 INFO TaskSetManager: Starting task 57.0 in stage 17.0 (TID 591) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:06 INFO TaskSetManager: Finished task 49.0 in stage 17.0 (TID 583) in 1840 ms on 10.0.0.133 (executor driver) (50/200) +26/04/01 08:50:06 INFO Executor: Running task 57.0 in stage 17.0 (TID 591) +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:06 INFO Executor: Finished task 50.0 in stage 17.0 (TID 584). 6401 bytes result sent to driver +26/04/01 08:50:06 INFO TaskSetManager: Starting task 58.0 in stage 17.0 (TID 592) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:06 INFO TaskSetManager: Finished task 50.0 in stage 17.0 (TID 584) in 1833 ms on 10.0.0.133 (executor driver) (51/200) +26/04/01 08:50:06 INFO Executor: Running task 58.0 in stage 17.0 (TID 592) +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:06 INFO Executor: Finished task 51.0 in stage 17.0 (TID 585). 6401 bytes result sent to driver +26/04/01 08:50:06 INFO TaskSetManager: Starting task 59.0 in stage 17.0 (TID 593) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:06 INFO Executor: Running task 59.0 in stage 17.0 (TID 593) +26/04/01 08:50:06 INFO TaskSetManager: Finished task 51.0 in stage 17.0 (TID 585) in 1849 ms on 10.0.0.133 (executor driver) (52/200) +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:06 INFO Executor: Finished task 52.0 in stage 17.0 (TID 586). 6401 bytes result sent to driver +26/04/01 08:50:06 INFO TaskSetManager: Starting task 60.0 in stage 17.0 (TID 594) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:06 INFO TaskSetManager: Finished task 52.0 in stage 17.0 (TID 586) in 1844 ms on 10.0.0.133 (executor driver) (53/200) +26/04/01 08:50:06 INFO Executor: Running task 60.0 in stage 17.0 (TID 594) +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:06 INFO Executor: Finished task 53.0 in stage 17.0 (TID 587). 6401 bytes result sent to driver +26/04/01 08:50:06 INFO TaskSetManager: Starting task 61.0 in stage 17.0 (TID 595) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:06 INFO TaskSetManager: Finished task 53.0 in stage 17.0 (TID 587) in 1862 ms on 10.0.0.133 (executor driver) (54/200) +26/04/01 08:50:06 INFO Executor: Running task 61.0 in stage 17.0 (TID 595) +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:06 INFO Executor: Finished task 54.0 in stage 17.0 (TID 588). 6401 bytes result sent to driver +26/04/01 08:50:06 INFO TaskSetManager: Starting task 62.0 in stage 17.0 (TID 596) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:06 INFO TaskSetManager: Finished task 54.0 in stage 17.0 (TID 588) in 1860 ms on 10.0.0.133 (executor driver) (55/200) +26/04/01 08:50:06 INFO Executor: Running task 62.0 in stage 17.0 (TID 596) +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:07 INFO Executor: Finished task 55.0 in stage 17.0 (TID 589). 6401 bytes result sent to driver +26/04/01 08:50:07 INFO TaskSetManager: Starting task 63.0 in stage 17.0 (TID 597) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:07 INFO TaskSetManager: Finished task 55.0 in stage 17.0 (TID 589) in 1858 ms on 10.0.0.133 (executor driver) (56/200) +26/04/01 08:50:07 INFO Executor: Running task 63.0 in stage 17.0 (TID 597) +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:07 INFO Executor: Finished task 56.0 in stage 17.0 (TID 590). 6401 bytes result sent to driver +26/04/01 08:50:07 INFO TaskSetManager: Starting task 64.0 in stage 17.0 (TID 598) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:07 INFO Executor: Running task 64.0 in stage 17.0 (TID 598) +26/04/01 08:50:07 INFO TaskSetManager: Finished task 56.0 in stage 17.0 (TID 590) in 1884 ms on 10.0.0.133 (executor driver) (57/200) +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:07 INFO Executor: Finished task 57.0 in stage 17.0 (TID 591). 6401 bytes result sent to driver +26/04/01 08:50:07 INFO TaskSetManager: Starting task 65.0 in stage 17.0 (TID 599) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:07 INFO TaskSetManager: Finished task 57.0 in stage 17.0 (TID 591) in 1919 ms on 10.0.0.133 (executor driver) (58/200) +26/04/01 08:50:07 INFO Executor: Running task 65.0 in stage 17.0 (TID 599) +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:07 INFO Executor: Finished task 58.0 in stage 17.0 (TID 592). 6401 bytes result sent to driver +26/04/01 08:50:07 INFO TaskSetManager: Starting task 66.0 in stage 17.0 (TID 600) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:07 INFO Executor: Running task 66.0 in stage 17.0 (TID 600) +26/04/01 08:50:07 INFO TaskSetManager: Finished task 58.0 in stage 17.0 (TID 592) in 1920 ms on 10.0.0.133 (executor driver) (59/200) +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:08 INFO Executor: Finished task 59.0 in stage 17.0 (TID 593). 6401 bytes result sent to driver +26/04/01 08:50:08 INFO TaskSetManager: Starting task 67.0 in stage 17.0 (TID 601) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:08 INFO TaskSetManager: Finished task 59.0 in stage 17.0 (TID 593) in 1897 ms on 10.0.0.133 (executor driver) (60/200) +26/04/01 08:50:08 INFO Executor: Running task 67.0 in stage 17.0 (TID 601) +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:08 INFO Executor: Finished task 60.0 in stage 17.0 (TID 594). 6401 bytes result sent to driver +26/04/01 08:50:08 INFO TaskSetManager: Starting task 68.0 in stage 17.0 (TID 602) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:08 INFO Executor: Running task 68.0 in stage 17.0 (TID 602) +26/04/01 08:50:08 INFO TaskSetManager: Finished task 60.0 in stage 17.0 (TID 594) in 1865 ms on 10.0.0.133 (executor driver) (61/200) +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:08 INFO Executor: Finished task 61.0 in stage 17.0 (TID 595). 6401 bytes result sent to driver +26/04/01 08:50:08 INFO Executor: Finished task 62.0 in stage 17.0 (TID 596). 6401 bytes result sent to driver +26/04/01 08:50:08 INFO TaskSetManager: Starting task 69.0 in stage 17.0 (TID 603) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:08 INFO Executor: Running task 69.0 in stage 17.0 (TID 603) +26/04/01 08:50:08 INFO TaskSetManager: Starting task 70.0 in stage 17.0 (TID 604) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:08 INFO Executor: Running task 70.0 in stage 17.0 (TID 604) +26/04/01 08:50:08 INFO TaskSetManager: Finished task 61.0 in stage 17.0 (TID 595) in 1862 ms on 10.0.0.133 (executor driver) (62/200) +26/04/01 08:50:08 INFO TaskSetManager: Finished task 62.0 in stage 17.0 (TID 596) in 1860 ms on 10.0.0.133 (executor driver) (63/200) +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:08 INFO Executor: Finished task 63.0 in stage 17.0 (TID 597). 6401 bytes result sent to driver +26/04/01 08:50:08 INFO TaskSetManager: Starting task 71.0 in stage 17.0 (TID 605) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:08 INFO TaskSetManager: Finished task 63.0 in stage 17.0 (TID 597) in 1859 ms on 10.0.0.133 (executor driver) (64/200) +26/04/01 08:50:08 INFO Executor: Running task 71.0 in stage 17.0 (TID 605) +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:09 INFO Executor: Finished task 64.0 in stage 17.0 (TID 598). 6401 bytes result sent to driver +26/04/01 08:50:09 INFO TaskSetManager: Starting task 72.0 in stage 17.0 (TID 606) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:09 INFO Executor: Running task 72.0 in stage 17.0 (TID 606) +26/04/01 08:50:09 INFO TaskSetManager: Finished task 64.0 in stage 17.0 (TID 598) in 1859 ms on 10.0.0.133 (executor driver) (65/200) +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:09 INFO Executor: Finished task 65.0 in stage 17.0 (TID 599). 6401 bytes result sent to driver +26/04/01 08:50:09 INFO TaskSetManager: Starting task 73.0 in stage 17.0 (TID 607) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:09 INFO TaskSetManager: Finished task 65.0 in stage 17.0 (TID 599) in 1845 ms on 10.0.0.133 (executor driver) (66/200) +26/04/01 08:50:09 INFO Executor: Running task 73.0 in stage 17.0 (TID 607) +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:09 INFO Executor: Finished task 66.0 in stage 17.0 (TID 600). 6401 bytes result sent to driver +26/04/01 08:50:09 INFO TaskSetManager: Starting task 74.0 in stage 17.0 (TID 608) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:09 INFO TaskSetManager: Finished task 66.0 in stage 17.0 (TID 600) in 1857 ms on 10.0.0.133 (executor driver) (67/200) +26/04/01 08:50:09 INFO Executor: Running task 74.0 in stage 17.0 (TID 608) +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:09 INFO Executor: Finished task 67.0 in stage 17.0 (TID 601). 6401 bytes result sent to driver +26/04/01 08:50:09 INFO TaskSetManager: Starting task 75.0 in stage 17.0 (TID 609) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:09 INFO TaskSetManager: Finished task 67.0 in stage 17.0 (TID 601) in 1858 ms on 10.0.0.133 (executor driver) (68/200) +26/04/01 08:50:09 INFO Executor: Running task 75.0 in stage 17.0 (TID 609) +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:10 INFO Executor: Finished task 68.0 in stage 17.0 (TID 602). 6401 bytes result sent to driver +26/04/01 08:50:10 INFO TaskSetManager: Starting task 76.0 in stage 17.0 (TID 610) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:10 INFO TaskSetManager: Finished task 68.0 in stage 17.0 (TID 602) in 1840 ms on 10.0.0.133 (executor driver) (69/200) +26/04/01 08:50:10 INFO Executor: Running task 76.0 in stage 17.0 (TID 610) +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:10 INFO Executor: Finished task 69.0 in stage 17.0 (TID 603). 6401 bytes result sent to driver +26/04/01 08:50:10 INFO TaskSetManager: Starting task 77.0 in stage 17.0 (TID 611) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:10 INFO TaskSetManager: Finished task 69.0 in stage 17.0 (TID 603) in 1861 ms on 10.0.0.133 (executor driver) (70/200) +26/04/01 08:50:10 INFO Executor: Running task 77.0 in stage 17.0 (TID 611) +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:10 INFO Executor: Finished task 70.0 in stage 17.0 (TID 604). 6401 bytes result sent to driver +26/04/01 08:50:10 INFO TaskSetManager: Starting task 78.0 in stage 17.0 (TID 612) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:10 INFO TaskSetManager: Finished task 70.0 in stage 17.0 (TID 604) in 1877 ms on 10.0.0.133 (executor driver) (71/200) +26/04/01 08:50:10 INFO Executor: Running task 78.0 in stage 17.0 (TID 612) +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:10 INFO Executor: Finished task 71.0 in stage 17.0 (TID 605). 6401 bytes result sent to driver +26/04/01 08:50:10 INFO TaskSetManager: Starting task 79.0 in stage 17.0 (TID 613) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:10 INFO TaskSetManager: Finished task 71.0 in stage 17.0 (TID 605) in 1846 ms on 10.0.0.133 (executor driver) (72/200) +26/04/01 08:50:10 INFO Executor: Running task 79.0 in stage 17.0 (TID 613) +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:11 INFO Executor: Finished task 72.0 in stage 17.0 (TID 606). 7132 bytes result sent to driver +26/04/01 08:50:11 INFO TaskSetManager: Starting task 80.0 in stage 17.0 (TID 614) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:11 INFO TaskSetManager: Finished task 72.0 in stage 17.0 (TID 606) in 1843 ms on 10.0.0.133 (executor driver) (73/200) +26/04/01 08:50:11 INFO Executor: Running task 80.0 in stage 17.0 (TID 614) +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:11 INFO Executor: Finished task 73.0 in stage 17.0 (TID 607). 7132 bytes result sent to driver +26/04/01 08:50:11 INFO TaskSetManager: Starting task 81.0 in stage 17.0 (TID 615) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:11 INFO Executor: Running task 81.0 in stage 17.0 (TID 615) +26/04/01 08:50:11 INFO TaskSetManager: Finished task 73.0 in stage 17.0 (TID 607) in 1839 ms on 10.0.0.133 (executor driver) (74/200) +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:11 INFO Executor: Finished task 74.0 in stage 17.0 (TID 608). 7132 bytes result sent to driver +26/04/01 08:50:11 INFO TaskSetManager: Starting task 82.0 in stage 17.0 (TID 616) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:11 INFO TaskSetManager: Finished task 74.0 in stage 17.0 (TID 608) in 1863 ms on 10.0.0.133 (executor driver) (75/200) +26/04/01 08:50:11 INFO Executor: Running task 82.0 in stage 17.0 (TID 616) +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:11 INFO Executor: Finished task 75.0 in stage 17.0 (TID 609). 7132 bytes result sent to driver +26/04/01 08:50:11 INFO TaskSetManager: Starting task 83.0 in stage 17.0 (TID 617) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:11 INFO TaskSetManager: Finished task 75.0 in stage 17.0 (TID 609) in 1857 ms on 10.0.0.133 (executor driver) (76/200) +26/04/01 08:50:11 INFO Executor: Running task 83.0 in stage 17.0 (TID 617) +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:12 INFO Executor: Finished task 76.0 in stage 17.0 (TID 610). 7132 bytes result sent to driver +26/04/01 08:50:12 INFO TaskSetManager: Starting task 84.0 in stage 17.0 (TID 618) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:12 INFO Executor: Running task 84.0 in stage 17.0 (TID 618) +26/04/01 08:50:12 INFO TaskSetManager: Finished task 76.0 in stage 17.0 (TID 610) in 1854 ms on 10.0.0.133 (executor driver) (77/200) +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:12 INFO Executor: Finished task 77.0 in stage 17.0 (TID 611). 7132 bytes result sent to driver +26/04/01 08:50:12 INFO TaskSetManager: Starting task 85.0 in stage 17.0 (TID 619) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:12 INFO Executor: Running task 85.0 in stage 17.0 (TID 619) +26/04/01 08:50:12 INFO TaskSetManager: Finished task 77.0 in stage 17.0 (TID 611) in 1868 ms on 10.0.0.133 (executor driver) (78/200) +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:12 INFO Executor: Finished task 78.0 in stage 17.0 (TID 612). 7132 bytes result sent to driver +26/04/01 08:50:12 INFO TaskSetManager: Starting task 86.0 in stage 17.0 (TID 620) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:12 INFO Executor: Running task 86.0 in stage 17.0 (TID 620) +26/04/01 08:50:12 INFO TaskSetManager: Finished task 78.0 in stage 17.0 (TID 612) in 1858 ms on 10.0.0.133 (executor driver) (79/200) +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:12 INFO Executor: Finished task 79.0 in stage 17.0 (TID 613). 7132 bytes result sent to driver +26/04/01 08:50:12 INFO TaskSetManager: Starting task 87.0 in stage 17.0 (TID 621) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:12 INFO Executor: Running task 87.0 in stage 17.0 (TID 621) +26/04/01 08:50:12 INFO TaskSetManager: Finished task 79.0 in stage 17.0 (TID 613) in 1882 ms on 10.0.0.133 (executor driver) (80/200) +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:13 INFO Executor: Finished task 81.0 in stage 17.0 (TID 615). 6401 bytes result sent to driver +26/04/01 08:50:13 INFO TaskSetManager: Starting task 88.0 in stage 17.0 (TID 622) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:13 INFO TaskSetManager: Finished task 81.0 in stage 17.0 (TID 615) in 1838 ms on 10.0.0.133 (executor driver) (81/200) +26/04/01 08:50:13 INFO Executor: Running task 88.0 in stage 17.0 (TID 622) +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:13 INFO Executor: Finished task 80.0 in stage 17.0 (TID 614). 6401 bytes result sent to driver +26/04/01 08:50:13 INFO TaskSetManager: Starting task 89.0 in stage 17.0 (TID 623) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:13 INFO TaskSetManager: Finished task 80.0 in stage 17.0 (TID 614) in 1863 ms on 10.0.0.133 (executor driver) (82/200) +26/04/01 08:50:13 INFO Executor: Running task 89.0 in stage 17.0 (TID 623) +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:13 INFO Executor: Finished task 82.0 in stage 17.0 (TID 616). 6401 bytes result sent to driver +26/04/01 08:50:13 INFO TaskSetManager: Starting task 90.0 in stage 17.0 (TID 624) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:13 INFO Executor: Running task 90.0 in stage 17.0 (TID 624) +26/04/01 08:50:13 INFO TaskSetManager: Finished task 82.0 in stage 17.0 (TID 616) in 1845 ms on 10.0.0.133 (executor driver) (83/200) +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:13 INFO Executor: Finished task 83.0 in stage 17.0 (TID 617). 6401 bytes result sent to driver +26/04/01 08:50:13 INFO TaskSetManager: Starting task 91.0 in stage 17.0 (TID 625) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:13 INFO TaskSetManager: Finished task 83.0 in stage 17.0 (TID 617) in 1850 ms on 10.0.0.133 (executor driver) (84/200) +26/04/01 08:50:13 INFO Executor: Running task 91.0 in stage 17.0 (TID 625) +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:14 INFO Executor: Finished task 84.0 in stage 17.0 (TID 618). 6401 bytes result sent to driver +26/04/01 08:50:14 INFO TaskSetManager: Starting task 92.0 in stage 17.0 (TID 626) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:14 INFO Executor: Running task 92.0 in stage 17.0 (TID 626) +26/04/01 08:50:14 INFO TaskSetManager: Finished task 84.0 in stage 17.0 (TID 618) in 1842 ms on 10.0.0.133 (executor driver) (85/200) +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:14 INFO Executor: Finished task 85.0 in stage 17.0 (TID 619). 6401 bytes result sent to driver +26/04/01 08:50:14 INFO TaskSetManager: Starting task 93.0 in stage 17.0 (TID 627) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:14 INFO Executor: Running task 93.0 in stage 17.0 (TID 627) +26/04/01 08:50:14 INFO TaskSetManager: Finished task 85.0 in stage 17.0 (TID 619) in 1846 ms on 10.0.0.133 (executor driver) (86/200) +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:14 INFO Executor: Finished task 86.0 in stage 17.0 (TID 620). 6401 bytes result sent to driver +26/04/01 08:50:14 INFO TaskSetManager: Starting task 94.0 in stage 17.0 (TID 628) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:14 INFO Executor: Running task 94.0 in stage 17.0 (TID 628) +26/04/01 08:50:14 INFO TaskSetManager: Finished task 86.0 in stage 17.0 (TID 620) in 1843 ms on 10.0.0.133 (executor driver) (87/200) +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:14 INFO Executor: Finished task 87.0 in stage 17.0 (TID 621). 6401 bytes result sent to driver +26/04/01 08:50:14 INFO TaskSetManager: Starting task 95.0 in stage 17.0 (TID 629) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:14 INFO TaskSetManager: Finished task 87.0 in stage 17.0 (TID 621) in 1835 ms on 10.0.0.133 (executor driver) (88/200) +26/04/01 08:50:14 INFO Executor: Running task 95.0 in stage 17.0 (TID 629) +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:15 INFO Executor: Finished task 89.0 in stage 17.0 (TID 623). 6401 bytes result sent to driver +26/04/01 08:50:15 INFO TaskSetManager: Starting task 96.0 in stage 17.0 (TID 630) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:15 INFO Executor: Running task 96.0 in stage 17.0 (TID 630) +26/04/01 08:50:15 INFO TaskSetManager: Finished task 89.0 in stage 17.0 (TID 623) in 1840 ms on 10.0.0.133 (executor driver) (89/200) +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:15 INFO Executor: Finished task 88.0 in stage 17.0 (TID 622). 6401 bytes result sent to driver +26/04/01 08:50:15 INFO TaskSetManager: Starting task 97.0 in stage 17.0 (TID 631) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:15 INFO TaskSetManager: Finished task 88.0 in stage 17.0 (TID 622) in 1851 ms on 10.0.0.133 (executor driver) (90/200) +26/04/01 08:50:15 INFO Executor: Running task 97.0 in stage 17.0 (TID 631) +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:15 INFO Executor: Finished task 90.0 in stage 17.0 (TID 624). 6401 bytes result sent to driver +26/04/01 08:50:15 INFO TaskSetManager: Starting task 98.0 in stage 17.0 (TID 632) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:15 INFO Executor: Running task 98.0 in stage 17.0 (TID 632) +26/04/01 08:50:15 INFO TaskSetManager: Finished task 90.0 in stage 17.0 (TID 624) in 1883 ms on 10.0.0.133 (executor driver) (91/200) +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:15 INFO Executor: Finished task 91.0 in stage 17.0 (TID 625). 6401 bytes result sent to driver +26/04/01 08:50:15 INFO TaskSetManager: Starting task 99.0 in stage 17.0 (TID 633) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:15 INFO TaskSetManager: Finished task 91.0 in stage 17.0 (TID 625) in 1860 ms on 10.0.0.133 (executor driver) (92/200) +26/04/01 08:50:15 INFO Executor: Running task 99.0 in stage 17.0 (TID 633) +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:15 INFO Executor: Finished task 92.0 in stage 17.0 (TID 626). 6401 bytes result sent to driver +26/04/01 08:50:15 INFO TaskSetManager: Starting task 100.0 in stage 17.0 (TID 634) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:15 INFO TaskSetManager: Finished task 92.0 in stage 17.0 (TID 626) in 1860 ms on 10.0.0.133 (executor driver) (93/200) +26/04/01 08:50:15 INFO Executor: Running task 100.0 in stage 17.0 (TID 634) +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:16 INFO Executor: Finished task 93.0 in stage 17.0 (TID 627). 6401 bytes result sent to driver +26/04/01 08:50:16 INFO TaskSetManager: Starting task 101.0 in stage 17.0 (TID 635) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:16 INFO TaskSetManager: Finished task 93.0 in stage 17.0 (TID 627) in 1853 ms on 10.0.0.133 (executor driver) (94/200) +26/04/01 08:50:16 INFO Executor: Running task 101.0 in stage 17.0 (TID 635) +26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:16 INFO Executor: Finished task 94.0 in stage 17.0 (TID 628). 6401 bytes result sent to driver +26/04/01 08:50:16 INFO TaskSetManager: Starting task 102.0 in stage 17.0 (TID 636) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:16 INFO TaskSetManager: Finished task 94.0 in stage 17.0 (TID 628) in 1858 ms on 10.0.0.133 (executor driver) (95/200) +26/04/01 08:50:16 INFO Executor: Running task 102.0 in stage 17.0 (TID 636) +26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:16 INFO Executor: Finished task 95.0 in stage 17.0 (TID 629). 6401 bytes result sent to driver +26/04/01 08:50:16 INFO TaskSetManager: Starting task 103.0 in stage 17.0 (TID 637) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:16 INFO Executor: Running task 103.0 in stage 17.0 (TID 637) +26/04/01 08:50:16 INFO TaskSetManager: Finished task 95.0 in stage 17.0 (TID 629) in 1882 ms on 10.0.0.133 (executor driver) (96/200) +26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:17 INFO Executor: Finished task 97.0 in stage 17.0 (TID 631). 6401 bytes result sent to driver +26/04/01 08:50:17 INFO TaskSetManager: Starting task 104.0 in stage 17.0 (TID 638) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:17 INFO Executor: Running task 104.0 in stage 17.0 (TID 638) +26/04/01 08:50:17 INFO TaskSetManager: Finished task 97.0 in stage 17.0 (TID 631) in 1891 ms on 10.0.0.133 (executor driver) (97/200) +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:17 INFO Executor: Finished task 96.0 in stage 17.0 (TID 630). 6401 bytes result sent to driver +26/04/01 08:50:17 INFO TaskSetManager: Starting task 105.0 in stage 17.0 (TID 639) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:17 INFO TaskSetManager: Finished task 96.0 in stage 17.0 (TID 630) in 1901 ms on 10.0.0.133 (executor driver) (98/200) +26/04/01 08:50:17 INFO Executor: Running task 105.0 in stage 17.0 (TID 639) +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:17 INFO Executor: Finished task 98.0 in stage 17.0 (TID 632). 6401 bytes result sent to driver +26/04/01 08:50:17 INFO TaskSetManager: Starting task 106.0 in stage 17.0 (TID 640) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:17 INFO TaskSetManager: Finished task 98.0 in stage 17.0 (TID 632) in 1873 ms on 10.0.0.133 (executor driver) (99/200) +26/04/01 08:50:17 INFO Executor: Running task 106.0 in stage 17.0 (TID 640) +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:17 INFO Executor: Finished task 99.0 in stage 17.0 (TID 633). 6401 bytes result sent to driver +26/04/01 08:50:17 INFO TaskSetManager: Starting task 107.0 in stage 17.0 (TID 641) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:17 INFO TaskSetManager: Finished task 99.0 in stage 17.0 (TID 633) in 1878 ms on 10.0.0.133 (executor driver) (100/200) +26/04/01 08:50:17 INFO Executor: Running task 107.0 in stage 17.0 (TID 641) +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:17 INFO Executor: Finished task 100.0 in stage 17.0 (TID 634). 6401 bytes result sent to driver +26/04/01 08:50:17 INFO TaskSetManager: Starting task 108.0 in stage 17.0 (TID 642) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:17 INFO TaskSetManager: Finished task 100.0 in stage 17.0 (TID 634) in 1879 ms on 10.0.0.133 (executor driver) (101/200) +26/04/01 08:50:17 INFO Executor: Running task 108.0 in stage 17.0 (TID 642) +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:18 INFO Executor: Finished task 101.0 in stage 17.0 (TID 635). 6401 bytes result sent to driver +26/04/01 08:50:18 INFO TaskSetManager: Starting task 109.0 in stage 17.0 (TID 643) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:18 INFO TaskSetManager: Finished task 101.0 in stage 17.0 (TID 635) in 1883 ms on 10.0.0.133 (executor driver) (102/200) +26/04/01 08:50:18 INFO Executor: Running task 109.0 in stage 17.0 (TID 643) +26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:18 INFO Executor: Finished task 102.0 in stage 17.0 (TID 636). 6401 bytes result sent to driver +26/04/01 08:50:18 INFO TaskSetManager: Starting task 110.0 in stage 17.0 (TID 644) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:18 INFO Executor: Running task 110.0 in stage 17.0 (TID 644) +26/04/01 08:50:18 INFO TaskSetManager: Finished task 102.0 in stage 17.0 (TID 636) in 1891 ms on 10.0.0.133 (executor driver) (103/200) +26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:18 INFO Executor: Finished task 103.0 in stage 17.0 (TID 637). 6401 bytes result sent to driver +26/04/01 08:50:18 INFO TaskSetManager: Starting task 111.0 in stage 17.0 (TID 645) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:18 INFO TaskSetManager: Finished task 103.0 in stage 17.0 (TID 637) in 1857 ms on 10.0.0.133 (executor driver) (104/200) +26/04/01 08:50:18 INFO Executor: Running task 111.0 in stage 17.0 (TID 645) +26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:19 INFO Executor: Finished task 105.0 in stage 17.0 (TID 639). 6401 bytes result sent to driver +26/04/01 08:50:19 INFO TaskSetManager: Starting task 112.0 in stage 17.0 (TID 646) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:19 INFO TaskSetManager: Finished task 105.0 in stage 17.0 (TID 639) in 1853 ms on 10.0.0.133 (executor driver) (105/200) +26/04/01 08:50:19 INFO Executor: Running task 112.0 in stage 17.0 (TID 646) +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:19 INFO Executor: Finished task 104.0 in stage 17.0 (TID 638). 6401 bytes result sent to driver +26/04/01 08:50:19 INFO TaskSetManager: Starting task 113.0 in stage 17.0 (TID 647) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:19 INFO Executor: Running task 113.0 in stage 17.0 (TID 647) +26/04/01 08:50:19 INFO TaskSetManager: Finished task 104.0 in stage 17.0 (TID 638) in 1863 ms on 10.0.0.133 (executor driver) (106/200) +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:19 INFO Executor: Finished task 106.0 in stage 17.0 (TID 640). 6401 bytes result sent to driver +26/04/01 08:50:19 INFO TaskSetManager: Starting task 114.0 in stage 17.0 (TID 648) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:19 INFO Executor: Running task 114.0 in stage 17.0 (TID 648) +26/04/01 08:50:19 INFO TaskSetManager: Finished task 106.0 in stage 17.0 (TID 640) in 1873 ms on 10.0.0.133 (executor driver) (107/200) +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:19 INFO Executor: Finished task 107.0 in stage 17.0 (TID 641). 6401 bytes result sent to driver +26/04/01 08:50:19 INFO TaskSetManager: Starting task 115.0 in stage 17.0 (TID 649) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:19 INFO TaskSetManager: Finished task 107.0 in stage 17.0 (TID 641) in 1878 ms on 10.0.0.133 (executor driver) (108/200) +26/04/01 08:50:19 INFO Executor: Running task 115.0 in stage 17.0 (TID 649) +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:19 INFO Executor: Finished task 108.0 in stage 17.0 (TID 642). 6401 bytes result sent to driver +26/04/01 08:50:19 INFO TaskSetManager: Starting task 116.0 in stage 17.0 (TID 650) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:19 INFO TaskSetManager: Finished task 108.0 in stage 17.0 (TID 642) in 1895 ms on 10.0.0.133 (executor driver) (109/200) +26/04/01 08:50:19 INFO Executor: Running task 116.0 in stage 17.0 (TID 650) +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:19 INFO Executor: Finished task 109.0 in stage 17.0 (TID 643). 6401 bytes result sent to driver +26/04/01 08:50:19 INFO TaskSetManager: Starting task 117.0 in stage 17.0 (TID 651) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:19 INFO TaskSetManager: Finished task 109.0 in stage 17.0 (TID 643) in 1892 ms on 10.0.0.133 (executor driver) (110/200) +26/04/01 08:50:19 INFO Executor: Running task 117.0 in stage 17.0 (TID 651) +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:19 INFO Executor: Finished task 110.0 in stage 17.0 (TID 644). 6401 bytes result sent to driver +26/04/01 08:50:19 INFO TaskSetManager: Starting task 118.0 in stage 17.0 (TID 652) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:19 INFO Executor: Running task 118.0 in stage 17.0 (TID 652) +26/04/01 08:50:19 INFO TaskSetManager: Finished task 110.0 in stage 17.0 (TID 644) in 1881 ms on 10.0.0.133 (executor driver) (111/200) +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (76.0 MiB) non-empty blocks including 208 (76.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (76.0 MiB) non-empty blocks including 208 (76.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:20 INFO Executor: Finished task 111.0 in stage 17.0 (TID 645). 6401 bytes result sent to driver +26/04/01 08:50:20 INFO TaskSetManager: Starting task 119.0 in stage 17.0 (TID 653) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:20 INFO TaskSetManager: Finished task 111.0 in stage 17.0 (TID 645) in 1871 ms on 10.0.0.133 (executor driver) (112/200) +26/04/01 08:50:20 INFO Executor: Running task 119.0 in stage 17.0 (TID 653) +26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:20 INFO Executor: Finished task 112.0 in stage 17.0 (TID 646). 6401 bytes result sent to driver +26/04/01 08:50:20 INFO TaskSetManager: Starting task 120.0 in stage 17.0 (TID 654) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:20 INFO TaskSetManager: Finished task 112.0 in stage 17.0 (TID 646) in 1867 ms on 10.0.0.133 (executor driver) (113/200) +26/04/01 08:50:20 INFO Executor: Running task 120.0 in stage 17.0 (TID 654) +26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:20 INFO Executor: Finished task 113.0 in stage 17.0 (TID 647). 6401 bytes result sent to driver +26/04/01 08:50:20 INFO TaskSetManager: Starting task 121.0 in stage 17.0 (TID 655) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:20 INFO Executor: Running task 121.0 in stage 17.0 (TID 655) +26/04/01 08:50:20 INFO TaskSetManager: Finished task 113.0 in stage 17.0 (TID 647) in 1865 ms on 10.0.0.133 (executor driver) (114/200) +26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:20 INFO Executor: Finished task 114.0 in stage 17.0 (TID 648). 6401 bytes result sent to driver +26/04/01 08:50:20 INFO TaskSetManager: Starting task 122.0 in stage 17.0 (TID 656) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:21 INFO TaskSetManager: Finished task 114.0 in stage 17.0 (TID 648) in 1847 ms on 10.0.0.133 (executor driver) (115/200) +26/04/01 08:50:21 INFO Executor: Running task 122.0 in stage 17.0 (TID 656) +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:21 INFO Executor: Finished task 115.0 in stage 17.0 (TID 649). 7132 bytes result sent to driver +26/04/01 08:50:21 INFO TaskSetManager: Starting task 123.0 in stage 17.0 (TID 657) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:21 INFO TaskSetManager: Finished task 115.0 in stage 17.0 (TID 649) in 1864 ms on 10.0.0.133 (executor driver) (116/200) +26/04/01 08:50:21 INFO Executor: Running task 123.0 in stage 17.0 (TID 657) +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:21 INFO Executor: Finished task 116.0 in stage 17.0 (TID 650). 7132 bytes result sent to driver +26/04/01 08:50:21 INFO TaskSetManager: Starting task 124.0 in stage 17.0 (TID 658) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:21 INFO TaskSetManager: Finished task 116.0 in stage 17.0 (TID 650) in 1855 ms on 10.0.0.133 (executor driver) (117/200) +26/04/01 08:50:21 INFO Executor: Running task 124.0 in stage 17.0 (TID 658) +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:21 INFO Executor: Finished task 117.0 in stage 17.0 (TID 651). 7132 bytes result sent to driver +26/04/01 08:50:21 INFO TaskSetManager: Starting task 125.0 in stage 17.0 (TID 659) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:21 INFO TaskSetManager: Finished task 117.0 in stage 17.0 (TID 651) in 1853 ms on 10.0.0.133 (executor driver) (118/200) +26/04/01 08:50:21 INFO Executor: Running task 125.0 in stage 17.0 (TID 659) +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:21 INFO Executor: Finished task 118.0 in stage 17.0 (TID 652). 7132 bytes result sent to driver +26/04/01 08:50:21 INFO TaskSetManager: Starting task 126.0 in stage 17.0 (TID 660) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:21 INFO TaskSetManager: Finished task 118.0 in stage 17.0 (TID 652) in 1860 ms on 10.0.0.133 (executor driver) (119/200) +26/04/01 08:50:21 INFO Executor: Running task 126.0 in stage 17.0 (TID 660) +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:21 INFO Executor: Finished task 119.0 in stage 17.0 (TID 653). 7132 bytes result sent to driver +26/04/01 08:50:21 INFO TaskSetManager: Starting task 127.0 in stage 17.0 (TID 661) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:21 INFO TaskSetManager: Finished task 119.0 in stage 17.0 (TID 653) in 1837 ms on 10.0.0.133 (executor driver) (120/200) +26/04/01 08:50:21 INFO Executor: Running task 127.0 in stage 17.0 (TID 661) +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:22 INFO Executor: Finished task 120.0 in stage 17.0 (TID 654). 7132 bytes result sent to driver +26/04/01 08:50:22 INFO TaskSetManager: Starting task 128.0 in stage 17.0 (TID 662) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:22 INFO Executor: Running task 128.0 in stage 17.0 (TID 662) +26/04/01 08:50:22 INFO TaskSetManager: Finished task 120.0 in stage 17.0 (TID 654) in 1835 ms on 10.0.0.133 (executor driver) (121/200) +26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:22 INFO Executor: Finished task 121.0 in stage 17.0 (TID 655). 7132 bytes result sent to driver +26/04/01 08:50:22 INFO TaskSetManager: Starting task 129.0 in stage 17.0 (TID 663) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:22 INFO Executor: Running task 129.0 in stage 17.0 (TID 663) +26/04/01 08:50:22 INFO TaskSetManager: Finished task 121.0 in stage 17.0 (TID 655) in 1833 ms on 10.0.0.133 (executor driver) (122/200) +26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:22 INFO Executor: Finished task 122.0 in stage 17.0 (TID 656). 7132 bytes result sent to driver +26/04/01 08:50:22 INFO TaskSetManager: Starting task 130.0 in stage 17.0 (TID 664) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:22 INFO Executor: Running task 130.0 in stage 17.0 (TID 664) +26/04/01 08:50:22 INFO TaskSetManager: Finished task 122.0 in stage 17.0 (TID 656) in 1835 ms on 10.0.0.133 (executor driver) (123/200) +26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:23 INFO Executor: Finished task 123.0 in stage 17.0 (TID 657). 6401 bytes result sent to driver +26/04/01 08:50:23 INFO TaskSetManager: Starting task 131.0 in stage 17.0 (TID 665) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:23 INFO TaskSetManager: Finished task 123.0 in stage 17.0 (TID 657) in 1835 ms on 10.0.0.133 (executor driver) (124/200) +26/04/01 08:50:23 INFO Executor: Running task 131.0 in stage 17.0 (TID 665) +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:23 INFO Executor: Finished task 124.0 in stage 17.0 (TID 658). 6401 bytes result sent to driver +26/04/01 08:50:23 INFO TaskSetManager: Starting task 132.0 in stage 17.0 (TID 666) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:23 INFO Executor: Running task 132.0 in stage 17.0 (TID 666) +26/04/01 08:50:23 INFO TaskSetManager: Finished task 124.0 in stage 17.0 (TID 658) in 1837 ms on 10.0.0.133 (executor driver) (125/200) +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:23 INFO Executor: Finished task 126.0 in stage 17.0 (TID 660). 6401 bytes result sent to driver +26/04/01 08:50:23 INFO TaskSetManager: Starting task 133.0 in stage 17.0 (TID 667) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:23 INFO TaskSetManager: Finished task 126.0 in stage 17.0 (TID 660) in 1835 ms on 10.0.0.133 (executor driver) (126/200) +26/04/01 08:50:23 INFO Executor: Running task 133.0 in stage 17.0 (TID 667) +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:23 INFO Executor: Finished task 125.0 in stage 17.0 (TID 659). 6401 bytes result sent to driver +26/04/01 08:50:23 INFO TaskSetManager: Starting task 134.0 in stage 17.0 (TID 668) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:23 INFO TaskSetManager: Finished task 125.0 in stage 17.0 (TID 659) in 1853 ms on 10.0.0.133 (executor driver) (127/200) +26/04/01 08:50:23 INFO Executor: Running task 134.0 in stage 17.0 (TID 668) +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:23 INFO Executor: Finished task 127.0 in stage 17.0 (TID 661). 6401 bytes result sent to driver +26/04/01 08:50:23 INFO TaskSetManager: Starting task 135.0 in stage 17.0 (TID 669) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:23 INFO TaskSetManager: Finished task 127.0 in stage 17.0 (TID 661) in 1847 ms on 10.0.0.133 (executor driver) (128/200) +26/04/01 08:50:23 INFO Executor: Running task 135.0 in stage 17.0 (TID 669) +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:24 INFO Executor: Finished task 129.0 in stage 17.0 (TID 663). 6401 bytes result sent to driver +26/04/01 08:50:24 INFO TaskSetManager: Starting task 136.0 in stage 17.0 (TID 670) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:24 INFO Executor: Running task 136.0 in stage 17.0 (TID 670) +26/04/01 08:50:24 INFO TaskSetManager: Finished task 129.0 in stage 17.0 (TID 663) in 1869 ms on 10.0.0.133 (executor driver) (129/200) +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:24 INFO Executor: Finished task 128.0 in stage 17.0 (TID 662). 6401 bytes result sent to driver +26/04/01 08:50:24 INFO TaskSetManager: Starting task 137.0 in stage 17.0 (TID 671) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:24 INFO Executor: Running task 137.0 in stage 17.0 (TID 671) +26/04/01 08:50:24 INFO TaskSetManager: Finished task 128.0 in stage 17.0 (TID 662) in 1878 ms on 10.0.0.133 (executor driver) (130/200) +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:24 INFO Executor: Finished task 130.0 in stage 17.0 (TID 664). 6401 bytes result sent to driver +26/04/01 08:50:24 INFO TaskSetManager: Starting task 138.0 in stage 17.0 (TID 672) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:24 INFO Executor: Running task 138.0 in stage 17.0 (TID 672) +26/04/01 08:50:24 INFO TaskSetManager: Finished task 130.0 in stage 17.0 (TID 664) in 1865 ms on 10.0.0.133 (executor driver) (131/200) +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:24 INFO Executor: Finished task 131.0 in stage 17.0 (TID 665). 6401 bytes result sent to driver +26/04/01 08:50:24 INFO TaskSetManager: Starting task 139.0 in stage 17.0 (TID 673) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:24 INFO Executor: Running task 139.0 in stage 17.0 (TID 673) +26/04/01 08:50:24 INFO TaskSetManager: Finished task 131.0 in stage 17.0 (TID 665) in 1853 ms on 10.0.0.133 (executor driver) (132/200) +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:25 INFO Executor: Finished task 132.0 in stage 17.0 (TID 666). 6401 bytes result sent to driver +26/04/01 08:50:25 INFO TaskSetManager: Starting task 140.0 in stage 17.0 (TID 674) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:25 INFO Executor: Running task 140.0 in stage 17.0 (TID 674) +26/04/01 08:50:25 INFO TaskSetManager: Finished task 132.0 in stage 17.0 (TID 666) in 1832 ms on 10.0.0.133 (executor driver) (133/200) +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:25 INFO Executor: Finished task 134.0 in stage 17.0 (TID 668). 6401 bytes result sent to driver +26/04/01 08:50:25 INFO Executor: Finished task 133.0 in stage 17.0 (TID 667). 6401 bytes result sent to driver +26/04/01 08:50:25 INFO TaskSetManager: Starting task 141.0 in stage 17.0 (TID 675) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:25 INFO Executor: Running task 141.0 in stage 17.0 (TID 675) +26/04/01 08:50:25 INFO TaskSetManager: Starting task 142.0 in stage 17.0 (TID 676) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:25 INFO TaskSetManager: Finished task 134.0 in stage 17.0 (TID 668) in 1822 ms on 10.0.0.133 (executor driver) (134/200) +26/04/01 08:50:25 INFO Executor: Running task 142.0 in stage 17.0 (TID 676) +26/04/01 08:50:25 INFO TaskSetManager: Finished task 133.0 in stage 17.0 (TID 667) in 1828 ms on 10.0.0.133 (executor driver) (135/200) +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:25 INFO Executor: Finished task 135.0 in stage 17.0 (TID 669). 6401 bytes result sent to driver +26/04/01 08:50:25 INFO TaskSetManager: Starting task 143.0 in stage 17.0 (TID 677) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:25 INFO Executor: Running task 143.0 in stage 17.0 (TID 677) +26/04/01 08:50:25 INFO TaskSetManager: Finished task 135.0 in stage 17.0 (TID 669) in 1833 ms on 10.0.0.133 (executor driver) (136/200) +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:26 INFO Executor: Finished task 136.0 in stage 17.0 (TID 670). 6401 bytes result sent to driver +26/04/01 08:50:26 INFO TaskSetManager: Starting task 144.0 in stage 17.0 (TID 678) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:26 INFO Executor: Running task 144.0 in stage 17.0 (TID 678) +26/04/01 08:50:26 INFO TaskSetManager: Finished task 136.0 in stage 17.0 (TID 670) in 1829 ms on 10.0.0.133 (executor driver) (137/200) +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:26 INFO Executor: Finished task 137.0 in stage 17.0 (TID 671). 6401 bytes result sent to driver +26/04/01 08:50:26 INFO TaskSetManager: Starting task 145.0 in stage 17.0 (TID 679) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:26 INFO TaskSetManager: Finished task 137.0 in stage 17.0 (TID 671) in 1836 ms on 10.0.0.133 (executor driver) (138/200) +26/04/01 08:50:26 INFO Executor: Running task 145.0 in stage 17.0 (TID 679) +26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:26 INFO Executor: Finished task 138.0 in stage 17.0 (TID 672). 6401 bytes result sent to driver +26/04/01 08:50:26 INFO TaskSetManager: Starting task 146.0 in stage 17.0 (TID 680) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:26 INFO TaskSetManager: Finished task 138.0 in stage 17.0 (TID 672) in 1836 ms on 10.0.0.133 (executor driver) (139/200) +26/04/01 08:50:26 INFO Executor: Running task 146.0 in stage 17.0 (TID 680) +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:26 INFO Executor: Finished task 139.0 in stage 17.0 (TID 673). 6401 bytes result sent to driver +26/04/01 08:50:26 INFO TaskSetManager: Starting task 147.0 in stage 17.0 (TID 681) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:26 INFO TaskSetManager: Finished task 139.0 in stage 17.0 (TID 673) in 1851 ms on 10.0.0.133 (executor driver) (140/200) +26/04/01 08:50:26 INFO Executor: Running task 147.0 in stage 17.0 (TID 681) +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:27 INFO Executor: Finished task 140.0 in stage 17.0 (TID 674). 6401 bytes result sent to driver +26/04/01 08:50:27 INFO TaskSetManager: Starting task 148.0 in stage 17.0 (TID 682) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:27 INFO TaskSetManager: Finished task 140.0 in stage 17.0 (TID 674) in 1835 ms on 10.0.0.133 (executor driver) (141/200) +26/04/01 08:50:27 INFO Executor: Running task 148.0 in stage 17.0 (TID 682) +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:27 INFO Executor: Finished task 141.0 in stage 17.0 (TID 675). 6401 bytes result sent to driver +26/04/01 08:50:27 INFO TaskSetManager: Starting task 149.0 in stage 17.0 (TID 683) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:27 INFO TaskSetManager: Finished task 141.0 in stage 17.0 (TID 675) in 1845 ms on 10.0.0.133 (executor driver) (142/200) +26/04/01 08:50:27 INFO Executor: Running task 149.0 in stage 17.0 (TID 683) +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:27 INFO Executor: Finished task 142.0 in stage 17.0 (TID 676). 6401 bytes result sent to driver +26/04/01 08:50:27 INFO TaskSetManager: Starting task 150.0 in stage 17.0 (TID 684) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:27 INFO TaskSetManager: Finished task 142.0 in stage 17.0 (TID 676) in 1847 ms on 10.0.0.133 (executor driver) (143/200) +26/04/01 08:50:27 INFO Executor: Running task 150.0 in stage 17.0 (TID 684) +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:27 INFO Executor: Finished task 143.0 in stage 17.0 (TID 677). 6401 bytes result sent to driver +26/04/01 08:50:27 INFO TaskSetManager: Starting task 151.0 in stage 17.0 (TID 685) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:27 INFO TaskSetManager: Finished task 143.0 in stage 17.0 (TID 677) in 1849 ms on 10.0.0.133 (executor driver) (144/200) +26/04/01 08:50:27 INFO Executor: Running task 151.0 in stage 17.0 (TID 685) +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:28 INFO Executor: Finished task 144.0 in stage 17.0 (TID 678). 6401 bytes result sent to driver +26/04/01 08:50:28 INFO TaskSetManager: Starting task 152.0 in stage 17.0 (TID 686) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:28 INFO TaskSetManager: Finished task 144.0 in stage 17.0 (TID 678) in 1846 ms on 10.0.0.133 (executor driver) (145/200) +26/04/01 08:50:28 INFO Executor: Running task 152.0 in stage 17.0 (TID 686) +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:28 INFO Executor: Finished task 145.0 in stage 17.0 (TID 679). 6401 bytes result sent to driver +26/04/01 08:50:28 INFO TaskSetManager: Starting task 153.0 in stage 17.0 (TID 687) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:28 INFO TaskSetManager: Finished task 145.0 in stage 17.0 (TID 679) in 1839 ms on 10.0.0.133 (executor driver) (146/200) +26/04/01 08:50:28 INFO Executor: Running task 153.0 in stage 17.0 (TID 687) +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:28 INFO Executor: Finished task 146.0 in stage 17.0 (TID 680). 6401 bytes result sent to driver +26/04/01 08:50:28 INFO TaskSetManager: Starting task 154.0 in stage 17.0 (TID 688) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:28 INFO TaskSetManager: Finished task 146.0 in stage 17.0 (TID 680) in 1846 ms on 10.0.0.133 (executor driver) (147/200) +26/04/01 08:50:28 INFO Executor: Running task 154.0 in stage 17.0 (TID 688) +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:28 INFO Executor: Finished task 147.0 in stage 17.0 (TID 681). 6401 bytes result sent to driver +26/04/01 08:50:28 INFO TaskSetManager: Starting task 155.0 in stage 17.0 (TID 689) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:28 INFO TaskSetManager: Finished task 147.0 in stage 17.0 (TID 681) in 1875 ms on 10.0.0.133 (executor driver) (148/200) +26/04/01 08:50:28 INFO Executor: Running task 155.0 in stage 17.0 (TID 689) +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:28 INFO Executor: Finished task 148.0 in stage 17.0 (TID 682). 6401 bytes result sent to driver +26/04/01 08:50:28 INFO TaskSetManager: Starting task 156.0 in stage 17.0 (TID 690) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:28 INFO TaskSetManager: Finished task 148.0 in stage 17.0 (TID 682) in 1874 ms on 10.0.0.133 (executor driver) (149/200) +26/04/01 08:50:28 INFO Executor: Running task 156.0 in stage 17.0 (TID 690) +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:29 INFO Executor: Finished task 150.0 in stage 17.0 (TID 684). 6401 bytes result sent to driver +26/04/01 08:50:29 INFO TaskSetManager: Starting task 157.0 in stage 17.0 (TID 691) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:29 INFO Executor: Finished task 149.0 in stage 17.0 (TID 683). 6401 bytes result sent to driver +26/04/01 08:50:29 INFO Executor: Running task 157.0 in stage 17.0 (TID 691) +26/04/01 08:50:29 INFO TaskSetManager: Starting task 158.0 in stage 17.0 (TID 692) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:29 INFO TaskSetManager: Finished task 150.0 in stage 17.0 (TID 684) in 1871 ms on 10.0.0.133 (executor driver) (150/200) +26/04/01 08:50:29 INFO Executor: Running task 158.0 in stage 17.0 (TID 692) +26/04/01 08:50:29 INFO TaskSetManager: Finished task 149.0 in stage 17.0 (TID 683) in 1874 ms on 10.0.0.133 (executor driver) (151/200) +26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:29 INFO Executor: Finished task 151.0 in stage 17.0 (TID 685). 6401 bytes result sent to driver +26/04/01 08:50:29 INFO TaskSetManager: Starting task 159.0 in stage 17.0 (TID 693) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:29 INFO Executor: Running task 159.0 in stage 17.0 (TID 693) +26/04/01 08:50:29 INFO TaskSetManager: Finished task 151.0 in stage 17.0 (TID 685) in 1858 ms on 10.0.0.133 (executor driver) (152/200) +26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:30 INFO Executor: Finished task 153.0 in stage 17.0 (TID 687). 6401 bytes result sent to driver +26/04/01 08:50:30 INFO TaskSetManager: Starting task 160.0 in stage 17.0 (TID 694) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:30 INFO TaskSetManager: Finished task 153.0 in stage 17.0 (TID 687) in 1899 ms on 10.0.0.133 (executor driver) (153/200) +26/04/01 08:50:30 INFO Executor: Running task 160.0 in stage 17.0 (TID 694) +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:30 INFO Executor: Finished task 152.0 in stage 17.0 (TID 686). 6401 bytes result sent to driver +26/04/01 08:50:30 INFO TaskSetManager: Starting task 161.0 in stage 17.0 (TID 695) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:30 INFO Executor: Running task 161.0 in stage 17.0 (TID 695) +26/04/01 08:50:30 INFO TaskSetManager: Finished task 152.0 in stage 17.0 (TID 686) in 1907 ms on 10.0.0.133 (executor driver) (154/200) +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:30 INFO Executor: Finished task 154.0 in stage 17.0 (TID 688). 6401 bytes result sent to driver +26/04/01 08:50:30 INFO TaskSetManager: Starting task 162.0 in stage 17.0 (TID 696) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:30 INFO TaskSetManager: Finished task 154.0 in stage 17.0 (TID 688) in 1871 ms on 10.0.0.133 (executor driver) (155/200) +26/04/01 08:50:30 INFO Executor: Running task 162.0 in stage 17.0 (TID 696) +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:30 INFO Executor: Finished task 155.0 in stage 17.0 (TID 689). 6401 bytes result sent to driver +26/04/01 08:50:30 INFO TaskSetManager: Starting task 163.0 in stage 17.0 (TID 697) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:30 INFO Executor: Running task 163.0 in stage 17.0 (TID 697) +26/04/01 08:50:30 INFO TaskSetManager: Finished task 155.0 in stage 17.0 (TID 689) in 1851 ms on 10.0.0.133 (executor driver) (156/200) +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:30 INFO Executor: Finished task 156.0 in stage 17.0 (TID 690). 6401 bytes result sent to driver +26/04/01 08:50:30 INFO TaskSetManager: Starting task 164.0 in stage 17.0 (TID 698) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:30 INFO Executor: Running task 164.0 in stage 17.0 (TID 698) +26/04/01 08:50:30 INFO TaskSetManager: Finished task 156.0 in stage 17.0 (TID 690) in 1867 ms on 10.0.0.133 (executor driver) (157/200) +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:30 INFO Executor: Finished task 157.0 in stage 17.0 (TID 691). 6401 bytes result sent to driver +26/04/01 08:50:30 INFO TaskSetManager: Starting task 165.0 in stage 17.0 (TID 699) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:30 INFO TaskSetManager: Finished task 157.0 in stage 17.0 (TID 691) in 1840 ms on 10.0.0.133 (executor driver) (158/200) +26/04/01 08:50:30 INFO Executor: Running task 165.0 in stage 17.0 (TID 699) +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:31 INFO Executor: Finished task 158.0 in stage 17.0 (TID 692). 6401 bytes result sent to driver +26/04/01 08:50:31 INFO TaskSetManager: Starting task 166.0 in stage 17.0 (TID 700) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:31 INFO TaskSetManager: Finished task 158.0 in stage 17.0 (TID 692) in 1863 ms on 10.0.0.133 (executor driver) (159/200) +26/04/01 08:50:31 INFO Executor: Running task 166.0 in stage 17.0 (TID 700) +26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:31 INFO Executor: Finished task 159.0 in stage 17.0 (TID 693). 7132 bytes result sent to driver +26/04/01 08:50:31 INFO TaskSetManager: Starting task 167.0 in stage 17.0 (TID 701) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:31 INFO TaskSetManager: Finished task 159.0 in stage 17.0 (TID 693) in 1868 ms on 10.0.0.133 (executor driver) (160/200) +26/04/01 08:50:31 INFO Executor: Running task 167.0 in stage 17.0 (TID 701) +26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO Executor: Finished task 161.0 in stage 17.0 (TID 695). 7132 bytes result sent to driver +26/04/01 08:50:32 INFO TaskSetManager: Starting task 168.0 in stage 17.0 (TID 702) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:32 INFO TaskSetManager: Finished task 161.0 in stage 17.0 (TID 695) in 1859 ms on 10.0.0.133 (executor driver) (161/200) +26/04/01 08:50:32 INFO Executor: Running task 168.0 in stage 17.0 (TID 702) +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO Executor: Finished task 160.0 in stage 17.0 (TID 694). 7132 bytes result sent to driver +26/04/01 08:50:32 INFO TaskSetManager: Starting task 169.0 in stage 17.0 (TID 703) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:32 INFO TaskSetManager: Finished task 160.0 in stage 17.0 (TID 694) in 1871 ms on 10.0.0.133 (executor driver) (162/200) +26/04/01 08:50:32 INFO Executor: Running task 169.0 in stage 17.0 (TID 703) +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO Executor: Finished task 162.0 in stage 17.0 (TID 696). 7132 bytes result sent to driver +26/04/01 08:50:32 INFO TaskSetManager: Starting task 170.0 in stage 17.0 (TID 704) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:32 INFO Executor: Running task 170.0 in stage 17.0 (TID 704) +26/04/01 08:50:32 INFO TaskSetManager: Finished task 162.0 in stage 17.0 (TID 696) in 1866 ms on 10.0.0.133 (executor driver) (163/200) +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO Executor: Finished task 163.0 in stage 17.0 (TID 697). 7132 bytes result sent to driver +26/04/01 08:50:32 INFO TaskSetManager: Starting task 171.0 in stage 17.0 (TID 705) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:32 INFO Executor: Running task 171.0 in stage 17.0 (TID 705) +26/04/01 08:50:32 INFO TaskSetManager: Finished task 163.0 in stage 17.0 (TID 697) in 1842 ms on 10.0.0.133 (executor driver) (164/200) +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO Executor: Finished task 164.0 in stage 17.0 (TID 698). 7132 bytes result sent to driver +26/04/01 08:50:32 INFO TaskSetManager: Starting task 172.0 in stage 17.0 (TID 706) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:32 INFO Executor: Running task 172.0 in stage 17.0 (TID 706) +26/04/01 08:50:32 INFO TaskSetManager: Finished task 164.0 in stage 17.0 (TID 698) in 1848 ms on 10.0.0.133 (executor driver) (165/200) +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO Executor: Finished task 165.0 in stage 17.0 (TID 699). 7132 bytes result sent to driver +26/04/01 08:50:32 INFO TaskSetManager: Starting task 173.0 in stage 17.0 (TID 707) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:32 INFO TaskSetManager: Finished task 165.0 in stage 17.0 (TID 699) in 1832 ms on 10.0.0.133 (executor driver) (166/200) +26/04/01 08:50:32 INFO Executor: Running task 173.0 in stage 17.0 (TID 707) +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO Executor: Finished task 166.0 in stage 17.0 (TID 700). 7132 bytes result sent to driver +26/04/01 08:50:32 INFO TaskSetManager: Starting task 174.0 in stage 17.0 (TID 708) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:32 INFO TaskSetManager: Finished task 166.0 in stage 17.0 (TID 700) in 1850 ms on 10.0.0.133 (executor driver) (167/200) +26/04/01 08:50:32 INFO Executor: Running task 174.0 in stage 17.0 (TID 708) +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO Executor: Finished task 167.0 in stage 17.0 (TID 701). 6401 bytes result sent to driver +26/04/01 08:50:32 INFO TaskSetManager: Starting task 175.0 in stage 17.0 (TID 709) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:32 INFO TaskSetManager: Finished task 167.0 in stage 17.0 (TID 701) in 1827 ms on 10.0.0.133 (executor driver) (168/200) +26/04/01 08:50:32 INFO Executor: Running task 175.0 in stage 17.0 (TID 709) +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:33 INFO Executor: Finished task 168.0 in stage 17.0 (TID 702). 6401 bytes result sent to driver +26/04/01 08:50:33 INFO TaskSetManager: Starting task 176.0 in stage 17.0 (TID 710) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:33 INFO TaskSetManager: Finished task 168.0 in stage 17.0 (TID 702) in 1819 ms on 10.0.0.133 (executor driver) (169/200) +26/04/01 08:50:33 INFO Executor: Running task 176.0 in stage 17.0 (TID 710) +26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:33 INFO Executor: Finished task 169.0 in stage 17.0 (TID 703). 6401 bytes result sent to driver +26/04/01 08:50:33 INFO TaskSetManager: Starting task 177.0 in stage 17.0 (TID 711) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:33 INFO Executor: Running task 177.0 in stage 17.0 (TID 711) +26/04/01 08:50:33 INFO TaskSetManager: Finished task 169.0 in stage 17.0 (TID 703) in 1839 ms on 10.0.0.133 (executor driver) (170/200) +26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:33 INFO Executor: Finished task 170.0 in stage 17.0 (TID 704). 6401 bytes result sent to driver +26/04/01 08:50:33 INFO TaskSetManager: Starting task 178.0 in stage 17.0 (TID 712) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:33 INFO Executor: Running task 178.0 in stage 17.0 (TID 712) +26/04/01 08:50:33 INFO TaskSetManager: Finished task 170.0 in stage 17.0 (TID 704) in 1836 ms on 10.0.0.133 (executor driver) (171/200) +26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:34 INFO Executor: Finished task 171.0 in stage 17.0 (TID 705). 6401 bytes result sent to driver +26/04/01 08:50:34 INFO TaskSetManager: Starting task 179.0 in stage 17.0 (TID 713) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:34 INFO Executor: Running task 179.0 in stage 17.0 (TID 713) +26/04/01 08:50:34 INFO TaskSetManager: Finished task 171.0 in stage 17.0 (TID 705) in 1848 ms on 10.0.0.133 (executor driver) (172/200) +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:34 INFO Executor: Finished task 172.0 in stage 17.0 (TID 706). 6401 bytes result sent to driver +26/04/01 08:50:34 INFO TaskSetManager: Starting task 180.0 in stage 17.0 (TID 714) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:34 INFO TaskSetManager: Finished task 172.0 in stage 17.0 (TID 706) in 1836 ms on 10.0.0.133 (executor driver) (173/200) +26/04/01 08:50:34 INFO Executor: Running task 180.0 in stage 17.0 (TID 714) +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:34 INFO Executor: Finished task 173.0 in stage 17.0 (TID 707). 6401 bytes result sent to driver +26/04/01 08:50:34 INFO TaskSetManager: Starting task 181.0 in stage 17.0 (TID 715) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:34 INFO TaskSetManager: Finished task 173.0 in stage 17.0 (TID 707) in 1848 ms on 10.0.0.133 (executor driver) (174/200) +26/04/01 08:50:34 INFO Executor: Running task 181.0 in stage 17.0 (TID 715) +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:34 INFO Executor: Finished task 174.0 in stage 17.0 (TID 708). 6401 bytes result sent to driver +26/04/01 08:50:34 INFO TaskSetManager: Starting task 182.0 in stage 17.0 (TID 716) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:34 INFO TaskSetManager: Finished task 174.0 in stage 17.0 (TID 708) in 1831 ms on 10.0.0.133 (executor driver) (175/200) +26/04/01 08:50:34 INFO Executor: Running task 182.0 in stage 17.0 (TID 716) +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:34 INFO Executor: Finished task 175.0 in stage 17.0 (TID 709). 6401 bytes result sent to driver +26/04/01 08:50:34 INFO TaskSetManager: Starting task 183.0 in stage 17.0 (TID 717) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:34 INFO TaskSetManager: Finished task 175.0 in stage 17.0 (TID 709) in 1830 ms on 10.0.0.133 (executor driver) (176/200) +26/04/01 08:50:34 INFO Executor: Running task 183.0 in stage 17.0 (TID 717) +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:35 INFO Executor: Finished task 176.0 in stage 17.0 (TID 710). 6401 bytes result sent to driver +26/04/01 08:50:35 INFO TaskSetManager: Starting task 184.0 in stage 17.0 (TID 718) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:35 INFO Executor: Running task 184.0 in stage 17.0 (TID 718) +26/04/01 08:50:35 INFO TaskSetManager: Finished task 176.0 in stage 17.0 (TID 710) in 1818 ms on 10.0.0.133 (executor driver) (177/200) +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:35 INFO Executor: Finished task 177.0 in stage 17.0 (TID 711). 6401 bytes result sent to driver +26/04/01 08:50:35 INFO TaskSetManager: Starting task 185.0 in stage 17.0 (TID 719) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:35 INFO Executor: Running task 185.0 in stage 17.0 (TID 719) +26/04/01 08:50:35 INFO TaskSetManager: Finished task 177.0 in stage 17.0 (TID 711) in 1831 ms on 10.0.0.133 (executor driver) (178/200) +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (77.0 MiB) non-empty blocks including 208 (77.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (77.0 MiB) non-empty blocks including 208 (77.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:35 INFO Executor: Finished task 178.0 in stage 17.0 (TID 712). 6401 bytes result sent to driver +26/04/01 08:50:35 INFO TaskSetManager: Starting task 186.0 in stage 17.0 (TID 720) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:35 INFO Executor: Running task 186.0 in stage 17.0 (TID 720) +26/04/01 08:50:35 INFO TaskSetManager: Finished task 178.0 in stage 17.0 (TID 712) in 1836 ms on 10.0.0.133 (executor driver) (179/200) +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:35 INFO Executor: Finished task 179.0 in stage 17.0 (TID 713). 6401 bytes result sent to driver +26/04/01 08:50:35 INFO TaskSetManager: Starting task 187.0 in stage 17.0 (TID 721) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:35 INFO TaskSetManager: Finished task 179.0 in stage 17.0 (TID 713) in 1823 ms on 10.0.0.133 (executor driver) (180/200) +26/04/01 08:50:35 INFO Executor: Running task 187.0 in stage 17.0 (TID 721) +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:36 INFO Executor: Finished task 180.0 in stage 17.0 (TID 714). 6401 bytes result sent to driver +26/04/01 08:50:36 INFO TaskSetManager: Starting task 188.0 in stage 17.0 (TID 722) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:36 INFO Executor: Running task 188.0 in stage 17.0 (TID 722) +26/04/01 08:50:36 INFO TaskSetManager: Finished task 180.0 in stage 17.0 (TID 714) in 1814 ms on 10.0.0.133 (executor driver) (181/200) +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:36 INFO Executor: Finished task 181.0 in stage 17.0 (TID 715). 6401 bytes result sent to driver +26/04/01 08:50:36 INFO TaskSetManager: Starting task 189.0 in stage 17.0 (TID 723) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:36 INFO TaskSetManager: Finished task 181.0 in stage 17.0 (TID 715) in 1828 ms on 10.0.0.133 (executor driver) (182/200) +26/04/01 08:50:36 INFO Executor: Running task 189.0 in stage 17.0 (TID 723) +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:36 INFO Executor: Finished task 182.0 in stage 17.0 (TID 716). 6401 bytes result sent to driver +26/04/01 08:50:36 INFO TaskSetManager: Starting task 190.0 in stage 17.0 (TID 724) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:36 INFO TaskSetManager: Finished task 182.0 in stage 17.0 (TID 716) in 1825 ms on 10.0.0.133 (executor driver) (183/200) +26/04/01 08:50:36 INFO Executor: Running task 190.0 in stage 17.0 (TID 724) +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:36 INFO Executor: Finished task 183.0 in stage 17.0 (TID 717). 6401 bytes result sent to driver +26/04/01 08:50:36 INFO TaskSetManager: Starting task 191.0 in stage 17.0 (TID 725) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:36 INFO TaskSetManager: Finished task 183.0 in stage 17.0 (TID 717) in 1832 ms on 10.0.0.133 (executor driver) (184/200) +26/04/01 08:50:36 INFO Executor: Running task 191.0 in stage 17.0 (TID 725) +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:37 INFO Executor: Finished task 184.0 in stage 17.0 (TID 718). 6401 bytes result sent to driver +26/04/01 08:50:37 INFO TaskSetManager: Starting task 192.0 in stage 17.0 (TID 726) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:37 INFO Executor: Running task 192.0 in stage 17.0 (TID 726) +26/04/01 08:50:37 INFO TaskSetManager: Finished task 184.0 in stage 17.0 (TID 718) in 1842 ms on 10.0.0.133 (executor driver) (185/200) +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:37 INFO Executor: Finished task 185.0 in stage 17.0 (TID 719). 6401 bytes result sent to driver +26/04/01 08:50:37 INFO TaskSetManager: Starting task 193.0 in stage 17.0 (TID 727) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:37 INFO TaskSetManager: Finished task 185.0 in stage 17.0 (TID 719) in 1831 ms on 10.0.0.133 (executor driver) (186/200) +26/04/01 08:50:37 INFO Executor: Running task 193.0 in stage 17.0 (TID 727) +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:37 INFO Executor: Finished task 186.0 in stage 17.0 (TID 720). 6401 bytes result sent to driver +26/04/01 08:50:37 INFO TaskSetManager: Starting task 194.0 in stage 17.0 (TID 728) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:37 INFO Executor: Running task 194.0 in stage 17.0 (TID 728) +26/04/01 08:50:37 INFO TaskSetManager: Finished task 186.0 in stage 17.0 (TID 720) in 1822 ms on 10.0.0.133 (executor driver) (187/200) +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:37 INFO Executor: Finished task 187.0 in stage 17.0 (TID 721). 6401 bytes result sent to driver +26/04/01 08:50:37 INFO TaskSetManager: Starting task 195.0 in stage 17.0 (TID 729) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:37 INFO TaskSetManager: Finished task 187.0 in stage 17.0 (TID 721) in 1851 ms on 10.0.0.133 (executor driver) (188/200) +26/04/01 08:50:37 INFO Executor: Running task 195.0 in stage 17.0 (TID 729) +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:38 INFO Executor: Finished task 188.0 in stage 17.0 (TID 722). 6401 bytes result sent to driver +26/04/01 08:50:38 INFO TaskSetManager: Starting task 196.0 in stage 17.0 (TID 730) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:38 INFO Executor: Running task 196.0 in stage 17.0 (TID 730) +26/04/01 08:50:38 INFO TaskSetManager: Finished task 188.0 in stage 17.0 (TID 722) in 1851 ms on 10.0.0.133 (executor driver) (189/200) +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:38 INFO Executor: Finished task 189.0 in stage 17.0 (TID 723). 6401 bytes result sent to driver +26/04/01 08:50:38 INFO TaskSetManager: Starting task 197.0 in stage 17.0 (TID 731) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:38 INFO Executor: Running task 197.0 in stage 17.0 (TID 731) +26/04/01 08:50:38 INFO TaskSetManager: Finished task 189.0 in stage 17.0 (TID 723) in 1840 ms on 10.0.0.133 (executor driver) (190/200) +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:38 INFO Executor: Finished task 190.0 in stage 17.0 (TID 724). 6401 bytes result sent to driver +26/04/01 08:50:38 INFO TaskSetManager: Starting task 198.0 in stage 17.0 (TID 732) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:38 INFO TaskSetManager: Finished task 190.0 in stage 17.0 (TID 724) in 1838 ms on 10.0.0.133 (executor driver) (191/200) +26/04/01 08:50:38 INFO Executor: Running task 198.0 in stage 17.0 (TID 732) +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:38 INFO Executor: Finished task 191.0 in stage 17.0 (TID 725). 6401 bytes result sent to driver +26/04/01 08:50:38 INFO TaskSetManager: Starting task 199.0 in stage 17.0 (TID 733) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9624 bytes) +26/04/01 08:50:38 INFO Executor: Running task 199.0 in stage 17.0 (TID 733) +26/04/01 08:50:38 INFO TaskSetManager: Finished task 191.0 in stage 17.0 (TID 725) in 1843 ms on 10.0.0.133 (executor driver) (192/200) +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:39 INFO Executor: Finished task 192.0 in stage 17.0 (TID 726). 6401 bytes result sent to driver +26/04/01 08:50:39 INFO TaskSetManager: Finished task 192.0 in stage 17.0 (TID 726) in 1842 ms on 10.0.0.133 (executor driver) (193/200) +26/04/01 08:50:39 INFO Executor: Finished task 193.0 in stage 17.0 (TID 727). 6401 bytes result sent to driver +26/04/01 08:50:39 INFO TaskSetManager: Finished task 193.0 in stage 17.0 (TID 727) in 1846 ms on 10.0.0.133 (executor driver) (194/200) +26/04/01 08:50:39 INFO Executor: Finished task 194.0 in stage 17.0 (TID 728). 6401 bytes result sent to driver +26/04/01 08:50:39 INFO TaskSetManager: Finished task 194.0 in stage 17.0 (TID 728) in 1827 ms on 10.0.0.133 (executor driver) (195/200) +26/04/01 08:50:39 INFO Executor: Finished task 195.0 in stage 17.0 (TID 729). 6401 bytes result sent to driver +26/04/01 08:50:39 INFO TaskSetManager: Finished task 195.0 in stage 17.0 (TID 729) in 1856 ms on 10.0.0.133 (executor driver) (196/200) +26/04/01 08:50:40 INFO Executor: Finished task 196.0 in stage 17.0 (TID 730). 6401 bytes result sent to driver +26/04/01 08:50:40 INFO TaskSetManager: Finished task 196.0 in stage 17.0 (TID 730) in 1846 ms on 10.0.0.133 (executor driver) (197/200) +26/04/01 08:50:40 INFO Executor: Finished task 197.0 in stage 17.0 (TID 731). 6401 bytes result sent to driver +26/04/01 08:50:40 INFO TaskSetManager: Finished task 197.0 in stage 17.0 (TID 731) in 1813 ms on 10.0.0.133 (executor driver) (198/200) +26/04/01 08:50:40 INFO Executor: Finished task 198.0 in stage 17.0 (TID 732). 6401 bytes result sent to driver +26/04/01 08:50:40 INFO TaskSetManager: Finished task 198.0 in stage 17.0 (TID 732) in 1809 ms on 10.0.0.133 (executor driver) (199/200) +26/04/01 08:50:40 INFO Executor: Finished task 199.0 in stage 17.0 (TID 733). 6401 bytes result sent to driver +26/04/01 08:50:40 INFO TaskSetManager: Finished task 199.0 in stage 17.0 (TID 733) in 1802 ms on 10.0.0.133 (executor driver) (200/200) +26/04/01 08:50:40 INFO TaskSchedulerImpl: Removed TaskSet 17.0, whose tasks have all completed, from pool +26/04/01 08:50:40 INFO DAGScheduler: ShuffleMapStage 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 47.315 s +26/04/01 08:50:40 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:50:40 INFO DAGScheduler: running: Set() +26/04/01 08:50:40 INFO DAGScheduler: waiting: Set() +26/04/01 08:50:40 INFO DAGScheduler: failed: Set() +26/04/01 08:50:40 INFO ShufflePartitionsUtil: For shuffle(5, 2), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 08:50:40 INFO DAGScheduler: Registering RDD 43 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 6 +26/04/01 08:50:40 INFO DAGScheduler: Got map stage job 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 14 output partitions +26/04/01 08:50:40 INFO DAGScheduler: Final stage: ShuffleMapStage 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:50:40 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 19, ShuffleMapStage 20) +26/04/01 08:50:40 INFO DAGScheduler: Missing parents: List() +26/04/01 08:50:40 INFO DAGScheduler: Submitting ShuffleMapStage 21 (MapPartitionsRDD[43] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:50:40 INFO MemoryStore: Block broadcast_24 stored as values in memory (estimated size 24.1 KiB, free 12.6 GiB) +26/04/01 08:50:40 INFO MemoryStore: Block broadcast_24_piece0 stored as bytes in memory (estimated size 9.9 KiB, free 12.6 GiB) +26/04/01 08:50:40 INFO BlockManagerInfo: Added broadcast_24_piece0 in memory on 10.0.0.133:59333 (size: 9.9 KiB, free: 12.6 GiB) +26/04/01 08:50:40 INFO SparkContext: Created broadcast 24 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:50:40 INFO DAGScheduler: Submitting 14 missing tasks from ShuffleMapStage 21 (MapPartitionsRDD[43] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)) +26/04/01 08:50:40 INFO TaskSchedulerImpl: Adding task set 21.0 with 14 tasks resource profile 0 +26/04/01 08:50:40 INFO TaskSetManager: Starting task 0.0 in stage 21.0 (TID 734) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:40 INFO TaskSetManager: Starting task 1.0 in stage 21.0 (TID 735) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:40 INFO TaskSetManager: Starting task 2.0 in stage 21.0 (TID 736) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:40 INFO TaskSetManager: Starting task 3.0 in stage 21.0 (TID 737) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:40 INFO TaskSetManager: Starting task 4.0 in stage 21.0 (TID 738) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:40 INFO TaskSetManager: Starting task 5.0 in stage 21.0 (TID 739) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:40 INFO TaskSetManager: Starting task 6.0 in stage 21.0 (TID 740) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:40 INFO TaskSetManager: Starting task 7.0 in stage 21.0 (TID 741) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:40 INFO Executor: Running task 1.0 in stage 21.0 (TID 735) +26/04/01 08:50:40 INFO Executor: Running task 2.0 in stage 21.0 (TID 736) +26/04/01 08:50:40 INFO Executor: Running task 7.0 in stage 21.0 (TID 741) +26/04/01 08:50:40 INFO Executor: Running task 0.0 in stage 21.0 (TID 734) +26/04/01 08:50:40 INFO Executor: Running task 6.0 in stage 21.0 (TID 740) +26/04/01 08:50:40 INFO Executor: Running task 3.0 in stage 21.0 (TID 737) +26/04/01 08:50:40 INFO Executor: Running task 4.0 in stage 21.0 (TID 738) +26/04/01 08:50:40 INFO Executor: Running task 5.0 in stage 21.0 (TID 739) +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.3 MiB) non-empty blocks including 200 (61.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.0 MiB) non-empty blocks including 200 (61.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.6 MiB) non-empty blocks including 200 (61.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (624.5 KiB) non-empty blocks including 8 (624.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (619.1 KiB) non-empty blocks including 8 (619.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (618.3 KiB) non-empty blocks including 8 (618.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (619.7 KiB) non-empty blocks including 8 (619.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (621.6 KiB) non-empty blocks including 8 (621.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (618.2 KiB) non-empty blocks including 8 (618.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.3 MiB) non-empty blocks including 200 (61.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.0 MiB) non-empty blocks including 200 (61.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.6 MiB) non-empty blocks including 200 (61.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (621.6 KiB) non-empty blocks including 8 (621.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (619.1 KiB) non-empty blocks including 8 (619.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (618.3 KiB) non-empty blocks including 8 (618.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (618.2 KiB) non-empty blocks including 8 (618.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (624.5 KiB) non-empty blocks including 8 (624.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (619.7 KiB) non-empty blocks including 8 (619.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:42 INFO BlockManagerInfo: Removed broadcast_23_piece0 on 10.0.0.133:59333 in memory (size: 11.0 KiB, free: 12.6 GiB) +26/04/01 08:50:50 INFO Executor: Finished task 1.0 in stage 21.0 (TID 735). 9594 bytes result sent to driver +26/04/01 08:50:50 INFO TaskSetManager: Starting task 8.0 in stage 21.0 (TID 742) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:50 INFO Executor: Running task 8.0 in stage 21.0 (TID 742) +26/04/01 08:50:50 INFO TaskSetManager: Finished task 1.0 in stage 21.0 (TID 735) in 9928 ms on 10.0.0.133 (executor driver) (1/14) +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (623.7 KiB) non-empty blocks including 8 (623.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (623.7 KiB) non-empty blocks including 8 (623.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:50 INFO Executor: Finished task 5.0 in stage 21.0 (TID 739). 9594 bytes result sent to driver +26/04/01 08:50:50 INFO TaskSetManager: Starting task 9.0 in stage 21.0 (TID 743) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:50 INFO TaskSetManager: Finished task 5.0 in stage 21.0 (TID 739) in 9984 ms on 10.0.0.133 (executor driver) (2/14) +26/04/01 08:50:50 INFO Executor: Running task 9.0 in stage 21.0 (TID 743) +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (623.6 KiB) non-empty blocks including 8 (623.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (623.6 KiB) non-empty blocks including 8 (623.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:50 INFO Executor: Finished task 7.0 in stage 21.0 (TID 741). 9594 bytes result sent to driver +26/04/01 08:50:50 INFO TaskSetManager: Starting task 10.0 in stage 21.0 (TID 744) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:50 INFO TaskSetManager: Finished task 7.0 in stage 21.0 (TID 741) in 10018 ms on 10.0.0.133 (executor driver) (3/14) +26/04/01 08:50:50 INFO Executor: Running task 10.0 in stage 21.0 (TID 744) +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.2 MiB) non-empty blocks including 200 (61.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (617.6 KiB) non-empty blocks including 8 (617.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.2 MiB) non-empty blocks including 200 (61.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (617.6 KiB) non-empty blocks including 8 (617.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:50 INFO Executor: Finished task 6.0 in stage 21.0 (TID 740). 9594 bytes result sent to driver +26/04/01 08:50:50 INFO TaskSetManager: Starting task 11.0 in stage 21.0 (TID 745) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:50 INFO Executor: Running task 11.0 in stage 21.0 (TID 745) +26/04/01 08:50:50 INFO TaskSetManager: Finished task 6.0 in stage 21.0 (TID 740) in 10044 ms on 10.0.0.133 (executor driver) (4/14) +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (622.2 KiB) non-empty blocks including 8 (622.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (622.2 KiB) non-empty blocks including 8 (622.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:50 INFO Executor: Finished task 4.0 in stage 21.0 (TID 738). 9594 bytes result sent to driver +26/04/01 08:50:50 INFO TaskSetManager: Starting task 12.0 in stage 21.0 (TID 746) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:50 INFO TaskSetManager: Finished task 4.0 in stage 21.0 (TID 738) in 10074 ms on 10.0.0.133 (executor driver) (5/14) +26/04/01 08:50:50 INFO Executor: Running task 12.0 in stage 21.0 (TID 746) +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (622.0 KiB) non-empty blocks including 8 (622.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (622.0 KiB) non-empty blocks including 8 (622.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:50 INFO Executor: Finished task 0.0 in stage 21.0 (TID 734). 9594 bytes result sent to driver +26/04/01 08:50:50 INFO TaskSetManager: Starting task 13.0 in stage 21.0 (TID 747) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:50 INFO TaskSetManager: Finished task 0.0 in stage 21.0 (TID 734) in 10122 ms on 10.0.0.133 (executor driver) (6/14) +26/04/01 08:50:50 INFO Executor: Running task 13.0 in stage 21.0 (TID 747) +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (20.4 MiB) non-empty blocks including 200 (20.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (207.3 KiB) non-empty blocks including 8 (207.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (20.4 MiB) non-empty blocks including 200 (20.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (207.3 KiB) non-empty blocks including 8 (207.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:50 INFO Executor: Finished task 2.0 in stage 21.0 (TID 736). 9594 bytes result sent to driver +26/04/01 08:50:50 INFO TaskSetManager: Finished task 2.0 in stage 21.0 (TID 736) in 10152 ms on 10.0.0.133 (executor driver) (7/14) +26/04/01 08:50:50 INFO Executor: Finished task 3.0 in stage 21.0 (TID 737). 9594 bytes result sent to driver +26/04/01 08:50:50 INFO TaskSetManager: Finished task 3.0 in stage 21.0 (TID 737) in 10181 ms on 10.0.0.133 (executor driver) (8/14) +26/04/01 08:50:53 INFO Executor: Finished task 13.0 in stage 21.0 (TID 747). 9594 bytes result sent to driver +26/04/01 08:50:53 INFO TaskSetManager: Finished task 13.0 in stage 21.0 (TID 747) in 2971 ms on 10.0.0.133 (executor driver) (9/14) +26/04/01 08:50:59 INFO Executor: Finished task 10.0 in stage 21.0 (TID 744). 9594 bytes result sent to driver +26/04/01 08:50:59 INFO TaskSetManager: Finished task 10.0 in stage 21.0 (TID 744) in 9232 ms on 10.0.0.133 (executor driver) (10/14) +26/04/01 08:50:59 INFO Executor: Finished task 12.0 in stage 21.0 (TID 746). 9594 bytes result sent to driver +26/04/01 08:50:59 INFO TaskSetManager: Finished task 12.0 in stage 21.0 (TID 746) in 9250 ms on 10.0.0.133 (executor driver) (11/14) +26/04/01 08:50:59 INFO Executor: Finished task 9.0 in stage 21.0 (TID 743). 9594 bytes result sent to driver +26/04/01 08:50:59 INFO TaskSetManager: Finished task 9.0 in stage 21.0 (TID 743) in 9345 ms on 10.0.0.133 (executor driver) (12/14) +26/04/01 08:50:59 INFO Executor: Finished task 8.0 in stage 21.0 (TID 742). 9594 bytes result sent to driver +26/04/01 08:50:59 INFO TaskSetManager: Finished task 8.0 in stage 21.0 (TID 742) in 9403 ms on 10.0.0.133 (executor driver) (13/14) +26/04/01 08:50:59 INFO Executor: Finished task 11.0 in stage 21.0 (TID 745). 9594 bytes result sent to driver +26/04/01 08:50:59 INFO TaskSetManager: Finished task 11.0 in stage 21.0 (TID 745) in 9323 ms on 10.0.0.133 (executor driver) (14/14) +26/04/01 08:50:59 INFO TaskSchedulerImpl: Removed TaskSet 21.0, whose tasks have all completed, from pool +26/04/01 08:50:59 INFO DAGScheduler: ShuffleMapStage 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 19.369 s +26/04/01 08:50:59 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:50:59 INFO DAGScheduler: running: Set() +26/04/01 08:50:59 INFO DAGScheduler: waiting: Set() +26/04/01 08:50:59 INFO DAGScheduler: failed: Set() +26/04/01 08:50:59 INFO ShufflePartitionsUtil: For shuffle(6, 3), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 08:50:59 INFO DAGScheduler: Registering RDD 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 7 +26/04/01 08:50:59 INFO DAGScheduler: Got map stage job 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 29 output partitions +26/04/01 08:50:59 INFO DAGScheduler: Final stage: ShuffleMapStage 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:50:59 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 25, ShuffleMapStage 26) +26/04/01 08:50:59 INFO DAGScheduler: Missing parents: List() +26/04/01 08:50:59 INFO DAGScheduler: Submitting ShuffleMapStage 27 (MapPartitionsRDD[47] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:50:59 INFO MemoryStore: Block broadcast_25 stored as values in memory (estimated size 28.9 KiB, free 12.6 GiB) +26/04/01 08:50:59 INFO MemoryStore: Block broadcast_25_piece0 stored as bytes in memory (estimated size 10.3 KiB, free 12.6 GiB) +26/04/01 08:50:59 INFO BlockManagerInfo: Added broadcast_25_piece0 in memory on 10.0.0.133:59333 (size: 10.3 KiB, free: 12.6 GiB) +26/04/01 08:50:59 INFO SparkContext: Created broadcast 25 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:50:59 INFO DAGScheduler: Submitting 29 missing tasks from ShuffleMapStage 27 (MapPartitionsRDD[47] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:50:59 INFO TaskSchedulerImpl: Adding task set 27.0 with 29 tasks resource profile 0 +26/04/01 08:50:59 INFO TaskSetManager: Starting task 0.0 in stage 27.0 (TID 748) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:59 INFO TaskSetManager: Starting task 1.0 in stage 27.0 (TID 749) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:59 INFO TaskSetManager: Starting task 2.0 in stage 27.0 (TID 750) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:59 INFO TaskSetManager: Starting task 3.0 in stage 27.0 (TID 751) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:59 INFO TaskSetManager: Starting task 4.0 in stage 27.0 (TID 752) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:59 INFO TaskSetManager: Starting task 5.0 in stage 27.0 (TID 753) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:59 INFO TaskSetManager: Starting task 6.0 in stage 27.0 (TID 754) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:59 INFO TaskSetManager: Starting task 7.0 in stage 27.0 (TID 755) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9428 bytes) +26/04/01 08:50:59 INFO Executor: Running task 2.0 in stage 27.0 (TID 750) +26/04/01 08:50:59 INFO Executor: Running task 0.0 in stage 27.0 (TID 748) +26/04/01 08:50:59 INFO Executor: Running task 5.0 in stage 27.0 (TID 753) +26/04/01 08:50:59 INFO Executor: Running task 6.0 in stage 27.0 (TID 754) +26/04/01 08:50:59 INFO Executor: Running task 7.0 in stage 27.0 (TID 755) +26/04/01 08:50:59 INFO Executor: Running task 4.0 in stage 27.0 (TID 752) +26/04/01 08:50:59 INFO Executor: Running task 3.0 in stage 27.0 (TID 751) +26/04/01 08:50:59 INFO Executor: Running task 1.0 in stage 27.0 (TID 749) +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.6 MiB) non-empty blocks including 14 (23.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.2 MiB) non-empty blocks including 14 (23.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.6 MiB) non-empty blocks including 14 (23.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.2 MiB) non-empty blocks including 14 (23.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:09 INFO Executor: Finished task 4.0 in stage 27.0 (TID 752). 9594 bytes result sent to driver +26/04/01 08:51:09 INFO TaskSetManager: Starting task 8.0 in stage 27.0 (TID 756) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:09 INFO TaskSetManager: Finished task 4.0 in stage 27.0 (TID 752) in 9377 ms on 10.0.0.133 (executor driver) (1/29) +26/04/01 08:51:09 INFO Executor: Running task 8.0 in stage 27.0 (TID 756) +26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:10 INFO Executor: Finished task 6.0 in stage 27.0 (TID 754). 9594 bytes result sent to driver +26/04/01 08:51:10 INFO TaskSetManager: Starting task 9.0 in stage 27.0 (TID 757) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:10 INFO TaskSetManager: Finished task 6.0 in stage 27.0 (TID 754) in 10383 ms on 10.0.0.133 (executor driver) (2/29) +26/04/01 08:51:10 INFO Executor: Running task 9.0 in stage 27.0 (TID 757) +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:10 INFO Executor: Finished task 3.0 in stage 27.0 (TID 751). 9594 bytes result sent to driver +26/04/01 08:51:10 INFO TaskSetManager: Starting task 10.0 in stage 27.0 (TID 758) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:10 INFO TaskSetManager: Finished task 3.0 in stage 27.0 (TID 751) in 10414 ms on 10.0.0.133 (executor driver) (3/29) +26/04/01 08:51:10 INFO Executor: Running task 10.0 in stage 27.0 (TID 758) +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:10 INFO Executor: Finished task 5.0 in stage 27.0 (TID 753). 9594 bytes result sent to driver +26/04/01 08:51:10 INFO TaskSetManager: Starting task 11.0 in stage 27.0 (TID 759) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:10 INFO TaskSetManager: Finished task 5.0 in stage 27.0 (TID 753) in 10560 ms on 10.0.0.133 (executor driver) (4/29) +26/04/01 08:51:10 INFO Executor: Running task 11.0 in stage 27.0 (TID 759) +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:10 INFO Executor: Finished task 2.0 in stage 27.0 (TID 750). 9637 bytes result sent to driver +26/04/01 08:51:10 INFO TaskSetManager: Starting task 12.0 in stage 27.0 (TID 760) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:10 INFO TaskSetManager: Finished task 2.0 in stage 27.0 (TID 750) in 10588 ms on 10.0.0.133 (executor driver) (5/29) +26/04/01 08:51:10 INFO Executor: Running task 12.0 in stage 27.0 (TID 760) +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:10 INFO Executor: Finished task 1.0 in stage 27.0 (TID 749). 9637 bytes result sent to driver +26/04/01 08:51:10 INFO TaskSetManager: Starting task 13.0 in stage 27.0 (TID 761) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:10 INFO Executor: Running task 13.0 in stage 27.0 (TID 761) +26/04/01 08:51:10 INFO TaskSetManager: Finished task 1.0 in stage 27.0 (TID 749) in 10786 ms on 10.0.0.133 (executor driver) (6/29) +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:10 INFO Executor: Finished task 0.0 in stage 27.0 (TID 748). 9594 bytes result sent to driver +26/04/01 08:51:10 INFO TaskSetManager: Starting task 14.0 in stage 27.0 (TID 762) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:10 INFO TaskSetManager: Finished task 0.0 in stage 27.0 (TID 748) in 10867 ms on 10.0.0.133 (executor driver) (7/29) +26/04/01 08:51:10 INFO Executor: Running task 14.0 in stage 27.0 (TID 762) +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:10 INFO Executor: Finished task 7.0 in stage 27.0 (TID 755). 9594 bytes result sent to driver +26/04/01 08:51:10 INFO TaskSetManager: Starting task 15.0 in stage 27.0 (TID 763) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:10 INFO TaskSetManager: Finished task 7.0 in stage 27.0 (TID 755) in 11015 ms on 10.0.0.133 (executor driver) (8/29) +26/04/01 08:51:10 INFO Executor: Running task 15.0 in stage 27.0 (TID 763) +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:17 INFO Executor: Finished task 8.0 in stage 27.0 (TID 756). 9594 bytes result sent to driver +26/04/01 08:51:17 INFO TaskSetManager: Starting task 16.0 in stage 27.0 (TID 764) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:17 INFO TaskSetManager: Finished task 8.0 in stage 27.0 (TID 756) in 8585 ms on 10.0.0.133 (executor driver) (9/29) +26/04/01 08:51:17 INFO Executor: Running task 16.0 in stage 27.0 (TID 764) +26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:19 INFO Executor: Finished task 9.0 in stage 27.0 (TID 757). 9594 bytes result sent to driver +26/04/01 08:51:19 INFO TaskSetManager: Starting task 17.0 in stage 27.0 (TID 765) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:19 INFO TaskSetManager: Finished task 9.0 in stage 27.0 (TID 757) in 8956 ms on 10.0.0.133 (executor driver) (10/29) +26/04/01 08:51:19 INFO Executor: Running task 17.0 in stage 27.0 (TID 765) +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:19 INFO Executor: Finished task 11.0 in stage 27.0 (TID 759). 9594 bytes result sent to driver +26/04/01 08:51:19 INFO TaskSetManager: Starting task 18.0 in stage 27.0 (TID 766) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:19 INFO TaskSetManager: Finished task 11.0 in stage 27.0 (TID 759) in 9564 ms on 10.0.0.133 (executor driver) (11/29) +26/04/01 08:51:19 INFO Executor: Running task 18.0 in stage 27.0 (TID 766) +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:19 INFO Executor: Finished task 10.0 in stage 27.0 (TID 758). 9594 bytes result sent to driver +26/04/01 08:51:19 INFO TaskSetManager: Starting task 19.0 in stage 27.0 (TID 767) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:19 INFO Executor: Running task 19.0 in stage 27.0 (TID 767) +26/04/01 08:51:19 INFO TaskSetManager: Finished task 10.0 in stage 27.0 (TID 758) in 9744 ms on 10.0.0.133 (executor driver) (12/29) +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:20 INFO Executor: Finished task 12.0 in stage 27.0 (TID 760). 9594 bytes result sent to driver +26/04/01 08:51:20 INFO TaskSetManager: Starting task 20.0 in stage 27.0 (TID 768) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:20 INFO Executor: Running task 20.0 in stage 27.0 (TID 768) +26/04/01 08:51:20 INFO TaskSetManager: Finished task 12.0 in stage 27.0 (TID 760) in 9795 ms on 10.0.0.133 (executor driver) (13/29) +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:20 INFO Executor: Finished task 15.0 in stage 27.0 (TID 763). 9594 bytes result sent to driver +26/04/01 08:51:20 INFO TaskSetManager: Starting task 21.0 in stage 27.0 (TID 769) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:20 INFO Executor: Running task 21.0 in stage 27.0 (TID 769) +26/04/01 08:51:20 INFO TaskSetManager: Finished task 15.0 in stage 27.0 (TID 763) in 9408 ms on 10.0.0.133 (executor driver) (14/29) +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:20 INFO Executor: Finished task 14.0 in stage 27.0 (TID 762). 9594 bytes result sent to driver +26/04/01 08:51:20 INFO TaskSetManager: Starting task 22.0 in stage 27.0 (TID 770) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:20 INFO Executor: Running task 22.0 in stage 27.0 (TID 770) +26/04/01 08:51:20 INFO TaskSetManager: Finished task 14.0 in stage 27.0 (TID 762) in 9615 ms on 10.0.0.133 (executor driver) (15/29) +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:20 INFO Executor: Finished task 13.0 in stage 27.0 (TID 761). 9594 bytes result sent to driver +26/04/01 08:51:20 INFO TaskSetManager: Starting task 23.0 in stage 27.0 (TID 771) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:20 INFO Executor: Running task 23.0 in stage 27.0 (TID 771) +26/04/01 08:51:20 INFO TaskSetManager: Finished task 13.0 in stage 27.0 (TID 761) in 9761 ms on 10.0.0.133 (executor driver) (16/29) +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:26 INFO Executor: Finished task 16.0 in stage 27.0 (TID 764). 9594 bytes result sent to driver +26/04/01 08:51:26 INFO TaskSetManager: Starting task 24.0 in stage 27.0 (TID 772) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:26 INFO Executor: Running task 24.0 in stage 27.0 (TID 772) +26/04/01 08:51:26 INFO TaskSetManager: Finished task 16.0 in stage 27.0 (TID 764) in 8810 ms on 10.0.0.133 (executor driver) (17/29) +26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:27 INFO Executor: Finished task 17.0 in stage 27.0 (TID 765). 9594 bytes result sent to driver +26/04/01 08:51:27 INFO TaskSetManager: Starting task 25.0 in stage 27.0 (TID 773) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:27 INFO TaskSetManager: Finished task 17.0 in stage 27.0 (TID 765) in 8907 ms on 10.0.0.133 (executor driver) (18/29) +26/04/01 08:51:27 INFO Executor: Running task 25.0 in stage 27.0 (TID 773) +26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:28 INFO Executor: Finished task 19.0 in stage 27.0 (TID 767). 9594 bytes result sent to driver +26/04/01 08:51:28 INFO TaskSetManager: Starting task 26.0 in stage 27.0 (TID 774) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:28 INFO TaskSetManager: Finished task 19.0 in stage 27.0 (TID 767) in 9161 ms on 10.0.0.133 (executor driver) (19/29) +26/04/01 08:51:28 INFO Executor: Running task 26.0 in stage 27.0 (TID 774) +26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:29 INFO Executor: Finished task 18.0 in stage 27.0 (TID 766). 9594 bytes result sent to driver +26/04/01 08:51:29 INFO TaskSetManager: Starting task 27.0 in stage 27.0 (TID 775) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:29 INFO Executor: Running task 27.0 in stage 27.0 (TID 775) +26/04/01 08:51:29 INFO TaskSetManager: Finished task 18.0 in stage 27.0 (TID 766) in 9739 ms on 10.0.0.133 (executor driver) (20/29) +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:29 INFO Executor: Finished task 22.0 in stage 27.0 (TID 770). 9594 bytes result sent to driver +26/04/01 08:51:29 INFO TaskSetManager: Starting task 28.0 in stage 27.0 (TID 776) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9428 bytes) +26/04/01 08:51:29 INFO Executor: Running task 28.0 in stage 27.0 (TID 776) +26/04/01 08:51:29 INFO TaskSetManager: Finished task 22.0 in stage 27.0 (TID 770) in 9628 ms on 10.0.0.133 (executor driver) (21/29) +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 14 (13.4 MiB) non-empty blocks including 14 (13.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 37 (21.8 MiB) non-empty blocks including 37 (21.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 14 (13.4 MiB) non-empty blocks including 14 (13.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 37 (21.8 MiB) non-empty blocks including 37 (21.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:29 INFO Executor: Finished task 20.0 in stage 27.0 (TID 768). 9594 bytes result sent to driver +26/04/01 08:51:29 INFO TaskSetManager: Finished task 20.0 in stage 27.0 (TID 768) in 9810 ms on 10.0.0.133 (executor driver) (22/29) +26/04/01 08:51:29 INFO Executor: Finished task 21.0 in stage 27.0 (TID 769). 9594 bytes result sent to driver +26/04/01 08:51:29 INFO TaskSetManager: Finished task 21.0 in stage 27.0 (TID 769) in 9799 ms on 10.0.0.133 (executor driver) (23/29) +26/04/01 08:51:29 INFO Executor: Finished task 23.0 in stage 27.0 (TID 771). 9594 bytes result sent to driver +26/04/01 08:51:29 INFO TaskSetManager: Finished task 23.0 in stage 27.0 (TID 771) in 9744 ms on 10.0.0.133 (executor driver) (24/29) +26/04/01 08:51:34 INFO Executor: Finished task 28.0 in stage 27.0 (TID 776). 9594 bytes result sent to driver +26/04/01 08:51:34 INFO TaskSetManager: Finished task 28.0 in stage 27.0 (TID 776) in 4896 ms on 10.0.0.133 (executor driver) (25/29) +26/04/01 08:51:35 INFO Executor: Finished task 24.0 in stage 27.0 (TID 772). 9594 bytes result sent to driver +26/04/01 08:51:35 INFO TaskSetManager: Finished task 24.0 in stage 27.0 (TID 772) in 8825 ms on 10.0.0.133 (executor driver) (26/29) +26/04/01 08:51:36 INFO Executor: Finished task 25.0 in stage 27.0 (TID 773). 9594 bytes result sent to driver +26/04/01 08:51:36 INFO TaskSetManager: Finished task 25.0 in stage 27.0 (TID 773) in 8790 ms on 10.0.0.133 (executor driver) (27/29) +26/04/01 08:51:37 INFO Executor: Finished task 26.0 in stage 27.0 (TID 774). 9594 bytes result sent to driver +26/04/01 08:51:37 INFO TaskSetManager: Finished task 26.0 in stage 27.0 (TID 774) in 8755 ms on 10.0.0.133 (executor driver) (28/29) +26/04/01 08:51:38 INFO Executor: Finished task 27.0 in stage 27.0 (TID 775). 9594 bytes result sent to driver +26/04/01 08:51:38 INFO TaskSetManager: Finished task 27.0 in stage 27.0 (TID 775) in 8708 ms on 10.0.0.133 (executor driver) (29/29) +26/04/01 08:51:38 INFO TaskSchedulerImpl: Removed TaskSet 27.0, whose tasks have all completed, from pool +26/04/01 08:51:38 INFO DAGScheduler: ShuffleMapStage 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 38.572 s +26/04/01 08:51:38 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:51:38 INFO DAGScheduler: running: Set() +26/04/01 08:51:38 INFO DAGScheduler: waiting: Set() +26/04/01 08:51:38 INFO DAGScheduler: failed: Set() +26/04/01 08:51:38 INFO ShufflePartitionsUtil: For shuffle(7, 4), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 08:51:38 INFO DAGScheduler: Registering RDD 52 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 8 +26/04/01 08:51:38 INFO DAGScheduler: Got map stage job 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 34 output partitions +26/04/01 08:51:38 INFO DAGScheduler: Final stage: ShuffleMapStage 35 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:51:38 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 33, ShuffleMapStage 34) +26/04/01 08:51:38 INFO DAGScheduler: Missing parents: List() +26/04/01 08:51:38 INFO DAGScheduler: Submitting ShuffleMapStage 35 (MapPartitionsRDD[52] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:51:38 INFO MemoryStore: Block broadcast_26 stored as values in memory (estimated size 44.4 KiB, free 12.6 GiB) +26/04/01 08:51:38 INFO MemoryStore: Block broadcast_26_piece0 stored as bytes in memory (estimated size 12.6 KiB, free 12.6 GiB) +26/04/01 08:51:38 INFO BlockManagerInfo: Added broadcast_26_piece0 in memory on 10.0.0.133:59333 (size: 12.6 KiB, free: 12.6 GiB) +26/04/01 08:51:38 INFO SparkContext: Created broadcast 26 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:51:38 INFO DAGScheduler: Submitting 34 missing tasks from ShuffleMapStage 35 (MapPartitionsRDD[52] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:51:38 INFO TaskSchedulerImpl: Adding task set 35.0 with 34 tasks resource profile 0 +26/04/01 08:51:38 INFO TaskSetManager: Starting task 0.0 in stage 35.0 (TID 777) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:38 INFO TaskSetManager: Starting task 1.0 in stage 35.0 (TID 778) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:38 INFO TaskSetManager: Starting task 2.0 in stage 35.0 (TID 779) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:38 INFO TaskSetManager: Starting task 3.0 in stage 35.0 (TID 780) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:38 INFO TaskSetManager: Starting task 4.0 in stage 35.0 (TID 781) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:38 INFO TaskSetManager: Starting task 5.0 in stage 35.0 (TID 782) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:38 INFO TaskSetManager: Starting task 6.0 in stage 35.0 (TID 783) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:38 INFO TaskSetManager: Starting task 7.0 in stage 35.0 (TID 784) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:38 INFO Executor: Running task 1.0 in stage 35.0 (TID 778) +26/04/01 08:51:38 INFO Executor: Running task 0.0 in stage 35.0 (TID 777) +26/04/01 08:51:38 INFO Executor: Running task 7.0 in stage 35.0 (TID 784) +26/04/01 08:51:38 INFO Executor: Running task 4.0 in stage 35.0 (TID 781) +26/04/01 08:51:38 INFO Executor: Running task 3.0 in stage 35.0 (TID 780) +26/04/01 08:51:38 INFO Executor: Running task 2.0 in stage 35.0 (TID 779) +26/04/01 08:51:38 INFO Executor: Running task 6.0 in stage 35.0 (TID 783) +26/04/01 08:51:38 INFO Executor: Running task 5.0 in stage 35.0 (TID 782) +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO Executor: Finished task 3.0 in stage 35.0 (TID 780). 10455 bytes result sent to driver +26/04/01 08:51:45 INFO TaskSetManager: Starting task 8.0 in stage 35.0 (TID 785) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:45 INFO Executor: Running task 8.0 in stage 35.0 (TID 785) +26/04/01 08:51:45 INFO TaskSetManager: Finished task 3.0 in stage 35.0 (TID 780) in 7062 ms on 10.0.0.133 (executor driver) (1/34) +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO Executor: Finished task 2.0 in stage 35.0 (TID 779). 10455 bytes result sent to driver +26/04/01 08:51:45 INFO TaskSetManager: Starting task 9.0 in stage 35.0 (TID 786) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:45 INFO TaskSetManager: Finished task 2.0 in stage 35.0 (TID 779) in 7288 ms on 10.0.0.133 (executor driver) (2/34) +26/04/01 08:51:45 INFO Executor: Running task 9.0 in stage 35.0 (TID 786) +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO Executor: Finished task 4.0 in stage 35.0 (TID 781). 10455 bytes result sent to driver +26/04/01 08:51:45 INFO TaskSetManager: Starting task 10.0 in stage 35.0 (TID 787) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:45 INFO TaskSetManager: Finished task 4.0 in stage 35.0 (TID 781) in 7324 ms on 10.0.0.133 (executor driver) (3/34) +26/04/01 08:51:45 INFO Executor: Running task 10.0 in stage 35.0 (TID 787) +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO Executor: Finished task 1.0 in stage 35.0 (TID 778). 10455 bytes result sent to driver +26/04/01 08:51:45 INFO TaskSetManager: Starting task 11.0 in stage 35.0 (TID 788) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:45 INFO TaskSetManager: Finished task 1.0 in stage 35.0 (TID 778) in 7358 ms on 10.0.0.133 (executor driver) (4/34) +26/04/01 08:51:45 INFO Executor: Running task 11.0 in stage 35.0 (TID 788) +26/04/01 08:51:45 INFO Executor: Finished task 6.0 in stage 35.0 (TID 783). 10455 bytes result sent to driver +26/04/01 08:51:45 INFO TaskSetManager: Starting task 12.0 in stage 35.0 (TID 789) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:45 INFO Executor: Running task 12.0 in stage 35.0 (TID 789) +26/04/01 08:51:45 INFO TaskSetManager: Finished task 6.0 in stage 35.0 (TID 783) in 7358 ms on 10.0.0.133 (executor driver) (5/34) +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO Executor: Finished task 7.0 in stage 35.0 (TID 784). 10455 bytes result sent to driver +26/04/01 08:51:45 INFO TaskSetManager: Starting task 13.0 in stage 35.0 (TID 790) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:45 INFO Executor: Running task 13.0 in stage 35.0 (TID 790) +26/04/01 08:51:45 INFO TaskSetManager: Finished task 7.0 in stage 35.0 (TID 784) in 7391 ms on 10.0.0.133 (executor driver) (6/34) +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO Executor: Finished task 0.0 in stage 35.0 (TID 777). 10455 bytes result sent to driver +26/04/01 08:51:45 INFO TaskSetManager: Starting task 14.0 in stage 35.0 (TID 791) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:45 INFO TaskSetManager: Finished task 0.0 in stage 35.0 (TID 777) in 7413 ms on 10.0.0.133 (executor driver) (7/34) +26/04/01 08:51:45 INFO Executor: Running task 14.0 in stage 35.0 (TID 791) +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO Executor: Finished task 5.0 in stage 35.0 (TID 782). 10455 bytes result sent to driver +26/04/01 08:51:45 INFO TaskSetManager: Starting task 15.0 in stage 35.0 (TID 792) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:45 INFO Executor: Running task 15.0 in stage 35.0 (TID 792) +26/04/01 08:51:45 INFO TaskSetManager: Finished task 5.0 in stage 35.0 (TID 782) in 7422 ms on 10.0.0.133 (executor driver) (8/34) +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO Executor: Finished task 8.0 in stage 35.0 (TID 785). 10455 bytes result sent to driver +26/04/01 08:51:52 INFO TaskSetManager: Starting task 16.0 in stage 35.0 (TID 793) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:52 INFO Executor: Running task 16.0 in stage 35.0 (TID 793) +26/04/01 08:51:52 INFO TaskSetManager: Finished task 8.0 in stage 35.0 (TID 785) in 6908 ms on 10.0.0.133 (executor driver) (9/34) +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO Executor: Finished task 9.0 in stage 35.0 (TID 786). 10455 bytes result sent to driver +26/04/01 08:51:52 INFO TaskSetManager: Starting task 17.0 in stage 35.0 (TID 794) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:52 INFO Executor: Running task 17.0 in stage 35.0 (TID 794) +26/04/01 08:51:52 INFO TaskSetManager: Finished task 9.0 in stage 35.0 (TID 786) in 6936 ms on 10.0.0.133 (executor driver) (10/34) +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO Executor: Finished task 10.0 in stage 35.0 (TID 787). 10455 bytes result sent to driver +26/04/01 08:51:52 INFO TaskSetManager: Starting task 18.0 in stage 35.0 (TID 795) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:52 INFO TaskSetManager: Finished task 10.0 in stage 35.0 (TID 787) in 6947 ms on 10.0.0.133 (executor driver) (11/34) +26/04/01 08:51:52 INFO Executor: Running task 18.0 in stage 35.0 (TID 795) +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO Executor: Finished task 12.0 in stage 35.0 (TID 789). 10455 bytes result sent to driver +26/04/01 08:51:52 INFO TaskSetManager: Starting task 19.0 in stage 35.0 (TID 796) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:52 INFO TaskSetManager: Finished task 12.0 in stage 35.0 (TID 789) in 7067 ms on 10.0.0.133 (executor driver) (12/34) +26/04/01 08:51:52 INFO Executor: Running task 19.0 in stage 35.0 (TID 796) +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO Executor: Finished task 11.0 in stage 35.0 (TID 788). 10455 bytes result sent to driver +26/04/01 08:51:52 INFO TaskSetManager: Starting task 20.0 in stage 35.0 (TID 797) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:52 INFO Executor: Running task 20.0 in stage 35.0 (TID 797) +26/04/01 08:51:52 INFO TaskSetManager: Finished task 11.0 in stage 35.0 (TID 788) in 7077 ms on 10.0.0.133 (executor driver) (13/34) +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO Executor: Finished task 13.0 in stage 35.0 (TID 790). 10455 bytes result sent to driver +26/04/01 08:51:52 INFO TaskSetManager: Starting task 21.0 in stage 35.0 (TID 798) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:52 INFO TaskSetManager: Finished task 13.0 in stage 35.0 (TID 790) in 7099 ms on 10.0.0.133 (executor driver) (14/34) +26/04/01 08:51:52 INFO Executor: Running task 21.0 in stage 35.0 (TID 798) +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO Executor: Finished task 14.0 in stage 35.0 (TID 791). 10455 bytes result sent to driver +26/04/01 08:51:52 INFO TaskSetManager: Starting task 22.0 in stage 35.0 (TID 799) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:52 INFO TaskSetManager: Finished task 14.0 in stage 35.0 (TID 791) in 7087 ms on 10.0.0.133 (executor driver) (15/34) +26/04/01 08:51:52 INFO Executor: Running task 22.0 in stage 35.0 (TID 799) +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO Executor: Finished task 15.0 in stage 35.0 (TID 792). 10455 bytes result sent to driver +26/04/01 08:51:52 INFO TaskSetManager: Starting task 23.0 in stage 35.0 (TID 800) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:52 INFO TaskSetManager: Finished task 15.0 in stage 35.0 (TID 792) in 7079 ms on 10.0.0.133 (executor driver) (16/34) +26/04/01 08:51:52 INFO Executor: Running task 23.0 in stage 35.0 (TID 800) +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO Executor: Finished task 16.0 in stage 35.0 (TID 793). 9724 bytes result sent to driver +26/04/01 08:51:59 INFO TaskSetManager: Starting task 24.0 in stage 35.0 (TID 801) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:59 INFO Executor: Running task 24.0 in stage 35.0 (TID 801) +26/04/01 08:51:59 INFO TaskSetManager: Finished task 16.0 in stage 35.0 (TID 793) in 6982 ms on 10.0.0.133 (executor driver) (17/34) +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO Executor: Finished task 17.0 in stage 35.0 (TID 794). 9724 bytes result sent to driver +26/04/01 08:51:59 INFO TaskSetManager: Starting task 25.0 in stage 35.0 (TID 802) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:59 INFO TaskSetManager: Finished task 17.0 in stage 35.0 (TID 794) in 6779 ms on 10.0.0.133 (executor driver) (18/34) +26/04/01 08:51:59 INFO Executor: Running task 25.0 in stage 35.0 (TID 802) +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO Executor: Finished task 18.0 in stage 35.0 (TID 795). 9724 bytes result sent to driver +26/04/01 08:51:59 INFO TaskSetManager: Starting task 26.0 in stage 35.0 (TID 803) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:59 INFO TaskSetManager: Finished task 18.0 in stage 35.0 (TID 795) in 6878 ms on 10.0.0.133 (executor driver) (19/34) +26/04/01 08:51:59 INFO Executor: Running task 26.0 in stage 35.0 (TID 803) +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO Executor: Finished task 20.0 in stage 35.0 (TID 797). 9724 bytes result sent to driver +26/04/01 08:51:59 INFO TaskSetManager: Starting task 27.0 in stage 35.0 (TID 804) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:59 INFO TaskSetManager: Finished task 20.0 in stage 35.0 (TID 797) in 6924 ms on 10.0.0.133 (executor driver) (20/34) +26/04/01 08:51:59 INFO Executor: Running task 27.0 in stage 35.0 (TID 804) +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO Executor: Finished task 19.0 in stage 35.0 (TID 796). 9724 bytes result sent to driver +26/04/01 08:51:59 INFO TaskSetManager: Starting task 28.0 in stage 35.0 (TID 805) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:59 INFO TaskSetManager: Finished task 19.0 in stage 35.0 (TID 796) in 6955 ms on 10.0.0.133 (executor driver) (21/34) +26/04/01 08:51:59 INFO Executor: Running task 28.0 in stage 35.0 (TID 805) +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO Executor: Finished task 22.0 in stage 35.0 (TID 799). 9724 bytes result sent to driver +26/04/01 08:51:59 INFO TaskSetManager: Starting task 29.0 in stage 35.0 (TID 806) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:59 INFO Executor: Running task 29.0 in stage 35.0 (TID 806) +26/04/01 08:51:59 INFO TaskSetManager: Finished task 22.0 in stage 35.0 (TID 799) in 6959 ms on 10.0.0.133 (executor driver) (22/34) +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO Executor: Finished task 21.0 in stage 35.0 (TID 798). 9724 bytes result sent to driver +26/04/01 08:51:59 INFO TaskSetManager: Starting task 30.0 in stage 35.0 (TID 807) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:59 INFO TaskSetManager: Finished task 21.0 in stage 35.0 (TID 798) in 6993 ms on 10.0.0.133 (executor driver) (23/34) +26/04/01 08:51:59 INFO Executor: Running task 30.0 in stage 35.0 (TID 807) +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO Executor: Finished task 23.0 in stage 35.0 (TID 800). 9724 bytes result sent to driver +26/04/01 08:51:59 INFO TaskSetManager: Starting task 31.0 in stage 35.0 (TID 808) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9644 bytes) +26/04/01 08:51:59 INFO Executor: Running task 31.0 in stage 35.0 (TID 808) +26/04/01 08:51:59 INFO TaskSetManager: Finished task 23.0 in stage 35.0 (TID 800) in 7043 ms on 10.0.0.133 (executor driver) (24/34) +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:52:06 INFO Executor: Finished task 24.0 in stage 35.0 (TID 801). 10455 bytes result sent to driver +26/04/01 08:52:06 INFO TaskSetManager: Starting task 32.0 in stage 35.0 (TID 809) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9644 bytes) +26/04/01 08:52:06 INFO TaskSetManager: Finished task 24.0 in stage 35.0 (TID 801) in 6883 ms on 10.0.0.133 (executor driver) (25/34) +26/04/01 08:52:06 INFO Executor: Running task 32.0 in stage 35.0 (TID 809) +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:52:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:52:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:52:06 INFO Executor: Finished task 25.0 in stage 35.0 (TID 802). 10455 bytes result sent to driver +26/04/01 08:52:06 INFO TaskSetManager: Starting task 33.0 in stage 35.0 (TID 810) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9644 bytes) +26/04/01 08:52:06 INFO Executor: Running task 33.0 in stage 35.0 (TID 810) +26/04/01 08:52:06 INFO TaskSetManager: Finished task 25.0 in stage 35.0 (TID 802) in 6981 ms on 10.0.0.133 (executor driver) (26/34) +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 29 (8.1 MiB) non-empty blocks including 29 (8.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 64 (12.0 MiB) non-empty blocks including 64 (12.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 29 (8.1 MiB) non-empty blocks including 29 (8.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 64 (12.0 MiB) non-empty blocks including 64 (12.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:52:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:52:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:52:06 INFO Executor: Finished task 26.0 in stage 35.0 (TID 803). 10455 bytes result sent to driver +26/04/01 08:52:06 INFO TaskSetManager: Finished task 26.0 in stage 35.0 (TID 803) in 6918 ms on 10.0.0.133 (executor driver) (27/34) +26/04/01 08:52:06 INFO Executor: Finished task 27.0 in stage 35.0 (TID 804). 10455 bytes result sent to driver +26/04/01 08:52:06 INFO TaskSetManager: Finished task 27.0 in stage 35.0 (TID 804) in 6927 ms on 10.0.0.133 (executor driver) (28/34) +26/04/01 08:52:06 INFO Executor: Finished task 31.0 in stage 35.0 (TID 808). 10455 bytes result sent to driver +26/04/01 08:52:06 INFO TaskSetManager: Finished task 31.0 in stage 35.0 (TID 808) in 6776 ms on 10.0.0.133 (executor driver) (29/34) +26/04/01 08:52:06 INFO Executor: Finished task 29.0 in stage 35.0 (TID 806). 10455 bytes result sent to driver +26/04/01 08:52:06 INFO TaskSetManager: Finished task 29.0 in stage 35.0 (TID 806) in 6880 ms on 10.0.0.133 (executor driver) (30/34) +26/04/01 08:52:06 INFO Executor: Finished task 28.0 in stage 35.0 (TID 805). 10455 bytes result sent to driver +26/04/01 08:52:06 INFO TaskSetManager: Finished task 28.0 in stage 35.0 (TID 805) in 6959 ms on 10.0.0.133 (executor driver) (31/34) +26/04/01 08:52:06 INFO Executor: Finished task 30.0 in stage 35.0 (TID 807). 10455 bytes result sent to driver +26/04/01 08:52:06 INFO TaskSetManager: Finished task 30.0 in stage 35.0 (TID 807) in 6953 ms on 10.0.0.133 (executor driver) (32/34) +26/04/01 08:52:08 INFO Executor: Finished task 33.0 in stage 35.0 (TID 810). 9724 bytes result sent to driver +26/04/01 08:52:08 INFO TaskSetManager: Finished task 33.0 in stage 35.0 (TID 810) in 2145 ms on 10.0.0.133 (executor driver) (33/34) +26/04/01 08:52:12 INFO Executor: Finished task 32.0 in stage 35.0 (TID 809). 10455 bytes result sent to driver +26/04/01 08:52:12 INFO TaskSetManager: Finished task 32.0 in stage 35.0 (TID 809) in 6571 ms on 10.0.0.133 (executor driver) (34/34) +26/04/01 08:52:12 INFO TaskSchedulerImpl: Removed TaskSet 35.0, whose tasks have all completed, from pool +26/04/01 08:52:12 INFO DAGScheduler: ShuffleMapStage 35 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 34.407 s +26/04/01 08:52:12 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:52:12 INFO DAGScheduler: running: Set() +26/04/01 08:52:12 INFO DAGScheduler: waiting: Set() +26/04/01 08:52:12 INFO DAGScheduler: failed: Set() +26/04/01 08:52:12 INFO ShufflePartitionsUtil: For shuffle(8), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 08:52:12 INFO CodeGenerator: Code generated in 61.72675 ms +26/04/01 08:52:12 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:52:12 INFO DAGScheduler: Got job 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 2 output partitions +26/04/01 08:52:12 INFO DAGScheduler: Final stage: ResultStage 44 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:52:12 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 43) +26/04/01 08:52:12 INFO DAGScheduler: Missing parents: List() +26/04/01 08:52:12 INFO DAGScheduler: Submitting ResultStage 44 (MapPartitionsRDD[57] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:52:12 INFO MemoryStore: Block broadcast_27 stored as values in memory (estimated size 20.9 KiB, free 12.6 GiB) +26/04/01 08:52:12 INFO MemoryStore: Block broadcast_27_piece0 stored as bytes in memory (estimated size 8.3 KiB, free 12.6 GiB) +26/04/01 08:52:12 INFO BlockManagerInfo: Added broadcast_27_piece0 in memory on 10.0.0.133:59333 (size: 8.3 KiB, free: 12.6 GiB) +26/04/01 08:52:12 INFO SparkContext: Created broadcast 27 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:52:12 INFO DAGScheduler: Submitting 2 missing tasks from ResultStage 44 (MapPartitionsRDD[57] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1)) +26/04/01 08:52:12 INFO TaskSchedulerImpl: Adding task set 44.0 with 2 tasks resource profile 0 +26/04/01 08:52:12 INFO TaskSetManager: Starting task 0.0 in stage 44.0 (TID 811) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) +26/04/01 08:52:12 INFO TaskSetManager: Starting task 1.0 in stage 44.0 (TID 812) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9415 bytes) +26/04/01 08:52:12 INFO Executor: Running task 1.0 in stage 44.0 (TID 812) +26/04/01 08:52:12 INFO Executor: Running task 0.0 in stage 44.0 (TID 811) +26/04/01 08:52:12 INFO ShuffleBlockFetcherIterator: Getting 34 (1186.0 KiB) non-empty blocks including 34 (1186.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:52:12 INFO ShuffleBlockFetcherIterator: Getting 34 (1039.8 KiB) non-empty blocks including 34 (1039.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:52:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:52:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:52:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:52:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:52:12 INFO CodeGenerator: Code generated in 7.274292 ms +26/04/01 08:52:13 INFO Executor: Finished task 0.0 in stage 44.0 (TID 811). 8901 bytes result sent to driver +26/04/01 08:52:13 INFO TaskSetManager: Finished task 0.0 in stage 44.0 (TID 811) in 501 ms on 10.0.0.133 (executor driver) (1/2) +26/04/01 08:52:13 INFO Executor: Finished task 1.0 in stage 44.0 (TID 812). 9919 bytes result sent to driver +26/04/01 08:52:13 INFO TaskSetManager: Finished task 1.0 in stage 44.0 (TID 812) in 561 ms on 10.0.0.133 (executor driver) (2/2) +26/04/01 08:52:13 INFO TaskSchedulerImpl: Removed TaskSet 44.0, whose tasks have all completed, from pool +26/04/01 08:52:13 INFO DAGScheduler: ResultStage 44 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.562 s +26/04/01 08:52:13 INFO DAGScheduler: Job 19 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:52:13 INFO TaskSchedulerImpl: Killing all running tasks in stage 44: Stage finished +26/04/01 08:52:13 INFO DAGScheduler: Job 19 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.564987 s +26/04/01 08:52:13 INFO DAGScheduler: Registering RDD 58 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 9 +26/04/01 08:52:13 INFO DAGScheduler: Got map stage job 20 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 2 output partitions +26/04/01 08:52:13 INFO DAGScheduler: Final stage: ShuffleMapStage 53 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:52:13 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 52) +26/04/01 08:52:13 INFO DAGScheduler: Missing parents: List() +26/04/01 08:52:13 INFO DAGScheduler: Submitting ShuffleMapStage 53 (MapPartitionsRDD[58] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:52:13 INFO MemoryStore: Block broadcast_28 stored as values in memory (estimated size 31.4 KiB, free 12.6 GiB) +26/04/01 08:52:13 INFO MemoryStore: Block broadcast_28_piece0 stored as bytes in memory (estimated size 10.6 KiB, free 12.6 GiB) +26/04/01 08:52:13 INFO BlockManagerInfo: Added broadcast_28_piece0 in memory on 10.0.0.133:59333 (size: 10.6 KiB, free: 12.6 GiB) +26/04/01 08:52:13 INFO SparkContext: Created broadcast 28 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:52:13 INFO DAGScheduler: Submitting 2 missing tasks from ShuffleMapStage 53 (MapPartitionsRDD[58] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1)) +26/04/01 08:52:13 INFO TaskSchedulerImpl: Adding task set 53.0 with 2 tasks resource profile 0 +26/04/01 08:52:13 INFO TaskSetManager: Starting task 0.0 in stage 53.0 (TID 813) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9404 bytes) +26/04/01 08:52:13 INFO TaskSetManager: Starting task 1.0 in stage 53.0 (TID 814) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9404 bytes) +26/04/01 08:52:13 INFO Executor: Running task 1.0 in stage 53.0 (TID 814) +26/04/01 08:52:13 INFO Executor: Running task 0.0 in stage 53.0 (TID 813) +26/04/01 08:52:13 INFO CodeGenerator: Code generated in 3.569375 ms +26/04/01 08:52:13 INFO ShuffleBlockFetcherIterator: Getting 34 (1186.0 KiB) non-empty blocks including 34 (1186.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:52:13 INFO ShuffleBlockFetcherIterator: Getting 34 (1039.8 KiB) non-empty blocks including 34 (1039.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:52:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:52:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:52:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:52:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:52:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:52:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:52:13 INFO Executor: Finished task 0.0 in stage 53.0 (TID 813). 6566 bytes result sent to driver +26/04/01 08:52:13 INFO TaskSetManager: Finished task 0.0 in stage 53.0 (TID 813) in 491 ms on 10.0.0.133 (executor driver) (1/2) +26/04/01 08:52:13 INFO Executor: Finished task 1.0 in stage 53.0 (TID 814). 6566 bytes result sent to driver +26/04/01 08:52:13 INFO TaskSetManager: Finished task 1.0 in stage 53.0 (TID 814) in 553 ms on 10.0.0.133 (executor driver) (2/2) +26/04/01 08:52:13 INFO TaskSchedulerImpl: Removed TaskSet 53.0, whose tasks have all completed, from pool +26/04/01 08:52:13 INFO DAGScheduler: ShuffleMapStage 53 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.555 s +26/04/01 08:52:13 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:52:13 INFO DAGScheduler: running: Set() +26/04/01 08:52:13 INFO DAGScheduler: waiting: Set() +26/04/01 08:52:13 INFO DAGScheduler: failed: Set() +26/04/01 08:52:13 INFO ShufflePartitionsUtil: For shuffle(9), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 08:52:13 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:52:13 INFO DAGScheduler: Got job 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 08:52:13 INFO DAGScheduler: Final stage: ResultStage 63 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:52:13 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 62) +26/04/01 08:52:13 INFO DAGScheduler: Missing parents: List() +26/04/01 08:52:13 INFO DAGScheduler: Submitting ResultStage 63 (MapPartitionsRDD[62] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:52:13 INFO MemoryStore: Block broadcast_29 stored as values in memory (estimated size 28.6 KiB, free 12.6 GiB) +26/04/01 08:52:13 INFO MemoryStore: Block broadcast_29_piece0 stored as bytes in memory (estimated size 10.4 KiB, free 12.6 GiB) +26/04/01 08:52:13 INFO BlockManagerInfo: Added broadcast_29_piece0 in memory on 10.0.0.133:59333 (size: 10.4 KiB, free: 12.6 GiB) +26/04/01 08:52:13 INFO SparkContext: Created broadcast 29 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:52:13 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 63 (MapPartitionsRDD[62] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:52:13 INFO TaskSchedulerImpl: Adding task set 63.0 with 1 tasks resource profile 0 +26/04/01 08:52:13 INFO TaskSetManager: Starting task 0.0 in stage 63.0 (TID 815) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) +26/04/01 08:52:13 INFO Executor: Running task 0.0 in stage 63.0 (TID 815) +26/04/01 08:52:13 INFO ShuffleBlockFetcherIterator: Getting 2 (76.4 KiB) non-empty blocks including 2 (76.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:52:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:52:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB +26/04/01 08:52:13 INFO Executor: Finished task 0.0 in stage 63.0 (TID 815). 8247 bytes result sent to driver +26/04/01 08:52:13 INFO TaskSetManager: Finished task 0.0 in stage 63.0 (TID 815) in 47 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:52:13 INFO TaskSchedulerImpl: Removed TaskSet 63.0, whose tasks have all completed, from pool +26/04/01 08:52:13 INFO DAGScheduler: ResultStage 63 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.048 s +26/04/01 08:52:13 INFO DAGScheduler: Job 21 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:52:13 INFO TaskSchedulerImpl: Killing all running tasks in stage 63: Stage finished +26/04/01 08:52:13 INFO DAGScheduler: Job 21 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.049815 s +26/04/01 08:52:13 INFO SparkContext: SparkContext is stopping with exitCode 0. +26/04/01 08:52:13 INFO CometDriverPlugin: CometDriverPlugin shutdown +26/04/01 08:52:13 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! +26/04/01 08:52:17 INFO MemoryStore: MemoryStore cleared +26/04/01 08:52:17 INFO BlockManager: BlockManager stopped +26/04/01 08:52:17 INFO BlockManagerMaster: BlockManagerMaster stopped +26/04/01 08:52:17 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! +26/04/01 08:52:17 INFO SparkContext: Successfully stopped SparkContext +26/04/01 08:52:18 INFO ShutdownHookManager: Shutdown hook called +26/04/01 08:52:18 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-7b7c98ca-005e-4989-9d8f-bb80c5e1e895 +26/04/01 08:52:18 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-ec23fb53-cea4-469c-8aca-32f0bdc724e4 +26/04/01 08:52:18 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-7b7c98ca-005e-4989-9d8f-bb80c5e1e895/pyspark-a55b127f-06d7-458f-8190-828f6f7a60ff + 384.01 real 3099.31 user 33.62 sys + 8859074560 maximum resident set size + 0 average shared memory size + 0 average unshared data size + 0 average unshared stack size + 623152 page reclaims + 70 page faults + 0 swaps + 0 block input operations + 0 block output operations + 1278 messages sent + 2365 messages received + 36 signals received + 104418 voluntary context switches + 893124 involuntary context switches + 43657947224407 instructions retired + 11278973166768 cycles elapsed + 3867993376 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.log b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.log new file mode 100644 index 0000000000..e62bd4aaf5 --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.log @@ -0,0 +1,80 @@ +Registering table customer from /opt/tpch/sf100/customer +Registering table lineitem from /opt/tpch/sf100/lineitem +Registering table nation from /opt/tpch/sf100/nation +Registering table orders from /opt/tpch/sf100/orders +Registering table part from /opt/tpch/sf100/part +Registering table partsupp from /opt/tpch/sf100/partsupp +Registering table region from /opt/tpch/sf100/region +Registering table supplier from /opt/tpch/sf100/supplier + +============================================================ +Starting iteration 1 of 1 +============================================================ + +Running query 1 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q1.sql +Executing: -- CometBench-H query 1 derived from TPC-H query 1 under the terms of the TPC Fair Use Policy. +-- TP... +== Physical Plan == +AdaptiveSparkPlan (9) ++- Sort (8) + +- Exchange (7) + +- HashAggregate (6) + +- Exchange (5) + +- HashAggregate (4) + +- Project (3) + +- Filter (2) + +- Scan parquet (1) + + +(1) Scan parquet +Output [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] +PushedFilters: [IsNotNull(l_shipdate), LessThanOrEqual(l_shipdate,1998-09-24)] +ReadSchema: struct + +(2) Filter +Input [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] +Condition : (isnotnull(l_shipdate#26) AND (l_shipdate#26 <= 1998-09-24)) + +(3) Project +Output [6]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25] +Input [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] + +(4) HashAggregate +Input [6]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25] +Keys [2]: [l_returnflag#24, l_linestatus#25] +Functions [8]: [partial_sum(l_quantity#20), partial_sum(l_extendedprice#21), partial_sum((l_extendedprice#21 * (1 - l_discount#22))), partial_sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23))), partial_avg(l_quantity#20), partial_avg(l_extendedprice#21), partial_avg(l_discount#22), partial_count(1)] +Aggregate Attributes [15]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, count#157L, sum#158, count#159L, sum#160, count#161L, count#162L] +Results [17]: [l_returnflag#24, l_linestatus#25, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, count#172L, sum#173, count#174L, sum#175, count#176L, count#177L] + +(5) Exchange +Input [17]: [l_returnflag#24, l_linestatus#25, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, count#172L, sum#173, count#174L, sum#175, count#176L, count#177L] +Arguments: hashpartitioning(l_returnflag#24, l_linestatus#25, 200), ENSURE_REQUIREMENTS, [plan_id=43] + +(6) HashAggregate +Input [17]: [l_returnflag#24, l_linestatus#25, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, count#172L, sum#173, count#174L, sum#175, count#176L, count#177L] +Keys [2]: [l_returnflag#24, l_linestatus#25] +Functions [8]: [sum(l_quantity#20), sum(l_extendedprice#21), sum((l_extendedprice#21 * (1 - l_discount#22))), sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23))), avg(l_quantity#20), avg(l_extendedprice#21), avg(l_discount#22), count(1)] +Aggregate Attributes [8]: [sum(l_quantity#20)#131, sum(l_extendedprice#21)#132, sum((l_extendedprice#21 * (1 - l_discount#22)))#136, sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23)))#137, avg(l_quantity#20)#133, avg(l_extendedprice#21)#134, avg(l_discount#22)#135, count(1)#130L] +Results [10]: [l_returnflag#24, l_linestatus#25, sum(l_quantity#20)#131 AS sum_qty#122, sum(l_extendedprice#21)#132 AS sum_base_price#123, sum((l_extendedprice#21 * (1 - l_discount#22)))#136 AS sum_disc_price#124, sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23)))#137 AS sum_charge#125, avg(l_quantity#20)#133 AS avg_qty#126, avg(l_extendedprice#21)#134 AS avg_price#127, avg(l_discount#22)#135 AS avg_disc#128, count(1)#130L AS count_order#129L] + +(7) Exchange +Input [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] +Arguments: rangepartitioning(l_returnflag#24 ASC NULLS FIRST, l_linestatus#25 ASC NULLS FIRST, 200), ENSURE_REQUIREMENTS, [plan_id=46] + +(8) Sort +Input [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] +Arguments: [l_returnflag#24 ASC NULLS FIRST, l_linestatus#25 ASC NULLS FIRST], true, 0 + +(9) AdaptiveSparkPlan +Output [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] +Arguments: isFinalPlan=false + + +Query 1 returned 4 rows, hash=6839cc802df6220ed2efdaaaa48373a6 +Query 1 took 86.17 seconds + +Iteration 1 took 86.17 seconds + +Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/spark-offheap4g-q1-tpch-1775044569301.json diff --git a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.time b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.time new file mode 100644 index 0000000000..a304cc0253 --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.time @@ -0,0 +1,1650 @@ +26/04/01 05:54:40 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) +26/04/01 05:54:40 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address +26/04/01 05:54:41 INFO SparkContext: Running Spark version 3.5.8 +26/04/01 05:54:41 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 05:54:41 INFO SparkContext: Java version 17.0.17 +26/04/01 05:54:41 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +26/04/01 05:54:41 INFO ResourceUtils: ============================================================== +26/04/01 05:54:41 INFO ResourceUtils: No custom resources configured for spark.driver. +26/04/01 05:54:41 INFO ResourceUtils: ============================================================== +26/04/01 05:54:41 INFO SparkContext: Submitted application: spark-offheap4g-q1 benchmark derived from tpch +26/04/01 05:54:41 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 4096, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) +26/04/01 05:54:41 INFO ResourceProfile: Limiting resource is cpu +26/04/01 05:54:41 INFO ResourceProfileManager: Added ResourceProfile id: 0 +26/04/01 05:54:41 INFO SecurityManager: Changing view acls to: andy +26/04/01 05:54:41 INFO SecurityManager: Changing modify acls to: andy +26/04/01 05:54:41 INFO SecurityManager: Changing view acls groups to: +26/04/01 05:54:41 INFO SecurityManager: Changing modify acls groups to: +26/04/01 05:54:41 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY +26/04/01 05:54:41 INFO Utils: Successfully started service 'sparkDriver' on port 58186. +26/04/01 05:54:41 INFO SparkEnv: Registering MapOutputTracker +26/04/01 05:54:41 INFO SparkEnv: Registering BlockManagerMaster +26/04/01 05:54:41 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information +26/04/01 05:54:41 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up +26/04/01 05:54:41 INFO SparkEnv: Registering BlockManagerMasterHeartbeat +26/04/01 05:54:41 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-4f91f881-2f04-42af-8e2f-9947d49e7a91 +26/04/01 05:54:41 INFO MemoryStore: MemoryStore started with capacity 8.6 GiB +26/04/01 05:54:41 INFO SparkEnv: Registering OutputCommitCoordinator +26/04/01 05:54:41 INFO Executor: Starting executor ID driver on host 10.0.0.133 +26/04/01 05:54:41 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 05:54:41 INFO Executor: Java version 17.0.17 +26/04/01 05:54:41 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' +26/04/01 05:54:41 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@1b3b2259 for default. +26/04/01 05:54:41 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 58187. +26/04/01 05:54:41 INFO NettyBlockTransferService: Server created on 10.0.0.133:58187 +26/04/01 05:54:41 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy +26/04/01 05:54:41 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 58187, None) +26/04/01 05:54:41 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:58187 with 8.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 58187, None) +26/04/01 05:54:41 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 58187, None) +26/04/01 05:54:41 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 58187, None) +26/04/01 05:54:41 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. +26/04/01 05:54:41 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. +26/04/01 05:54:41 INFO InMemoryFileIndex: It took 21 ms to list leaf files for 1 paths. +26/04/01 05:54:42 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:54:42 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:54:42 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:54:42 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:54:42 INFO DAGScheduler: Missing parents: List() +26/04/01 05:54:42 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:54:42 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:54:42 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:54:42 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:42 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:54:42 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:54:42 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 +26/04/01 05:54:42 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) +26/04/01 05:54:42 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) +26/04/01 05:54:42 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 1998 bytes result sent to driver +26/04/01 05:54:42 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 127 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:54:42 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool +26/04/01 05:54:42 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.328 s +26/04/01 05:54:42 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:54:42 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished +26/04/01 05:54:42 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.339591 s +26/04/01 05:54:42 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:42 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. +26/04/01 05:54:42 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:54:42 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:54:42 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:54:42 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:54:42 INFO DAGScheduler: Missing parents: List() +26/04/01 05:54:42 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:54:42 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:54:42 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:54:42 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:42 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:54:42 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:54:42 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 +26/04/01 05:54:42 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) +26/04/01 05:54:42 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) +26/04/01 05:54:42 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2344 bytes result sent to driver +26/04/01 05:54:42 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 15 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:54:42 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool +26/04/01 05:54:42 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.023 s +26/04/01 05:54:42 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:54:42 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished +26/04/01 05:54:42 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.025069 s +26/04/01 05:54:42 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. +26/04/01 05:54:42 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:54:42 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:54:42 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:54:42 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:54:42 INFO DAGScheduler: Missing parents: List() +26/04/01 05:54:42 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:54:42 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:54:42 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:54:42 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:42 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:54:42 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:54:42 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 +26/04/01 05:54:42 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) +26/04/01 05:54:42 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) +26/04/01 05:54:42 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver +26/04/01 05:54:42 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 7 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:54:42 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool +26/04/01 05:54:42 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.016 s +26/04/01 05:54:42 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:54:42 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished +26/04/01 05:54:42 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.017186 s +26/04/01 05:54:42 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. +26/04/01 05:54:42 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:54:42 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:54:42 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:54:42 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:54:42 INFO DAGScheduler: Missing parents: List() +26/04/01 05:54:42 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:54:42 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:42 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:54:42 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:42 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:54:42 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:42 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:54:42 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:54:42 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 +26/04/01 05:54:42 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) +26/04/01 05:54:42 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) +26/04/01 05:54:42 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver +26/04/01 05:54:42 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 7 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:54:42 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool +26/04/01 05:54:42 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.017 s +26/04/01 05:54:42 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:54:42 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished +26/04/01 05:54:42 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.018134 s +26/04/01 05:54:42 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. +26/04/01 05:54:42 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:54:42 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:54:42 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:54:42 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:54:42 INFO DAGScheduler: Missing parents: List() +26/04/01 05:54:42 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:54:42 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:54:42 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:54:42 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:42 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:54:42 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:54:42 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 +26/04/01 05:54:42 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9134 bytes) +26/04/01 05:54:42 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) +26/04/01 05:54:42 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver +26/04/01 05:54:42 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:54:42 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool +26/04/01 05:54:42 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s +26/04/01 05:54:42 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:54:42 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished +26/04/01 05:54:42 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.015374 s +26/04/01 05:54:42 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. +26/04/01 05:54:43 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:54:43 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:54:43 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:54:43 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:54:43 INFO DAGScheduler: Missing parents: List() +26/04/01 05:54:43 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:54:43 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:54:43 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:54:43 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:43 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:43 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:54:43 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:54:43 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 +26/04/01 05:54:43 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) +26/04/01 05:54:43 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:43 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) +26/04/01 05:54:43 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver +26/04/01 05:54:43 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:54:43 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool +26/04/01 05:54:43 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s +26/04/01 05:54:43 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:54:43 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished +26/04/01 05:54:43 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.016021 s +26/04/01 05:54:43 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. +26/04/01 05:54:43 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:54:43 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:54:43 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:54:43 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:54:43 INFO DAGScheduler: Missing parents: List() +26/04/01 05:54:43 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:54:43 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:54:43 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:54:43 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:43 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:54:43 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:54:43 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 +26/04/01 05:54:43 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) +26/04/01 05:54:43 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) +26/04/01 05:54:43 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver +26/04/01 05:54:43 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:54:43 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool +26/04/01 05:54:43 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.011 s +26/04/01 05:54:43 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:54:43 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished +26/04/01 05:54:43 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.013125 s +26/04/01 05:54:43 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 05:54:43 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:54:43 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:54:43 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:54:43 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:54:43 INFO DAGScheduler: Missing parents: List() +26/04/01 05:54:43 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:54:43 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:54:43 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:54:43 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:43 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:43 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:54:43 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:54:43 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 +26/04/01 05:54:43 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) +26/04/01 05:54:43 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:43 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) +26/04/01 05:54:43 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver +26/04/01 05:54:43 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:54:43 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool +26/04/01 05:54:43 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s +26/04/01 05:54:43 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:54:43 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished +26/04/01 05:54:43 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.016616 s +26/04/01 05:54:43 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_shipdate),LessThanOrEqual(l_shipdate,1998-09-24) +26/04/01 05:54:43 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_shipdate#26),(l_shipdate#26 <= 1998-09-24) +26/04/01 05:54:43 INFO HashAggregateExec: spark.sql.codegen.aggregate.map.twolevel.enabled is set to true, but current version of codegened fast hashmap does not support this aggregate. +26/04/01 05:54:43 INFO CodeGenerator: Code generated in 113.466292 ms +26/04/01 05:54:43 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 202.5 KiB, free 8.6 GiB) +26/04/01 05:54:43 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.3 KiB, free 8.6 GiB) +26/04/01 05:54:43 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:58187 (size: 35.3 KiB, free: 8.6 GiB) +26/04/01 05:54:43 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:54:43 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 05:54:43 INFO DAGScheduler: Registering RDD 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 +26/04/01 05:54:43 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions +26/04/01 05:54:43 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:54:43 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:54:43 INFO DAGScheduler: Missing parents: List() +26/04/01 05:54:43 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[19] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:54:43 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 72.6 KiB, free 8.6 GiB) +26/04/01 05:54:43 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 27.2 KiB, free 8.6 GiB) +26/04/01 05:54:43 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:58187 (size: 27.2 KiB, free: 8.6 GiB) +26/04/01 05:54:43 INFO SparkContext: Created broadcast 9 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:54:43 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[19] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 05:54:43 INFO TaskSchedulerImpl: Adding task set 8.0 with 208 tasks resource profile 0 +26/04/01 05:54:43 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:43 INFO TaskSetManager: Starting task 1.0 in stage 8.0 (TID 9) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:43 INFO TaskSetManager: Starting task 2.0 in stage 8.0 (TID 10) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:43 INFO TaskSetManager: Starting task 3.0 in stage 8.0 (TID 11) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:43 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) +26/04/01 05:54:43 INFO Executor: Running task 1.0 in stage 8.0 (TID 9) +26/04/01 05:54:43 INFO Executor: Running task 2.0 in stage 8.0 (TID 10) +26/04/01 05:54:43 INFO Executor: Running task 3.0 in stage 8.0 (TID 11) +26/04/01 05:54:43 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:54:43 INFO CodeGenerator: Code generated in 21.949417 ms +26/04/01 05:54:43 INFO CodeGenerator: Code generated in 5.071291 ms +26/04/01 05:54:43 INFO CodeGenerator: Code generated in 2.986917 ms +26/04/01 05:54:43 INFO CodeGenerator: Code generated in 2.117208 ms +26/04/01 05:54:43 INFO CodeGenerator: Code generated in 5.845834 ms +26/04/01 05:54:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:54:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:54:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:54:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:54:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:43 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 05:54:43 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 05:54:43 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 05:54:43 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 05:54:45 INFO Executor: Finished task 3.0 in stage 8.0 (TID 11). 3027 bytes result sent to driver +26/04/01 05:54:45 INFO Executor: Finished task 1.0 in stage 8.0 (TID 9). 3027 bytes result sent to driver +26/04/01 05:54:45 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 3027 bytes result sent to driver +26/04/01 05:54:45 INFO Executor: Finished task 2.0 in stage 8.0 (TID 10). 3027 bytes result sent to driver +26/04/01 05:54:45 INFO TaskSetManager: Starting task 4.0 in stage 8.0 (TID 12) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:45 INFO Executor: Running task 4.0 in stage 8.0 (TID 12) +26/04/01 05:54:45 INFO TaskSetManager: Starting task 5.0 in stage 8.0 (TID 13) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:45 INFO TaskSetManager: Finished task 3.0 in stage 8.0 (TID 11) in 2319 ms on 10.0.0.133 (executor driver) (1/208) +26/04/01 05:54:45 INFO Executor: Running task 5.0 in stage 8.0 (TID 13) +26/04/01 05:54:45 INFO TaskSetManager: Starting task 6.0 in stage 8.0 (TID 14) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:45 INFO Executor: Running task 6.0 in stage 8.0 (TID 14) +26/04/01 05:54:45 INFO TaskSetManager: Finished task 2.0 in stage 8.0 (TID 10) in 2320 ms on 10.0.0.133 (executor driver) (2/208) +26/04/01 05:54:45 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 2322 ms on 10.0.0.133 (executor driver) (3/208) +26/04/01 05:54:45 INFO TaskSetManager: Starting task 7.0 in stage 8.0 (TID 15) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:45 INFO Executor: Running task 7.0 in stage 8.0 (TID 15) +26/04/01 05:54:45 INFO TaskSetManager: Finished task 1.0 in stage 8.0 (TID 9) in 2321 ms on 10.0.0.133 (executor driver) (4/208) +26/04/01 05:54:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:54:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:54:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:54:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:54:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:47 INFO Executor: Finished task 7.0 in stage 8.0 (TID 15). 3027 bytes result sent to driver +26/04/01 05:54:47 INFO Executor: Finished task 6.0 in stage 8.0 (TID 14). 3027 bytes result sent to driver +26/04/01 05:54:47 INFO TaskSetManager: Starting task 8.0 in stage 8.0 (TID 16) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:47 INFO Executor: Running task 8.0 in stage 8.0 (TID 16) +26/04/01 05:54:47 INFO TaskSetManager: Starting task 9.0 in stage 8.0 (TID 17) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:47 INFO TaskSetManager: Finished task 7.0 in stage 8.0 (TID 15) in 1741 ms on 10.0.0.133 (executor driver) (5/208) +26/04/01 05:54:47 INFO Executor: Running task 9.0 in stage 8.0 (TID 17) +26/04/01 05:54:47 INFO TaskSetManager: Finished task 6.0 in stage 8.0 (TID 14) in 1742 ms on 10.0.0.133 (executor driver) (6/208) +26/04/01 05:54:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:54:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:54:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:47 INFO Executor: Finished task 5.0 in stage 8.0 (TID 13). 3027 bytes result sent to driver +26/04/01 05:54:47 INFO TaskSetManager: Starting task 10.0 in stage 8.0 (TID 18) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:47 INFO TaskSetManager: Finished task 5.0 in stage 8.0 (TID 13) in 1761 ms on 10.0.0.133 (executor driver) (7/208) +26/04/01 05:54:47 INFO Executor: Running task 10.0 in stage 8.0 (TID 18) +26/04/01 05:54:47 INFO Executor: Finished task 4.0 in stage 8.0 (TID 12). 3027 bytes result sent to driver +26/04/01 05:54:47 INFO TaskSetManager: Starting task 11.0 in stage 8.0 (TID 19) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:47 INFO Executor: Running task 11.0 in stage 8.0 (TID 19) +26/04/01 05:54:47 INFO TaskSetManager: Finished task 4.0 in stage 8.0 (TID 12) in 1767 ms on 10.0.0.133 (executor driver) (8/208) +26/04/01 05:54:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:54:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:54:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:49 INFO Executor: Finished task 9.0 in stage 8.0 (TID 17). 3027 bytes result sent to driver +26/04/01 05:54:49 INFO TaskSetManager: Starting task 12.0 in stage 8.0 (TID 20) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:49 INFO Executor: Running task 12.0 in stage 8.0 (TID 20) +26/04/01 05:54:49 INFO TaskSetManager: Finished task 9.0 in stage 8.0 (TID 17) in 1708 ms on 10.0.0.133 (executor driver) (9/208) +26/04/01 05:54:49 INFO Executor: Finished task 8.0 in stage 8.0 (TID 16). 3027 bytes result sent to driver +26/04/01 05:54:49 INFO TaskSetManager: Starting task 13.0 in stage 8.0 (TID 21) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:49 INFO TaskSetManager: Finished task 8.0 in stage 8.0 (TID 16) in 1712 ms on 10.0.0.133 (executor driver) (10/208) +26/04/01 05:54:49 INFO Executor: Running task 13.0 in stage 8.0 (TID 21) +26/04/01 05:54:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:54:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:54:49 INFO Executor: Finished task 10.0 in stage 8.0 (TID 18). 3027 bytes result sent to driver +26/04/01 05:54:49 INFO TaskSetManager: Starting task 14.0 in stage 8.0 (TID 22) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:49 INFO TaskSetManager: Finished task 10.0 in stage 8.0 (TID 18) in 1700 ms on 10.0.0.133 (executor driver) (11/208) +26/04/01 05:54:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:49 INFO Executor: Running task 14.0 in stage 8.0 (TID 22) +26/04/01 05:54:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:49 INFO Executor: Finished task 11.0 in stage 8.0 (TID 19). 3027 bytes result sent to driver +26/04/01 05:54:49 INFO TaskSetManager: Starting task 15.0 in stage 8.0 (TID 23) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:54:49 INFO TaskSetManager: Finished task 11.0 in stage 8.0 (TID 19) in 1700 ms on 10.0.0.133 (executor driver) (12/208) +26/04/01 05:54:49 INFO Executor: Running task 15.0 in stage 8.0 (TID 23) +26/04/01 05:54:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:54:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:51 INFO Executor: Finished task 13.0 in stage 8.0 (TID 21). 3027 bytes result sent to driver +26/04/01 05:54:51 INFO Executor: Finished task 12.0 in stage 8.0 (TID 20). 3027 bytes result sent to driver +26/04/01 05:54:51 INFO TaskSetManager: Starting task 16.0 in stage 8.0 (TID 24) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:51 INFO TaskSetManager: Finished task 13.0 in stage 8.0 (TID 21) in 1664 ms on 10.0.0.133 (executor driver) (13/208) +26/04/01 05:54:51 INFO Executor: Running task 16.0 in stage 8.0 (TID 24) +26/04/01 05:54:51 INFO TaskSetManager: Starting task 17.0 in stage 8.0 (TID 25) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:51 INFO TaskSetManager: Finished task 12.0 in stage 8.0 (TID 20) in 1668 ms on 10.0.0.133 (executor driver) (14/208) +26/04/01 05:54:51 INFO Executor: Running task 17.0 in stage 8.0 (TID 25) +26/04/01 05:54:51 INFO Executor: Finished task 14.0 in stage 8.0 (TID 22). 3027 bytes result sent to driver +26/04/01 05:54:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:54:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:54:51 INFO TaskSetManager: Starting task 18.0 in stage 8.0 (TID 26) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:54:51 INFO TaskSetManager: Finished task 14.0 in stage 8.0 (TID 22) in 1662 ms on 10.0.0.133 (executor driver) (15/208) +26/04/01 05:54:51 INFO Executor: Running task 18.0 in stage 8.0 (TID 26) +26/04/01 05:54:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:54:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:51 INFO Executor: Finished task 15.0 in stage 8.0 (TID 23). 3027 bytes result sent to driver +26/04/01 05:54:51 INFO TaskSetManager: Starting task 19.0 in stage 8.0 (TID 27) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:54:51 INFO Executor: Running task 19.0 in stage 8.0 (TID 27) +26/04/01 05:54:51 INFO TaskSetManager: Finished task 15.0 in stage 8.0 (TID 23) in 1664 ms on 10.0.0.133 (executor driver) (16/208) +26/04/01 05:54:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:54:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:52 INFO Executor: Finished task 16.0 in stage 8.0 (TID 24). 3027 bytes result sent to driver +26/04/01 05:54:52 INFO TaskSetManager: Starting task 20.0 in stage 8.0 (TID 28) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:54:52 INFO TaskSetManager: Finished task 16.0 in stage 8.0 (TID 24) in 1686 ms on 10.0.0.133 (executor driver) (17/208) +26/04/01 05:54:52 INFO Executor: Running task 20.0 in stage 8.0 (TID 28) +26/04/01 05:54:52 INFO Executor: Finished task 19.0 in stage 8.0 (TID 27). 3027 bytes result sent to driver +26/04/01 05:54:52 INFO TaskSetManager: Starting task 21.0 in stage 8.0 (TID 29) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:54:52 INFO Executor: Running task 21.0 in stage 8.0 (TID 29) +26/04/01 05:54:52 INFO TaskSetManager: Finished task 19.0 in stage 8.0 (TID 27) in 1675 ms on 10.0.0.133 (executor driver) (18/208) +26/04/01 05:54:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:54:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:54:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:52 INFO Executor: Finished task 18.0 in stage 8.0 (TID 26). 3027 bytes result sent to driver +26/04/01 05:54:52 INFO TaskSetManager: Starting task 22.0 in stage 8.0 (TID 30) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:54:52 INFO TaskSetManager: Finished task 18.0 in stage 8.0 (TID 26) in 1706 ms on 10.0.0.133 (executor driver) (19/208) +26/04/01 05:54:52 INFO Executor: Running task 22.0 in stage 8.0 (TID 30) +26/04/01 05:54:52 INFO Executor: Finished task 17.0 in stage 8.0 (TID 25). 3027 bytes result sent to driver +26/04/01 05:54:52 INFO TaskSetManager: Starting task 23.0 in stage 8.0 (TID 31) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:54:52 INFO TaskSetManager: Finished task 17.0 in stage 8.0 (TID 25) in 1712 ms on 10.0.0.133 (executor driver) (20/208) +26/04/01 05:54:52 INFO Executor: Running task 23.0 in stage 8.0 (TID 31) +26/04/01 05:54:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:54:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:54:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:54 INFO Executor: Finished task 21.0 in stage 8.0 (TID 29). 3027 bytes result sent to driver +26/04/01 05:54:54 INFO TaskSetManager: Starting task 24.0 in stage 8.0 (TID 32) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:54 INFO Executor: Running task 24.0 in stage 8.0 (TID 32) +26/04/01 05:54:54 INFO TaskSetManager: Finished task 21.0 in stage 8.0 (TID 29) in 1659 ms on 10.0.0.133 (executor driver) (21/208) +26/04/01 05:54:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:54:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:54 INFO Executor: Finished task 20.0 in stage 8.0 (TID 28). 3027 bytes result sent to driver +26/04/01 05:54:54 INFO TaskSetManager: Starting task 25.0 in stage 8.0 (TID 33) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:54 INFO TaskSetManager: Finished task 20.0 in stage 8.0 (TID 28) in 1681 ms on 10.0.0.133 (executor driver) (22/208) +26/04/01 05:54:54 INFO Executor: Running task 25.0 in stage 8.0 (TID 33) +26/04/01 05:54:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:54:54 INFO Executor: Finished task 22.0 in stage 8.0 (TID 30). 3027 bytes result sent to driver +26/04/01 05:54:54 INFO TaskSetManager: Starting task 26.0 in stage 8.0 (TID 34) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:54 INFO TaskSetManager: Finished task 22.0 in stage 8.0 (TID 30) in 1662 ms on 10.0.0.133 (executor driver) (23/208) +26/04/01 05:54:54 INFO Executor: Running task 26.0 in stage 8.0 (TID 34) +26/04/01 05:54:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:54:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:54 INFO Executor: Finished task 23.0 in stage 8.0 (TID 31). 2984 bytes result sent to driver +26/04/01 05:54:54 INFO TaskSetManager: Starting task 27.0 in stage 8.0 (TID 35) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:54 INFO TaskSetManager: Finished task 23.0 in stage 8.0 (TID 31) in 1672 ms on 10.0.0.133 (executor driver) (24/208) +26/04/01 05:54:54 INFO Executor: Running task 27.0 in stage 8.0 (TID 35) +26/04/01 05:54:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:54:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:56 INFO Executor: Finished task 24.0 in stage 8.0 (TID 32). 3027 bytes result sent to driver +26/04/01 05:54:56 INFO TaskSetManager: Starting task 28.0 in stage 8.0 (TID 36) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:56 INFO TaskSetManager: Finished task 24.0 in stage 8.0 (TID 32) in 1680 ms on 10.0.0.133 (executor driver) (25/208) +26/04/01 05:54:56 INFO Executor: Running task 28.0 in stage 8.0 (TID 36) +26/04/01 05:54:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:54:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:56 INFO Executor: Finished task 25.0 in stage 8.0 (TID 33). 3027 bytes result sent to driver +26/04/01 05:54:56 INFO TaskSetManager: Starting task 29.0 in stage 8.0 (TID 37) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:56 INFO TaskSetManager: Finished task 25.0 in stage 8.0 (TID 33) in 1683 ms on 10.0.0.133 (executor driver) (26/208) +26/04/01 05:54:56 INFO Executor: Running task 29.0 in stage 8.0 (TID 37) +26/04/01 05:54:56 INFO Executor: Finished task 26.0 in stage 8.0 (TID 34). 3027 bytes result sent to driver +26/04/01 05:54:56 INFO TaskSetManager: Starting task 30.0 in stage 8.0 (TID 38) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:56 INFO Executor: Running task 30.0 in stage 8.0 (TID 38) +26/04/01 05:54:56 INFO TaskSetManager: Finished task 26.0 in stage 8.0 (TID 34) in 1677 ms on 10.0.0.133 (executor driver) (27/208) +26/04/01 05:54:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:54:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:54:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:56 INFO Executor: Finished task 27.0 in stage 8.0 (TID 35). 3027 bytes result sent to driver +26/04/01 05:54:56 INFO TaskSetManager: Starting task 31.0 in stage 8.0 (TID 39) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:56 INFO TaskSetManager: Finished task 27.0 in stage 8.0 (TID 35) in 1679 ms on 10.0.0.133 (executor driver) (28/208) +26/04/01 05:54:56 INFO Executor: Running task 31.0 in stage 8.0 (TID 39) +26/04/01 05:54:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:54:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:57 INFO Executor: Finished task 28.0 in stage 8.0 (TID 36). 3027 bytes result sent to driver +26/04/01 05:54:57 INFO TaskSetManager: Starting task 32.0 in stage 8.0 (TID 40) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:57 INFO Executor: Running task 32.0 in stage 8.0 (TID 40) +26/04/01 05:54:57 INFO TaskSetManager: Finished task 28.0 in stage 8.0 (TID 36) in 1687 ms on 10.0.0.133 (executor driver) (29/208) +26/04/01 05:54:57 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:54:57 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:57 INFO Executor: Finished task 30.0 in stage 8.0 (TID 38). 3027 bytes result sent to driver +26/04/01 05:54:57 INFO TaskSetManager: Starting task 33.0 in stage 8.0 (TID 41) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:57 INFO Executor: Running task 33.0 in stage 8.0 (TID 41) +26/04/01 05:54:57 INFO TaskSetManager: Finished task 30.0 in stage 8.0 (TID 38) in 1681 ms on 10.0.0.133 (executor driver) (30/208) +26/04/01 05:54:57 INFO Executor: Finished task 29.0 in stage 8.0 (TID 37). 3027 bytes result sent to driver +26/04/01 05:54:57 INFO TaskSetManager: Starting task 34.0 in stage 8.0 (TID 42) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:57 INFO TaskSetManager: Finished task 29.0 in stage 8.0 (TID 37) in 1685 ms on 10.0.0.133 (executor driver) (31/208) +26/04/01 05:54:57 INFO Executor: Running task 34.0 in stage 8.0 (TID 42) +26/04/01 05:54:57 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:54:57 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:54:57 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:57 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:57 INFO Executor: Finished task 31.0 in stage 8.0 (TID 39). 3027 bytes result sent to driver +26/04/01 05:54:57 INFO TaskSetManager: Starting task 35.0 in stage 8.0 (TID 43) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:57 INFO TaskSetManager: Finished task 31.0 in stage 8.0 (TID 39) in 1682 ms on 10.0.0.133 (executor driver) (32/208) +26/04/01 05:54:57 INFO Executor: Running task 35.0 in stage 8.0 (TID 43) +26/04/01 05:54:57 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:54:57 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:59 INFO Executor: Finished task 32.0 in stage 8.0 (TID 40). 3027 bytes result sent to driver +26/04/01 05:54:59 INFO TaskSetManager: Starting task 36.0 in stage 8.0 (TID 44) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:59 INFO TaskSetManager: Finished task 32.0 in stage 8.0 (TID 40) in 1679 ms on 10.0.0.133 (executor driver) (33/208) +26/04/01 05:54:59 INFO Executor: Running task 36.0 in stage 8.0 (TID 44) +26/04/01 05:54:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:54:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:59 INFO Executor: Finished task 33.0 in stage 8.0 (TID 41). 3027 bytes result sent to driver +26/04/01 05:54:59 INFO TaskSetManager: Starting task 37.0 in stage 8.0 (TID 45) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:59 INFO Executor: Finished task 34.0 in stage 8.0 (TID 42). 3027 bytes result sent to driver +26/04/01 05:54:59 INFO Executor: Running task 37.0 in stage 8.0 (TID 45) +26/04/01 05:54:59 INFO TaskSetManager: Starting task 38.0 in stage 8.0 (TID 46) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:59 INFO TaskSetManager: Finished task 33.0 in stage 8.0 (TID 41) in 1671 ms on 10.0.0.133 (executor driver) (34/208) +26/04/01 05:54:59 INFO Executor: Running task 38.0 in stage 8.0 (TID 46) +26/04/01 05:54:59 INFO TaskSetManager: Finished task 34.0 in stage 8.0 (TID 42) in 1669 ms on 10.0.0.133 (executor driver) (35/208) +26/04/01 05:54:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:54:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:54:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:54:59 INFO Executor: Finished task 35.0 in stage 8.0 (TID 43). 3027 bytes result sent to driver +26/04/01 05:54:59 INFO TaskSetManager: Starting task 39.0 in stage 8.0 (TID 47) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:54:59 INFO TaskSetManager: Finished task 35.0 in stage 8.0 (TID 43) in 1673 ms on 10.0.0.133 (executor driver) (36/208) +26/04/01 05:54:59 INFO Executor: Running task 39.0 in stage 8.0 (TID 47) +26/04/01 05:54:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:54:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:01 INFO Executor: Finished task 36.0 in stage 8.0 (TID 44). 3027 bytes result sent to driver +26/04/01 05:55:01 INFO TaskSetManager: Starting task 40.0 in stage 8.0 (TID 48) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:01 INFO Executor: Running task 40.0 in stage 8.0 (TID 48) +26/04/01 05:55:01 INFO TaskSetManager: Finished task 36.0 in stage 8.0 (TID 44) in 1669 ms on 10.0.0.133 (executor driver) (37/208) +26/04/01 05:55:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:01 INFO Executor: Finished task 38.0 in stage 8.0 (TID 46). 3027 bytes result sent to driver +26/04/01 05:55:01 INFO Executor: Finished task 37.0 in stage 8.0 (TID 45). 3027 bytes result sent to driver +26/04/01 05:55:01 INFO TaskSetManager: Starting task 41.0 in stage 8.0 (TID 49) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:01 INFO Executor: Running task 41.0 in stage 8.0 (TID 49) +26/04/01 05:55:01 INFO TaskSetManager: Starting task 42.0 in stage 8.0 (TID 50) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:01 INFO Executor: Running task 42.0 in stage 8.0 (TID 50) +26/04/01 05:55:01 INFO TaskSetManager: Finished task 38.0 in stage 8.0 (TID 46) in 1674 ms on 10.0.0.133 (executor driver) (38/208) +26/04/01 05:55:01 INFO TaskSetManager: Finished task 37.0 in stage 8.0 (TID 45) in 1674 ms on 10.0.0.133 (executor driver) (39/208) +26/04/01 05:55:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:01 INFO Executor: Finished task 39.0 in stage 8.0 (TID 47). 3027 bytes result sent to driver +26/04/01 05:55:01 INFO TaskSetManager: Starting task 43.0 in stage 8.0 (TID 51) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:01 INFO TaskSetManager: Finished task 39.0 in stage 8.0 (TID 47) in 1675 ms on 10.0.0.133 (executor driver) (40/208) +26/04/01 05:55:01 INFO Executor: Running task 43.0 in stage 8.0 (TID 51) +26/04/01 05:55:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:02 INFO Executor: Finished task 40.0 in stage 8.0 (TID 48). 3027 bytes result sent to driver +26/04/01 05:55:02 INFO TaskSetManager: Starting task 44.0 in stage 8.0 (TID 52) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:02 INFO TaskSetManager: Finished task 40.0 in stage 8.0 (TID 48) in 1683 ms on 10.0.0.133 (executor driver) (41/208) +26/04/01 05:55:02 INFO Executor: Running task 44.0 in stage 8.0 (TID 52) +26/04/01 05:55:02 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:02 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:02 INFO Executor: Finished task 41.0 in stage 8.0 (TID 49). 3027 bytes result sent to driver +26/04/01 05:55:02 INFO TaskSetManager: Starting task 45.0 in stage 8.0 (TID 53) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:02 INFO TaskSetManager: Finished task 41.0 in stage 8.0 (TID 49) in 1676 ms on 10.0.0.133 (executor driver) (42/208) +26/04/01 05:55:02 INFO Executor: Running task 45.0 in stage 8.0 (TID 53) +26/04/01 05:55:02 INFO Executor: Finished task 42.0 in stage 8.0 (TID 50). 2984 bytes result sent to driver +26/04/01 05:55:02 INFO TaskSetManager: Starting task 46.0 in stage 8.0 (TID 54) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:02 INFO Executor: Running task 46.0 in stage 8.0 (TID 54) +26/04/01 05:55:02 INFO TaskSetManager: Finished task 42.0 in stage 8.0 (TID 50) in 1679 ms on 10.0.0.133 (executor driver) (43/208) +26/04/01 05:55:02 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:02 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:02 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:02 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:02 INFO Executor: Finished task 43.0 in stage 8.0 (TID 51). 3027 bytes result sent to driver +26/04/01 05:55:02 INFO TaskSetManager: Starting task 47.0 in stage 8.0 (TID 55) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:02 INFO TaskSetManager: Finished task 43.0 in stage 8.0 (TID 51) in 1671 ms on 10.0.0.133 (executor driver) (44/208) +26/04/01 05:55:02 INFO Executor: Running task 47.0 in stage 8.0 (TID 55) +26/04/01 05:55:02 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:02 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:04 INFO Executor: Finished task 44.0 in stage 8.0 (TID 52). 3027 bytes result sent to driver +26/04/01 05:55:04 INFO TaskSetManager: Starting task 48.0 in stage 8.0 (TID 56) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:04 INFO TaskSetManager: Finished task 44.0 in stage 8.0 (TID 52) in 1678 ms on 10.0.0.133 (executor driver) (45/208) +26/04/01 05:55:04 INFO Executor: Running task 48.0 in stage 8.0 (TID 56) +26/04/01 05:55:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:04 INFO Executor: Finished task 45.0 in stage 8.0 (TID 53). 3027 bytes result sent to driver +26/04/01 05:55:04 INFO TaskSetManager: Starting task 49.0 in stage 8.0 (TID 57) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:04 INFO Executor: Running task 49.0 in stage 8.0 (TID 57) +26/04/01 05:55:04 INFO TaskSetManager: Finished task 45.0 in stage 8.0 (TID 53) in 1682 ms on 10.0.0.133 (executor driver) (46/208) +26/04/01 05:55:04 INFO Executor: Finished task 46.0 in stage 8.0 (TID 54). 2984 bytes result sent to driver +26/04/01 05:55:04 INFO TaskSetManager: Starting task 50.0 in stage 8.0 (TID 58) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:04 INFO TaskSetManager: Finished task 46.0 in stage 8.0 (TID 54) in 1680 ms on 10.0.0.133 (executor driver) (47/208) +26/04/01 05:55:04 INFO Executor: Running task 50.0 in stage 8.0 (TID 58) +26/04/01 05:55:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:04 INFO Executor: Finished task 47.0 in stage 8.0 (TID 55). 2984 bytes result sent to driver +26/04/01 05:55:04 INFO TaskSetManager: Starting task 51.0 in stage 8.0 (TID 59) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:04 INFO Executor: Running task 51.0 in stage 8.0 (TID 59) +26/04/01 05:55:04 INFO TaskSetManager: Finished task 47.0 in stage 8.0 (TID 55) in 1678 ms on 10.0.0.133 (executor driver) (48/208) +26/04/01 05:55:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:06 INFO Executor: Finished task 48.0 in stage 8.0 (TID 56). 3027 bytes result sent to driver +26/04/01 05:55:06 INFO TaskSetManager: Starting task 52.0 in stage 8.0 (TID 60) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:06 INFO Executor: Running task 52.0 in stage 8.0 (TID 60) +26/04/01 05:55:06 INFO TaskSetManager: Finished task 48.0 in stage 8.0 (TID 56) in 1681 ms on 10.0.0.133 (executor driver) (49/208) +26/04/01 05:55:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:06 INFO Executor: Finished task 49.0 in stage 8.0 (TID 57). 3027 bytes result sent to driver +26/04/01 05:55:06 INFO TaskSetManager: Starting task 53.0 in stage 8.0 (TID 61) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:06 INFO Executor: Running task 53.0 in stage 8.0 (TID 61) +26/04/01 05:55:06 INFO TaskSetManager: Finished task 49.0 in stage 8.0 (TID 57) in 1677 ms on 10.0.0.133 (executor driver) (50/208) +26/04/01 05:55:06 INFO Executor: Finished task 50.0 in stage 8.0 (TID 58). 3027 bytes result sent to driver +26/04/01 05:55:06 INFO TaskSetManager: Starting task 54.0 in stage 8.0 (TID 62) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:06 INFO TaskSetManager: Finished task 50.0 in stage 8.0 (TID 58) in 1679 ms on 10.0.0.133 (executor driver) (51/208) +26/04/01 05:55:06 INFO Executor: Running task 54.0 in stage 8.0 (TID 62) +26/04/01 05:55:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:06 INFO Executor: Finished task 51.0 in stage 8.0 (TID 59). 3027 bytes result sent to driver +26/04/01 05:55:06 INFO TaskSetManager: Starting task 55.0 in stage 8.0 (TID 63) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:06 INFO TaskSetManager: Finished task 51.0 in stage 8.0 (TID 59) in 1677 ms on 10.0.0.133 (executor driver) (52/208) +26/04/01 05:55:06 INFO Executor: Running task 55.0 in stage 8.0 (TID 63) +26/04/01 05:55:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:07 INFO Executor: Finished task 52.0 in stage 8.0 (TID 60). 3027 bytes result sent to driver +26/04/01 05:55:07 INFO TaskSetManager: Starting task 56.0 in stage 8.0 (TID 64) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:07 INFO TaskSetManager: Finished task 52.0 in stage 8.0 (TID 60) in 1682 ms on 10.0.0.133 (executor driver) (53/208) +26/04/01 05:55:07 INFO Executor: Running task 56.0 in stage 8.0 (TID 64) +26/04/01 05:55:07 INFO Executor: Finished task 53.0 in stage 8.0 (TID 61). 2984 bytes result sent to driver +26/04/01 05:55:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:07 INFO TaskSetManager: Starting task 57.0 in stage 8.0 (TID 65) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:07 INFO TaskSetManager: Finished task 53.0 in stage 8.0 (TID 61) in 1676 ms on 10.0.0.133 (executor driver) (54/208) +26/04/01 05:55:07 INFO Executor: Running task 57.0 in stage 8.0 (TID 65) +26/04/01 05:55:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:07 INFO Executor: Finished task 54.0 in stage 8.0 (TID 62). 2984 bytes result sent to driver +26/04/01 05:55:07 INFO TaskSetManager: Starting task 58.0 in stage 8.0 (TID 66) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:07 INFO TaskSetManager: Finished task 54.0 in stage 8.0 (TID 62) in 1681 ms on 10.0.0.133 (executor driver) (55/208) +26/04/01 05:55:07 INFO Executor: Running task 58.0 in stage 8.0 (TID 66) +26/04/01 05:55:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:07 INFO Executor: Finished task 55.0 in stage 8.0 (TID 63). 2984 bytes result sent to driver +26/04/01 05:55:07 INFO TaskSetManager: Starting task 59.0 in stage 8.0 (TID 67) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:07 INFO Executor: Running task 59.0 in stage 8.0 (TID 67) +26/04/01 05:55:07 INFO TaskSetManager: Finished task 55.0 in stage 8.0 (TID 63) in 1677 ms on 10.0.0.133 (executor driver) (56/208) +26/04/01 05:55:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:09 INFO Executor: Finished task 56.0 in stage 8.0 (TID 64). 3027 bytes result sent to driver +26/04/01 05:55:09 INFO TaskSetManager: Starting task 60.0 in stage 8.0 (TID 68) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:09 INFO TaskSetManager: Finished task 56.0 in stage 8.0 (TID 64) in 1676 ms on 10.0.0.133 (executor driver) (57/208) +26/04/01 05:55:09 INFO Executor: Running task 60.0 in stage 8.0 (TID 68) +26/04/01 05:55:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:09 INFO Executor: Finished task 57.0 in stage 8.0 (TID 65). 2984 bytes result sent to driver +26/04/01 05:55:09 INFO TaskSetManager: Starting task 61.0 in stage 8.0 (TID 69) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:09 INFO Executor: Running task 61.0 in stage 8.0 (TID 69) +26/04/01 05:55:09 INFO TaskSetManager: Finished task 57.0 in stage 8.0 (TID 65) in 1678 ms on 10.0.0.133 (executor driver) (58/208) +26/04/01 05:55:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:09 INFO Executor: Finished task 58.0 in stage 8.0 (TID 66). 2984 bytes result sent to driver +26/04/01 05:55:09 INFO TaskSetManager: Starting task 62.0 in stage 8.0 (TID 70) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:09 INFO TaskSetManager: Finished task 58.0 in stage 8.0 (TID 66) in 1675 ms on 10.0.0.133 (executor driver) (59/208) +26/04/01 05:55:09 INFO Executor: Running task 62.0 in stage 8.0 (TID 70) +26/04/01 05:55:09 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:09 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:09 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:09 INFO Executor: Finished task 59.0 in stage 8.0 (TID 67). 3027 bytes result sent to driver +26/04/01 05:55:09 INFO TaskSetManager: Starting task 63.0 in stage 8.0 (TID 71) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:09 INFO TaskSetManager: Finished task 59.0 in stage 8.0 (TID 67) in 1685 ms on 10.0.0.133 (executor driver) (60/208) +26/04/01 05:55:09 INFO Executor: Running task 63.0 in stage 8.0 (TID 71) +26/04/01 05:55:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:09 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:11 INFO Executor: Finished task 61.0 in stage 8.0 (TID 69). 2984 bytes result sent to driver +26/04/01 05:55:11 INFO TaskSetManager: Starting task 64.0 in stage 8.0 (TID 72) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:11 INFO TaskSetManager: Finished task 61.0 in stage 8.0 (TID 69) in 1670 ms on 10.0.0.133 (executor driver) (61/208) +26/04/01 05:55:11 INFO Executor: Running task 64.0 in stage 8.0 (TID 72) +26/04/01 05:55:11 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:11 INFO Executor: Finished task 60.0 in stage 8.0 (TID 68). 3027 bytes result sent to driver +26/04/01 05:55:11 INFO TaskSetManager: Starting task 65.0 in stage 8.0 (TID 73) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:11 INFO TaskSetManager: Finished task 60.0 in stage 8.0 (TID 68) in 1681 ms on 10.0.0.133 (executor driver) (62/208) +26/04/01 05:55:11 INFO Executor: Running task 65.0 in stage 8.0 (TID 73) +26/04/01 05:55:11 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:11 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:11 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:11 INFO Executor: Finished task 62.0 in stage 8.0 (TID 70). 2984 bytes result sent to driver +26/04/01 05:55:11 INFO TaskSetManager: Starting task 66.0 in stage 8.0 (TID 74) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:11 INFO TaskSetManager: Finished task 62.0 in stage 8.0 (TID 70) in 1680 ms on 10.0.0.133 (executor driver) (63/208) +26/04/01 05:55:11 INFO Executor: Running task 66.0 in stage 8.0 (TID 74) +26/04/01 05:55:11 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:11 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:11 INFO Executor: Finished task 63.0 in stage 8.0 (TID 71). 2984 bytes result sent to driver +26/04/01 05:55:11 INFO TaskSetManager: Starting task 67.0 in stage 8.0 (TID 75) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:11 INFO Executor: Running task 67.0 in stage 8.0 (TID 75) +26/04/01 05:55:11 INFO TaskSetManager: Finished task 63.0 in stage 8.0 (TID 71) in 1681 ms on 10.0.0.133 (executor driver) (64/208) +26/04/01 05:55:11 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:11 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:12 INFO Executor: Finished task 64.0 in stage 8.0 (TID 72). 3027 bytes result sent to driver +26/04/01 05:55:12 INFO Executor: Finished task 65.0 in stage 8.0 (TID 73). 2984 bytes result sent to driver +26/04/01 05:55:12 INFO TaskSetManager: Starting task 68.0 in stage 8.0 (TID 76) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:12 INFO Executor: Running task 68.0 in stage 8.0 (TID 76) +26/04/01 05:55:12 INFO TaskSetManager: Finished task 64.0 in stage 8.0 (TID 72) in 1686 ms on 10.0.0.133 (executor driver) (65/208) +26/04/01 05:55:12 INFO TaskSetManager: Starting task 69.0 in stage 8.0 (TID 77) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:12 INFO Executor: Running task 69.0 in stage 8.0 (TID 77) +26/04/01 05:55:12 INFO TaskSetManager: Finished task 65.0 in stage 8.0 (TID 73) in 1683 ms on 10.0.0.133 (executor driver) (66/208) +26/04/01 05:55:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:12 INFO Executor: Finished task 66.0 in stage 8.0 (TID 74). 2984 bytes result sent to driver +26/04/01 05:55:12 INFO TaskSetManager: Starting task 70.0 in stage 8.0 (TID 78) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:12 INFO TaskSetManager: Finished task 66.0 in stage 8.0 (TID 74) in 1679 ms on 10.0.0.133 (executor driver) (67/208) +26/04/01 05:55:12 INFO Executor: Running task 70.0 in stage 8.0 (TID 78) +26/04/01 05:55:12 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:12 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:12 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:12 INFO Executor: Finished task 67.0 in stage 8.0 (TID 75). 2984 bytes result sent to driver +26/04/01 05:55:12 INFO TaskSetManager: Starting task 71.0 in stage 8.0 (TID 79) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:12 INFO TaskSetManager: Finished task 67.0 in stage 8.0 (TID 75) in 1682 ms on 10.0.0.133 (executor driver) (68/208) +26/04/01 05:55:12 INFO Executor: Running task 71.0 in stage 8.0 (TID 79) +26/04/01 05:55:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:12 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:14 INFO Executor: Finished task 68.0 in stage 8.0 (TID 76). 3027 bytes result sent to driver +26/04/01 05:55:14 INFO Executor: Finished task 70.0 in stage 8.0 (TID 78). 2984 bytes result sent to driver +26/04/01 05:55:14 INFO TaskSetManager: Starting task 72.0 in stage 8.0 (TID 80) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:14 INFO TaskSetManager: Starting task 73.0 in stage 8.0 (TID 81) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:14 INFO Executor: Running task 72.0 in stage 8.0 (TID 80) +26/04/01 05:55:14 INFO TaskSetManager: Finished task 68.0 in stage 8.0 (TID 76) in 1685 ms on 10.0.0.133 (executor driver) (69/208) +26/04/01 05:55:14 INFO Executor: Running task 73.0 in stage 8.0 (TID 81) +26/04/01 05:55:14 INFO TaskSetManager: Finished task 70.0 in stage 8.0 (TID 78) in 1679 ms on 10.0.0.133 (executor driver) (70/208) +26/04/01 05:55:14 INFO Executor: Finished task 69.0 in stage 8.0 (TID 77). 3027 bytes result sent to driver +26/04/01 05:55:14 INFO TaskSetManager: Starting task 74.0 in stage 8.0 (TID 82) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:14 INFO TaskSetManager: Finished task 69.0 in stage 8.0 (TID 77) in 1685 ms on 10.0.0.133 (executor driver) (71/208) +26/04/01 05:55:14 INFO Executor: Running task 74.0 in stage 8.0 (TID 82) +26/04/01 05:55:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:14 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:14 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:14 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:14 INFO Executor: Finished task 71.0 in stage 8.0 (TID 79). 2984 bytes result sent to driver +26/04/01 05:55:14 INFO TaskSetManager: Starting task 75.0 in stage 8.0 (TID 83) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:14 INFO TaskSetManager: Finished task 71.0 in stage 8.0 (TID 79) in 1680 ms on 10.0.0.133 (executor driver) (72/208) +26/04/01 05:55:14 INFO Executor: Running task 75.0 in stage 8.0 (TID 83) +26/04/01 05:55:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:14 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:16 INFO Executor: Finished task 73.0 in stage 8.0 (TID 81). 3027 bytes result sent to driver +26/04/01 05:55:16 INFO TaskSetManager: Starting task 76.0 in stage 8.0 (TID 84) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:16 INFO Executor: Running task 76.0 in stage 8.0 (TID 84) +26/04/01 05:55:16 INFO TaskSetManager: Finished task 73.0 in stage 8.0 (TID 81) in 1685 ms on 10.0.0.133 (executor driver) (73/208) +26/04/01 05:55:16 INFO Executor: Finished task 72.0 in stage 8.0 (TID 80). 3027 bytes result sent to driver +26/04/01 05:55:16 INFO TaskSetManager: Starting task 77.0 in stage 8.0 (TID 85) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:16 INFO TaskSetManager: Finished task 72.0 in stage 8.0 (TID 80) in 1686 ms on 10.0.0.133 (executor driver) (74/208) +26/04/01 05:55:16 INFO Executor: Running task 77.0 in stage 8.0 (TID 85) +26/04/01 05:55:16 INFO Executor: Finished task 74.0 in stage 8.0 (TID 82). 2984 bytes result sent to driver +26/04/01 05:55:16 INFO TaskSetManager: Starting task 78.0 in stage 8.0 (TID 86) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:16 INFO TaskSetManager: Finished task 74.0 in stage 8.0 (TID 82) in 1687 ms on 10.0.0.133 (executor driver) (75/208) +26/04/01 05:55:16 INFO Executor: Running task 78.0 in stage 8.0 (TID 86) +26/04/01 05:55:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:16 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:16 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:16 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:16 INFO Executor: Finished task 75.0 in stage 8.0 (TID 83). 2984 bytes result sent to driver +26/04/01 05:55:16 INFO TaskSetManager: Starting task 79.0 in stage 8.0 (TID 87) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:16 INFO TaskSetManager: Finished task 75.0 in stage 8.0 (TID 83) in 1683 ms on 10.0.0.133 (executor driver) (76/208) +26/04/01 05:55:16 INFO Executor: Running task 79.0 in stage 8.0 (TID 87) +26/04/01 05:55:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:16 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:17 INFO Executor: Finished task 78.0 in stage 8.0 (TID 86). 2984 bytes result sent to driver +26/04/01 05:55:17 INFO Executor: Finished task 76.0 in stage 8.0 (TID 84). 3027 bytes result sent to driver +26/04/01 05:55:17 INFO TaskSetManager: Starting task 80.0 in stage 8.0 (TID 88) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:17 INFO TaskSetManager: Starting task 81.0 in stage 8.0 (TID 89) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:17 INFO Executor: Running task 80.0 in stage 8.0 (TID 88) +26/04/01 05:55:17 INFO TaskSetManager: Finished task 78.0 in stage 8.0 (TID 86) in 1674 ms on 10.0.0.133 (executor driver) (77/208) +26/04/01 05:55:17 INFO Executor: Running task 81.0 in stage 8.0 (TID 89) +26/04/01 05:55:17 INFO TaskSetManager: Finished task 76.0 in stage 8.0 (TID 84) in 1677 ms on 10.0.0.133 (executor driver) (78/208) +26/04/01 05:55:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:17 INFO Executor: Finished task 77.0 in stage 8.0 (TID 85). 2984 bytes result sent to driver +26/04/01 05:55:17 INFO TaskSetManager: Starting task 82.0 in stage 8.0 (TID 90) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:17 INFO TaskSetManager: Finished task 77.0 in stage 8.0 (TID 85) in 1683 ms on 10.0.0.133 (executor driver) (79/208) +26/04/01 05:55:17 INFO Executor: Running task 82.0 in stage 8.0 (TID 90) +26/04/01 05:55:17 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:17 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:17 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:17 INFO Executor: Finished task 79.0 in stage 8.0 (TID 87). 2984 bytes result sent to driver +26/04/01 05:55:17 INFO TaskSetManager: Starting task 83.0 in stage 8.0 (TID 91) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:17 INFO TaskSetManager: Finished task 79.0 in stage 8.0 (TID 87) in 1683 ms on 10.0.0.133 (executor driver) (80/208) +26/04/01 05:55:17 INFO Executor: Running task 83.0 in stage 8.0 (TID 91) +26/04/01 05:55:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:17 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:19 INFO Executor: Finished task 80.0 in stage 8.0 (TID 88). 3027 bytes result sent to driver +26/04/01 05:55:19 INFO TaskSetManager: Starting task 84.0 in stage 8.0 (TID 92) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:19 INFO TaskSetManager: Finished task 80.0 in stage 8.0 (TID 88) in 1680 ms on 10.0.0.133 (executor driver) (81/208) +26/04/01 05:55:19 INFO Executor: Running task 84.0 in stage 8.0 (TID 92) +26/04/01 05:55:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:19 INFO Executor: Finished task 81.0 in stage 8.0 (TID 89). 3027 bytes result sent to driver +26/04/01 05:55:19 INFO TaskSetManager: Starting task 85.0 in stage 8.0 (TID 93) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:19 INFO TaskSetManager: Finished task 81.0 in stage 8.0 (TID 89) in 1686 ms on 10.0.0.133 (executor driver) (82/208) +26/04/01 05:55:19 INFO Executor: Running task 85.0 in stage 8.0 (TID 93) +26/04/01 05:55:19 INFO Executor: Finished task 82.0 in stage 8.0 (TID 90). 2984 bytes result sent to driver +26/04/01 05:55:19 INFO TaskSetManager: Starting task 86.0 in stage 8.0 (TID 94) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:19 INFO TaskSetManager: Finished task 82.0 in stage 8.0 (TID 90) in 1682 ms on 10.0.0.133 (executor driver) (83/208) +26/04/01 05:55:19 INFO Executor: Running task 86.0 in stage 8.0 (TID 94) +26/04/01 05:55:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:19 INFO Executor: Finished task 83.0 in stage 8.0 (TID 91). 3027 bytes result sent to driver +26/04/01 05:55:19 INFO TaskSetManager: Starting task 87.0 in stage 8.0 (TID 95) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:19 INFO TaskSetManager: Finished task 83.0 in stage 8.0 (TID 91) in 1677 ms on 10.0.0.133 (executor driver) (84/208) +26/04/01 05:55:19 INFO Executor: Running task 87.0 in stage 8.0 (TID 95) +26/04/01 05:55:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:21 INFO Executor: Finished task 84.0 in stage 8.0 (TID 92). 3027 bytes result sent to driver +26/04/01 05:55:21 INFO TaskSetManager: Starting task 88.0 in stage 8.0 (TID 96) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:21 INFO TaskSetManager: Finished task 84.0 in stage 8.0 (TID 92) in 1678 ms on 10.0.0.133 (executor driver) (85/208) +26/04/01 05:55:21 INFO Executor: Running task 88.0 in stage 8.0 (TID 96) +26/04/01 05:55:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:21 INFO Executor: Finished task 86.0 in stage 8.0 (TID 94). 2984 bytes result sent to driver +26/04/01 05:55:21 INFO TaskSetManager: Starting task 89.0 in stage 8.0 (TID 97) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:21 INFO TaskSetManager: Finished task 86.0 in stage 8.0 (TID 94) in 1679 ms on 10.0.0.133 (executor driver) (86/208) +26/04/01 05:55:21 INFO Executor: Running task 89.0 in stage 8.0 (TID 97) +26/04/01 05:55:21 INFO Executor: Finished task 85.0 in stage 8.0 (TID 93). 2984 bytes result sent to driver +26/04/01 05:55:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:21 INFO TaskSetManager: Starting task 90.0 in stage 8.0 (TID 98) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:21 INFO Executor: Running task 90.0 in stage 8.0 (TID 98) +26/04/01 05:55:21 INFO TaskSetManager: Finished task 85.0 in stage 8.0 (TID 93) in 1681 ms on 10.0.0.133 (executor driver) (87/208) +26/04/01 05:55:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:21 INFO Executor: Finished task 87.0 in stage 8.0 (TID 95). 2984 bytes result sent to driver +26/04/01 05:55:21 INFO TaskSetManager: Starting task 91.0 in stage 8.0 (TID 99) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:21 INFO TaskSetManager: Finished task 87.0 in stage 8.0 (TID 95) in 1680 ms on 10.0.0.133 (executor driver) (88/208) +26/04/01 05:55:21 INFO Executor: Running task 91.0 in stage 8.0 (TID 99) +26/04/01 05:55:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:22 INFO Executor: Finished task 90.0 in stage 8.0 (TID 98). 2984 bytes result sent to driver +26/04/01 05:55:22 INFO TaskSetManager: Starting task 92.0 in stage 8.0 (TID 100) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:22 INFO Executor: Running task 92.0 in stage 8.0 (TID 100) +26/04/01 05:55:22 INFO TaskSetManager: Finished task 90.0 in stage 8.0 (TID 98) in 1673 ms on 10.0.0.133 (executor driver) (89/208) +26/04/01 05:55:22 INFO Executor: Finished task 89.0 in stage 8.0 (TID 97). 2984 bytes result sent to driver +26/04/01 05:55:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:22 INFO TaskSetManager: Starting task 93.0 in stage 8.0 (TID 101) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:22 INFO TaskSetManager: Finished task 89.0 in stage 8.0 (TID 97) in 1678 ms on 10.0.0.133 (executor driver) (90/208) +26/04/01 05:55:22 INFO Executor: Running task 93.0 in stage 8.0 (TID 101) +26/04/01 05:55:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:22 INFO Executor: Finished task 88.0 in stage 8.0 (TID 96). 3027 bytes result sent to driver +26/04/01 05:55:22 INFO TaskSetManager: Starting task 94.0 in stage 8.0 (TID 102) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:22 INFO Executor: Running task 94.0 in stage 8.0 (TID 102) +26/04/01 05:55:22 INFO TaskSetManager: Finished task 88.0 in stage 8.0 (TID 96) in 1692 ms on 10.0.0.133 (executor driver) (91/208) +26/04/01 05:55:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:22 INFO Executor: Finished task 91.0 in stage 8.0 (TID 99). 2984 bytes result sent to driver +26/04/01 05:55:22 INFO TaskSetManager: Starting task 95.0 in stage 8.0 (TID 103) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:22 INFO TaskSetManager: Finished task 91.0 in stage 8.0 (TID 99) in 1694 ms on 10.0.0.133 (executor driver) (92/208) +26/04/01 05:55:22 INFO Executor: Running task 95.0 in stage 8.0 (TID 103) +26/04/01 05:55:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:24 INFO Executor: Finished task 93.0 in stage 8.0 (TID 101). 2984 bytes result sent to driver +26/04/01 05:55:24 INFO TaskSetManager: Starting task 96.0 in stage 8.0 (TID 104) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:24 INFO Executor: Running task 96.0 in stage 8.0 (TID 104) +26/04/01 05:55:24 INFO TaskSetManager: Finished task 93.0 in stage 8.0 (TID 101) in 1676 ms on 10.0.0.133 (executor driver) (93/208) +26/04/01 05:55:24 INFO Executor: Finished task 92.0 in stage 8.0 (TID 100). 3027 bytes result sent to driver +26/04/01 05:55:24 INFO TaskSetManager: Starting task 97.0 in stage 8.0 (TID 105) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:24 INFO Executor: Finished task 94.0 in stage 8.0 (TID 102). 2984 bytes result sent to driver +26/04/01 05:55:24 INFO TaskSetManager: Finished task 92.0 in stage 8.0 (TID 100) in 1682 ms on 10.0.0.133 (executor driver) (94/208) +26/04/01 05:55:24 INFO Executor: Running task 97.0 in stage 8.0 (TID 105) +26/04/01 05:55:24 INFO TaskSetManager: Starting task 98.0 in stage 8.0 (TID 106) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:24 INFO TaskSetManager: Finished task 94.0 in stage 8.0 (TID 102) in 1673 ms on 10.0.0.133 (executor driver) (95/208) +26/04/01 05:55:24 INFO Executor: Running task 98.0 in stage 8.0 (TID 106) +26/04/01 05:55:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:24 INFO Executor: Finished task 95.0 in stage 8.0 (TID 103). 3027 bytes result sent to driver +26/04/01 05:55:24 INFO TaskSetManager: Starting task 99.0 in stage 8.0 (TID 107) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:24 INFO TaskSetManager: Finished task 95.0 in stage 8.0 (TID 103) in 1685 ms on 10.0.0.133 (executor driver) (96/208) +26/04/01 05:55:24 INFO Executor: Running task 99.0 in stage 8.0 (TID 107) +26/04/01 05:55:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:26 INFO Executor: Finished task 97.0 in stage 8.0 (TID 105). 2984 bytes result sent to driver +26/04/01 05:55:26 INFO TaskSetManager: Starting task 100.0 in stage 8.0 (TID 108) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:26 INFO TaskSetManager: Finished task 97.0 in stage 8.0 (TID 105) in 1674 ms on 10.0.0.133 (executor driver) (97/208) +26/04/01 05:55:26 INFO Executor: Running task 100.0 in stage 8.0 (TID 108) +26/04/01 05:55:26 INFO Executor: Finished task 98.0 in stage 8.0 (TID 106). 2984 bytes result sent to driver +26/04/01 05:55:26 INFO TaskSetManager: Starting task 101.0 in stage 8.0 (TID 109) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:26 INFO Executor: Running task 101.0 in stage 8.0 (TID 109) +26/04/01 05:55:26 INFO TaskSetManager: Finished task 98.0 in stage 8.0 (TID 106) in 1676 ms on 10.0.0.133 (executor driver) (98/208) +26/04/01 05:55:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:26 INFO Executor: Finished task 96.0 in stage 8.0 (TID 104). 3027 bytes result sent to driver +26/04/01 05:55:26 INFO TaskSetManager: Starting task 102.0 in stage 8.0 (TID 110) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:26 INFO TaskSetManager: Finished task 96.0 in stage 8.0 (TID 104) in 1681 ms on 10.0.0.133 (executor driver) (99/208) +26/04/01 05:55:26 INFO Executor: Running task 102.0 in stage 8.0 (TID 110) +26/04/01 05:55:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:26 INFO Executor: Finished task 99.0 in stage 8.0 (TID 107). 2984 bytes result sent to driver +26/04/01 05:55:26 INFO TaskSetManager: Starting task 103.0 in stage 8.0 (TID 111) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:26 INFO TaskSetManager: Finished task 99.0 in stage 8.0 (TID 107) in 1680 ms on 10.0.0.133 (executor driver) (100/208) +26/04/01 05:55:26 INFO Executor: Running task 103.0 in stage 8.0 (TID 111) +26/04/01 05:55:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:27 INFO Executor: Finished task 100.0 in stage 8.0 (TID 108). 3027 bytes result sent to driver +26/04/01 05:55:27 INFO Executor: Finished task 101.0 in stage 8.0 (TID 109). 2984 bytes result sent to driver +26/04/01 05:55:27 INFO TaskSetManager: Starting task 104.0 in stage 8.0 (TID 112) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:27 INFO Executor: Running task 104.0 in stage 8.0 (TID 112) +26/04/01 05:55:27 INFO TaskSetManager: Starting task 105.0 in stage 8.0 (TID 113) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:27 INFO Executor: Running task 105.0 in stage 8.0 (TID 113) +26/04/01 05:55:27 INFO TaskSetManager: Finished task 100.0 in stage 8.0 (TID 108) in 1677 ms on 10.0.0.133 (executor driver) (101/208) +26/04/01 05:55:27 INFO TaskSetManager: Finished task 101.0 in stage 8.0 (TID 109) in 1674 ms on 10.0.0.133 (executor driver) (102/208) +26/04/01 05:55:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:27 INFO Executor: Finished task 102.0 in stage 8.0 (TID 110). 2984 bytes result sent to driver +26/04/01 05:55:27 INFO TaskSetManager: Starting task 106.0 in stage 8.0 (TID 114) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:27 INFO TaskSetManager: Finished task 102.0 in stage 8.0 (TID 110) in 1678 ms on 10.0.0.133 (executor driver) (103/208) +26/04/01 05:55:27 INFO Executor: Running task 106.0 in stage 8.0 (TID 114) +26/04/01 05:55:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:28 INFO Executor: Finished task 103.0 in stage 8.0 (TID 111). 3027 bytes result sent to driver +26/04/01 05:55:28 INFO TaskSetManager: Starting task 107.0 in stage 8.0 (TID 115) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:28 INFO TaskSetManager: Finished task 103.0 in stage 8.0 (TID 111) in 1675 ms on 10.0.0.133 (executor driver) (104/208) +26/04/01 05:55:28 INFO Executor: Running task 107.0 in stage 8.0 (TID 115) +26/04/01 05:55:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:28 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:29 INFO Executor: Finished task 105.0 in stage 8.0 (TID 113). 3027 bytes result sent to driver +26/04/01 05:55:29 INFO TaskSetManager: Starting task 108.0 in stage 8.0 (TID 116) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:29 INFO Executor: Running task 108.0 in stage 8.0 (TID 116) +26/04/01 05:55:29 INFO TaskSetManager: Finished task 105.0 in stage 8.0 (TID 113) in 1675 ms on 10.0.0.133 (executor driver) (105/208) +26/04/01 05:55:29 INFO Executor: Finished task 104.0 in stage 8.0 (TID 112). 3027 bytes result sent to driver +26/04/01 05:55:29 INFO TaskSetManager: Starting task 109.0 in stage 8.0 (TID 117) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:29 INFO TaskSetManager: Finished task 104.0 in stage 8.0 (TID 112) in 1676 ms on 10.0.0.133 (executor driver) (106/208) +26/04/01 05:55:29 INFO Executor: Running task 109.0 in stage 8.0 (TID 117) +26/04/01 05:55:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:29 INFO Executor: Finished task 106.0 in stage 8.0 (TID 114). 2984 bytes result sent to driver +26/04/01 05:55:29 INFO TaskSetManager: Starting task 110.0 in stage 8.0 (TID 118) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:29 INFO TaskSetManager: Finished task 106.0 in stage 8.0 (TID 114) in 1674 ms on 10.0.0.133 (executor driver) (107/208) +26/04/01 05:55:29 INFO Executor: Running task 110.0 in stage 8.0 (TID 118) +26/04/01 05:55:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:29 INFO Executor: Finished task 107.0 in stage 8.0 (TID 115). 3027 bytes result sent to driver +26/04/01 05:55:29 INFO TaskSetManager: Starting task 111.0 in stage 8.0 (TID 119) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:29 INFO TaskSetManager: Finished task 107.0 in stage 8.0 (TID 115) in 1677 ms on 10.0.0.133 (executor driver) (108/208) +26/04/01 05:55:29 INFO Executor: Running task 111.0 in stage 8.0 (TID 119) +26/04/01 05:55:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:31 INFO Executor: Finished task 108.0 in stage 8.0 (TID 116). 3027 bytes result sent to driver +26/04/01 05:55:31 INFO Executor: Finished task 109.0 in stage 8.0 (TID 117). 2984 bytes result sent to driver +26/04/01 05:55:31 INFO TaskSetManager: Starting task 112.0 in stage 8.0 (TID 120) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:31 INFO TaskSetManager: Starting task 113.0 in stage 8.0 (TID 121) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:31 INFO Executor: Running task 112.0 in stage 8.0 (TID 120) +26/04/01 05:55:31 INFO TaskSetManager: Finished task 108.0 in stage 8.0 (TID 116) in 1674 ms on 10.0.0.133 (executor driver) (109/208) +26/04/01 05:55:31 INFO Executor: Running task 113.0 in stage 8.0 (TID 121) +26/04/01 05:55:31 INFO TaskSetManager: Finished task 109.0 in stage 8.0 (TID 117) in 1672 ms on 10.0.0.133 (executor driver) (110/208) +26/04/01 05:55:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:31 INFO Executor: Finished task 110.0 in stage 8.0 (TID 118). 2984 bytes result sent to driver +26/04/01 05:55:31 INFO TaskSetManager: Starting task 114.0 in stage 8.0 (TID 122) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:31 INFO Executor: Running task 114.0 in stage 8.0 (TID 122) +26/04/01 05:55:31 INFO TaskSetManager: Finished task 110.0 in stage 8.0 (TID 118) in 1675 ms on 10.0.0.133 (executor driver) (111/208) +26/04/01 05:55:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:31 INFO Executor: Finished task 111.0 in stage 8.0 (TID 119). 3027 bytes result sent to driver +26/04/01 05:55:31 INFO TaskSetManager: Starting task 115.0 in stage 8.0 (TID 123) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:31 INFO TaskSetManager: Finished task 111.0 in stage 8.0 (TID 119) in 1667 ms on 10.0.0.133 (executor driver) (112/208) +26/04/01 05:55:31 INFO Executor: Running task 115.0 in stage 8.0 (TID 123) +26/04/01 05:55:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:32 INFO Executor: Finished task 113.0 in stage 8.0 (TID 121). 3027 bytes result sent to driver +26/04/01 05:55:32 INFO TaskSetManager: Starting task 116.0 in stage 8.0 (TID 124) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:32 INFO TaskSetManager: Finished task 113.0 in stage 8.0 (TID 121) in 1675 ms on 10.0.0.133 (executor driver) (113/208) +26/04/01 05:55:32 INFO Executor: Running task 116.0 in stage 8.0 (TID 124) +26/04/01 05:55:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:32 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:33 INFO Executor: Finished task 112.0 in stage 8.0 (TID 120). 3027 bytes result sent to driver +26/04/01 05:55:33 INFO TaskSetManager: Starting task 117.0 in stage 8.0 (TID 125) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:33 INFO TaskSetManager: Finished task 112.0 in stage 8.0 (TID 120) in 1702 ms on 10.0.0.133 (executor driver) (114/208) +26/04/01 05:55:33 INFO Executor: Running task 117.0 in stage 8.0 (TID 125) +26/04/01 05:55:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:33 INFO Executor: Finished task 115.0 in stage 8.0 (TID 123). 3027 bytes result sent to driver +26/04/01 05:55:33 INFO TaskSetManager: Starting task 118.0 in stage 8.0 (TID 126) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:33 INFO TaskSetManager: Finished task 115.0 in stage 8.0 (TID 123) in 1678 ms on 10.0.0.133 (executor driver) (115/208) +26/04/01 05:55:33 INFO Executor: Running task 118.0 in stage 8.0 (TID 126) +26/04/01 05:55:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:33 INFO Executor: Finished task 114.0 in stage 8.0 (TID 122). 2984 bytes result sent to driver +26/04/01 05:55:33 INFO TaskSetManager: Starting task 119.0 in stage 8.0 (TID 127) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:33 INFO TaskSetManager: Finished task 114.0 in stage 8.0 (TID 122) in 1710 ms on 10.0.0.133 (executor driver) (116/208) +26/04/01 05:55:33 INFO Executor: Running task 119.0 in stage 8.0 (TID 127) +26/04/01 05:55:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:34 INFO Executor: Finished task 116.0 in stage 8.0 (TID 124). 3027 bytes result sent to driver +26/04/01 05:55:34 INFO TaskSetManager: Starting task 120.0 in stage 8.0 (TID 128) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:34 INFO Executor: Running task 120.0 in stage 8.0 (TID 128) +26/04/01 05:55:34 INFO TaskSetManager: Finished task 116.0 in stage 8.0 (TID 124) in 1674 ms on 10.0.0.133 (executor driver) (117/208) +26/04/01 05:55:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:34 INFO Executor: Finished task 117.0 in stage 8.0 (TID 125). 2984 bytes result sent to driver +26/04/01 05:55:34 INFO TaskSetManager: Starting task 121.0 in stage 8.0 (TID 129) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:34 INFO TaskSetManager: Finished task 117.0 in stage 8.0 (TID 125) in 1672 ms on 10.0.0.133 (executor driver) (118/208) +26/04/01 05:55:34 INFO Executor: Running task 121.0 in stage 8.0 (TID 129) +26/04/01 05:55:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:34 INFO Executor: Finished task 118.0 in stage 8.0 (TID 126). 2984 bytes result sent to driver +26/04/01 05:55:34 INFO TaskSetManager: Starting task 122.0 in stage 8.0 (TID 130) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:34 INFO Executor: Running task 122.0 in stage 8.0 (TID 130) +26/04/01 05:55:34 INFO TaskSetManager: Finished task 118.0 in stage 8.0 (TID 126) in 1676 ms on 10.0.0.133 (executor driver) (119/208) +26/04/01 05:55:34 INFO Executor: Finished task 119.0 in stage 8.0 (TID 127). 3027 bytes result sent to driver +26/04/01 05:55:34 INFO TaskSetManager: Starting task 123.0 in stage 8.0 (TID 131) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:34 INFO Executor: Running task 123.0 in stage 8.0 (TID 131) +26/04/01 05:55:34 INFO TaskSetManager: Finished task 119.0 in stage 8.0 (TID 127) in 1674 ms on 10.0.0.133 (executor driver) (120/208) +26/04/01 05:55:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:36 INFO Executor: Finished task 120.0 in stage 8.0 (TID 128). 3027 bytes result sent to driver +26/04/01 05:55:36 INFO TaskSetManager: Starting task 124.0 in stage 8.0 (TID 132) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:36 INFO Executor: Running task 124.0 in stage 8.0 (TID 132) +26/04/01 05:55:36 INFO TaskSetManager: Finished task 120.0 in stage 8.0 (TID 128) in 1675 ms on 10.0.0.133 (executor driver) (121/208) +26/04/01 05:55:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:36 INFO Executor: Finished task 121.0 in stage 8.0 (TID 129). 3027 bytes result sent to driver +26/04/01 05:55:36 INFO TaskSetManager: Starting task 125.0 in stage 8.0 (TID 133) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:36 INFO TaskSetManager: Finished task 121.0 in stage 8.0 (TID 129) in 1670 ms on 10.0.0.133 (executor driver) (122/208) +26/04/01 05:55:36 INFO Executor: Running task 125.0 in stage 8.0 (TID 133) +26/04/01 05:55:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:36 INFO Executor: Finished task 123.0 in stage 8.0 (TID 131). 2984 bytes result sent to driver +26/04/01 05:55:36 INFO TaskSetManager: Starting task 126.0 in stage 8.0 (TID 134) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:36 INFO TaskSetManager: Finished task 123.0 in stage 8.0 (TID 131) in 1669 ms on 10.0.0.133 (executor driver) (123/208) +26/04/01 05:55:36 INFO Executor: Running task 126.0 in stage 8.0 (TID 134) +26/04/01 05:55:36 INFO Executor: Finished task 122.0 in stage 8.0 (TID 130). 2984 bytes result sent to driver +26/04/01 05:55:36 INFO TaskSetManager: Starting task 127.0 in stage 8.0 (TID 135) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:36 INFO Executor: Running task 127.0 in stage 8.0 (TID 135) +26/04/01 05:55:36 INFO TaskSetManager: Finished task 122.0 in stage 8.0 (TID 130) in 1673 ms on 10.0.0.133 (executor driver) (124/208) +26/04/01 05:55:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:38 INFO Executor: Finished task 124.0 in stage 8.0 (TID 132). 3027 bytes result sent to driver +26/04/01 05:55:38 INFO TaskSetManager: Starting task 128.0 in stage 8.0 (TID 136) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:38 INFO Executor: Running task 128.0 in stage 8.0 (TID 136) +26/04/01 05:55:38 INFO TaskSetManager: Finished task 124.0 in stage 8.0 (TID 132) in 1671 ms on 10.0.0.133 (executor driver) (125/208) +26/04/01 05:55:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:38 INFO Executor: Finished task 125.0 in stage 8.0 (TID 133). 2984 bytes result sent to driver +26/04/01 05:55:38 INFO TaskSetManager: Starting task 129.0 in stage 8.0 (TID 137) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:38 INFO TaskSetManager: Finished task 125.0 in stage 8.0 (TID 133) in 1665 ms on 10.0.0.133 (executor driver) (126/208) +26/04/01 05:55:38 INFO Executor: Running task 129.0 in stage 8.0 (TID 137) +26/04/01 05:55:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:38 INFO Executor: Finished task 126.0 in stage 8.0 (TID 134). 2984 bytes result sent to driver +26/04/01 05:55:38 INFO TaskSetManager: Starting task 130.0 in stage 8.0 (TID 138) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:38 INFO TaskSetManager: Finished task 126.0 in stage 8.0 (TID 134) in 1673 ms on 10.0.0.133 (executor driver) (127/208) +26/04/01 05:55:38 INFO Executor: Running task 130.0 in stage 8.0 (TID 138) +26/04/01 05:55:38 INFO Executor: Finished task 127.0 in stage 8.0 (TID 135). 3027 bytes result sent to driver +26/04/01 05:55:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:38 INFO TaskSetManager: Starting task 131.0 in stage 8.0 (TID 139) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:38 INFO Executor: Running task 131.0 in stage 8.0 (TID 139) +26/04/01 05:55:38 INFO TaskSetManager: Finished task 127.0 in stage 8.0 (TID 135) in 1673 ms on 10.0.0.133 (executor driver) (128/208) +26/04/01 05:55:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:39 INFO Executor: Finished task 128.0 in stage 8.0 (TID 136). 3027 bytes result sent to driver +26/04/01 05:55:39 INFO TaskSetManager: Starting task 132.0 in stage 8.0 (TID 140) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:39 INFO TaskSetManager: Finished task 128.0 in stage 8.0 (TID 136) in 1680 ms on 10.0.0.133 (executor driver) (129/208) +26/04/01 05:55:39 INFO Executor: Running task 132.0 in stage 8.0 (TID 140) +26/04/01 05:55:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:39 INFO Executor: Finished task 129.0 in stage 8.0 (TID 137). 2984 bytes result sent to driver +26/04/01 05:55:39 INFO TaskSetManager: Starting task 133.0 in stage 8.0 (TID 141) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:39 INFO TaskSetManager: Finished task 129.0 in stage 8.0 (TID 137) in 1682 ms on 10.0.0.133 (executor driver) (130/208) +26/04/01 05:55:39 INFO Executor: Running task 133.0 in stage 8.0 (TID 141) +26/04/01 05:55:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:39 INFO Executor: Finished task 130.0 in stage 8.0 (TID 138). 3027 bytes result sent to driver +26/04/01 05:55:39 INFO TaskSetManager: Starting task 134.0 in stage 8.0 (TID 142) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:39 INFO Executor: Finished task 131.0 in stage 8.0 (TID 139). 2984 bytes result sent to driver +26/04/01 05:55:39 INFO Executor: Running task 134.0 in stage 8.0 (TID 142) +26/04/01 05:55:39 INFO TaskSetManager: Starting task 135.0 in stage 8.0 (TID 143) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:39 INFO TaskSetManager: Finished task 130.0 in stage 8.0 (TID 138) in 1677 ms on 10.0.0.133 (executor driver) (131/208) +26/04/01 05:55:39 INFO Executor: Running task 135.0 in stage 8.0 (TID 143) +26/04/01 05:55:39 INFO TaskSetManager: Finished task 131.0 in stage 8.0 (TID 139) in 1673 ms on 10.0.0.133 (executor driver) (132/208) +26/04/01 05:55:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:41 INFO Executor: Finished task 133.0 in stage 8.0 (TID 141). 2984 bytes result sent to driver +26/04/01 05:55:41 INFO TaskSetManager: Starting task 136.0 in stage 8.0 (TID 144) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:41 INFO TaskSetManager: Finished task 133.0 in stage 8.0 (TID 141) in 1676 ms on 10.0.0.133 (executor driver) (133/208) +26/04/01 05:55:41 INFO Executor: Running task 136.0 in stage 8.0 (TID 144) +26/04/01 05:55:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:41 INFO Executor: Finished task 134.0 in stage 8.0 (TID 142). 2984 bytes result sent to driver +26/04/01 05:55:41 INFO TaskSetManager: Starting task 137.0 in stage 8.0 (TID 145) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:41 INFO TaskSetManager: Finished task 134.0 in stage 8.0 (TID 142) in 1673 ms on 10.0.0.133 (executor driver) (134/208) +26/04/01 05:55:41 INFO Executor: Running task 137.0 in stage 8.0 (TID 145) +26/04/01 05:55:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:41 INFO Executor: Finished task 132.0 in stage 8.0 (TID 140). 3027 bytes result sent to driver +26/04/01 05:55:41 INFO TaskSetManager: Starting task 138.0 in stage 8.0 (TID 146) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:41 INFO TaskSetManager: Finished task 132.0 in stage 8.0 (TID 140) in 1739 ms on 10.0.0.133 (executor driver) (135/208) +26/04/01 05:55:41 INFO Executor: Running task 138.0 in stage 8.0 (TID 146) +26/04/01 05:55:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:41 INFO Executor: Finished task 135.0 in stage 8.0 (TID 143). 2984 bytes result sent to driver +26/04/01 05:55:41 INFO TaskSetManager: Starting task 139.0 in stage 8.0 (TID 147) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:41 INFO TaskSetManager: Finished task 135.0 in stage 8.0 (TID 143) in 1734 ms on 10.0.0.133 (executor driver) (136/208) +26/04/01 05:55:41 INFO Executor: Running task 139.0 in stage 8.0 (TID 147) +26/04/01 05:55:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:43 INFO Executor: Finished task 136.0 in stage 8.0 (TID 144). 3027 bytes result sent to driver +26/04/01 05:55:43 INFO TaskSetManager: Starting task 140.0 in stage 8.0 (TID 148) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:43 INFO TaskSetManager: Finished task 136.0 in stage 8.0 (TID 144) in 1679 ms on 10.0.0.133 (executor driver) (137/208) +26/04/01 05:55:43 INFO Executor: Running task 140.0 in stage 8.0 (TID 148) +26/04/01 05:55:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:43 INFO Executor: Finished task 137.0 in stage 8.0 (TID 145). 3027 bytes result sent to driver +26/04/01 05:55:43 INFO TaskSetManager: Starting task 141.0 in stage 8.0 (TID 149) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:43 INFO TaskSetManager: Finished task 137.0 in stage 8.0 (TID 145) in 1675 ms on 10.0.0.133 (executor driver) (138/208) +26/04/01 05:55:43 INFO Executor: Running task 141.0 in stage 8.0 (TID 149) +26/04/01 05:55:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:43 INFO Executor: Finished task 138.0 in stage 8.0 (TID 146). 3027 bytes result sent to driver +26/04/01 05:55:43 INFO TaskSetManager: Starting task 142.0 in stage 8.0 (TID 150) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:43 INFO TaskSetManager: Finished task 138.0 in stage 8.0 (TID 146) in 1673 ms on 10.0.0.133 (executor driver) (139/208) +26/04/01 05:55:43 INFO Executor: Running task 142.0 in stage 8.0 (TID 150) +26/04/01 05:55:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:43 INFO Executor: Finished task 139.0 in stage 8.0 (TID 147). 2984 bytes result sent to driver +26/04/01 05:55:43 INFO TaskSetManager: Starting task 143.0 in stage 8.0 (TID 151) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:43 INFO TaskSetManager: Finished task 139.0 in stage 8.0 (TID 147) in 1675 ms on 10.0.0.133 (executor driver) (140/208) +26/04/01 05:55:43 INFO Executor: Running task 143.0 in stage 8.0 (TID 151) +26/04/01 05:55:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:44 INFO Executor: Finished task 141.0 in stage 8.0 (TID 149). 2984 bytes result sent to driver +26/04/01 05:55:44 INFO TaskSetManager: Starting task 144.0 in stage 8.0 (TID 152) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:44 INFO TaskSetManager: Finished task 141.0 in stage 8.0 (TID 149) in 1674 ms on 10.0.0.133 (executor driver) (141/208) +26/04/01 05:55:44 INFO Executor: Running task 144.0 in stage 8.0 (TID 152) +26/04/01 05:55:44 INFO Executor: Finished task 140.0 in stage 8.0 (TID 148). 3027 bytes result sent to driver +26/04/01 05:55:44 INFO TaskSetManager: Starting task 145.0 in stage 8.0 (TID 153) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:44 INFO TaskSetManager: Finished task 140.0 in stage 8.0 (TID 148) in 1683 ms on 10.0.0.133 (executor driver) (142/208) +26/04/01 05:55:44 INFO Executor: Running task 145.0 in stage 8.0 (TID 153) +26/04/01 05:55:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:44 INFO Executor: Finished task 142.0 in stage 8.0 (TID 150). 2984 bytes result sent to driver +26/04/01 05:55:44 INFO TaskSetManager: Starting task 146.0 in stage 8.0 (TID 154) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:44 INFO Executor: Running task 146.0 in stage 8.0 (TID 154) +26/04/01 05:55:44 INFO TaskSetManager: Finished task 142.0 in stage 8.0 (TID 150) in 1679 ms on 10.0.0.133 (executor driver) (143/208) +26/04/01 05:55:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:44 INFO Executor: Finished task 143.0 in stage 8.0 (TID 151). 2984 bytes result sent to driver +26/04/01 05:55:44 INFO TaskSetManager: Starting task 147.0 in stage 8.0 (TID 155) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:44 INFO TaskSetManager: Finished task 143.0 in stage 8.0 (TID 151) in 1672 ms on 10.0.0.133 (executor driver) (144/208) +26/04/01 05:55:44 INFO Executor: Running task 147.0 in stage 8.0 (TID 155) +26/04/01 05:55:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:46 INFO Executor: Finished task 144.0 in stage 8.0 (TID 152). 3027 bytes result sent to driver +26/04/01 05:55:46 INFO TaskSetManager: Starting task 148.0 in stage 8.0 (TID 156) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:46 INFO TaskSetManager: Finished task 144.0 in stage 8.0 (TID 152) in 1710 ms on 10.0.0.133 (executor driver) (145/208) +26/04/01 05:55:46 INFO Executor: Running task 148.0 in stage 8.0 (TID 156) +26/04/01 05:55:46 INFO Executor: Finished task 145.0 in stage 8.0 (TID 153). 2984 bytes result sent to driver +26/04/01 05:55:46 INFO TaskSetManager: Starting task 149.0 in stage 8.0 (TID 157) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:46 INFO TaskSetManager: Finished task 145.0 in stage 8.0 (TID 153) in 1712 ms on 10.0.0.133 (executor driver) (146/208) +26/04/01 05:55:46 INFO Executor: Running task 149.0 in stage 8.0 (TID 157) +26/04/01 05:55:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:46 INFO Executor: Finished task 146.0 in stage 8.0 (TID 154). 2984 bytes result sent to driver +26/04/01 05:55:46 INFO TaskSetManager: Starting task 150.0 in stage 8.0 (TID 158) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:46 INFO TaskSetManager: Finished task 146.0 in stage 8.0 (TID 154) in 1682 ms on 10.0.0.133 (executor driver) (147/208) +26/04/01 05:55:46 INFO Executor: Running task 150.0 in stage 8.0 (TID 158) +26/04/01 05:55:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:46 INFO Executor: Finished task 147.0 in stage 8.0 (TID 155). 2984 bytes result sent to driver +26/04/01 05:55:46 INFO TaskSetManager: Starting task 151.0 in stage 8.0 (TID 159) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:46 INFO TaskSetManager: Finished task 147.0 in stage 8.0 (TID 155) in 1679 ms on 10.0.0.133 (executor driver) (148/208) +26/04/01 05:55:46 INFO Executor: Running task 151.0 in stage 8.0 (TID 159) +26/04/01 05:55:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:48 INFO Executor: Finished task 148.0 in stage 8.0 (TID 156). 3027 bytes result sent to driver +26/04/01 05:55:48 INFO TaskSetManager: Starting task 152.0 in stage 8.0 (TID 160) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:48 INFO TaskSetManager: Finished task 148.0 in stage 8.0 (TID 156) in 1675 ms on 10.0.0.133 (executor driver) (149/208) +26/04/01 05:55:48 INFO Executor: Running task 152.0 in stage 8.0 (TID 160) +26/04/01 05:55:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:48 INFO Executor: Finished task 149.0 in stage 8.0 (TID 157). 2984 bytes result sent to driver +26/04/01 05:55:48 INFO TaskSetManager: Starting task 153.0 in stage 8.0 (TID 161) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:48 INFO TaskSetManager: Finished task 149.0 in stage 8.0 (TID 157) in 1679 ms on 10.0.0.133 (executor driver) (150/208) +26/04/01 05:55:48 INFO Executor: Running task 153.0 in stage 8.0 (TID 161) +26/04/01 05:55:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:48 INFO Executor: Finished task 150.0 in stage 8.0 (TID 158). 2984 bytes result sent to driver +26/04/01 05:55:48 INFO TaskSetManager: Starting task 154.0 in stage 8.0 (TID 162) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:48 INFO TaskSetManager: Finished task 150.0 in stage 8.0 (TID 158) in 1673 ms on 10.0.0.133 (executor driver) (151/208) +26/04/01 05:55:48 INFO Executor: Running task 154.0 in stage 8.0 (TID 162) +26/04/01 05:55:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:48 INFO Executor: Finished task 151.0 in stage 8.0 (TID 159). 2984 bytes result sent to driver +26/04/01 05:55:48 INFO TaskSetManager: Starting task 155.0 in stage 8.0 (TID 163) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:48 INFO TaskSetManager: Finished task 151.0 in stage 8.0 (TID 159) in 1675 ms on 10.0.0.133 (executor driver) (152/208) +26/04/01 05:55:48 INFO Executor: Running task 155.0 in stage 8.0 (TID 163) +26/04/01 05:55:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:49 INFO Executor: Finished task 154.0 in stage 8.0 (TID 162). 2984 bytes result sent to driver +26/04/01 05:55:49 INFO TaskSetManager: Starting task 156.0 in stage 8.0 (TID 164) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:49 INFO TaskSetManager: Finished task 154.0 in stage 8.0 (TID 162) in 1667 ms on 10.0.0.133 (executor driver) (153/208) +26/04/01 05:55:49 INFO Executor: Running task 156.0 in stage 8.0 (TID 164) +26/04/01 05:55:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:49 INFO Executor: Finished task 152.0 in stage 8.0 (TID 160). 3027 bytes result sent to driver +26/04/01 05:55:49 INFO TaskSetManager: Starting task 157.0 in stage 8.0 (TID 165) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:49 INFO TaskSetManager: Finished task 152.0 in stage 8.0 (TID 160) in 1683 ms on 10.0.0.133 (executor driver) (154/208) +26/04/01 05:55:49 INFO Executor: Running task 157.0 in stage 8.0 (TID 165) +26/04/01 05:55:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:49 INFO Executor: Finished task 153.0 in stage 8.0 (TID 161). 2984 bytes result sent to driver +26/04/01 05:55:49 INFO TaskSetManager: Starting task 158.0 in stage 8.0 (TID 166) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:49 INFO TaskSetManager: Finished task 153.0 in stage 8.0 (TID 161) in 1678 ms on 10.0.0.133 (executor driver) (155/208) +26/04/01 05:55:49 INFO Executor: Running task 158.0 in stage 8.0 (TID 166) +26/04/01 05:55:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:49 INFO Executor: Finished task 155.0 in stage 8.0 (TID 163). 2984 bytes result sent to driver +26/04/01 05:55:49 INFO TaskSetManager: Starting task 159.0 in stage 8.0 (TID 167) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:49 INFO Executor: Running task 159.0 in stage 8.0 (TID 167) +26/04/01 05:55:49 INFO TaskSetManager: Finished task 155.0 in stage 8.0 (TID 163) in 1684 ms on 10.0.0.133 (executor driver) (156/208) +26/04/01 05:55:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:51 INFO Executor: Finished task 157.0 in stage 8.0 (TID 165). 2984 bytes result sent to driver +26/04/01 05:55:51 INFO TaskSetManager: Starting task 160.0 in stage 8.0 (TID 168) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:51 INFO TaskSetManager: Finished task 157.0 in stage 8.0 (TID 165) in 1672 ms on 10.0.0.133 (executor driver) (157/208) +26/04/01 05:55:51 INFO Executor: Running task 160.0 in stage 8.0 (TID 168) +26/04/01 05:55:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:51 INFO Executor: Finished task 158.0 in stage 8.0 (TID 166). 2984 bytes result sent to driver +26/04/01 05:55:51 INFO TaskSetManager: Starting task 161.0 in stage 8.0 (TID 169) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:51 INFO TaskSetManager: Finished task 158.0 in stage 8.0 (TID 166) in 1675 ms on 10.0.0.133 (executor driver) (158/208) +26/04/01 05:55:51 INFO Executor: Running task 161.0 in stage 8.0 (TID 169) +26/04/01 05:55:51 INFO Executor: Finished task 156.0 in stage 8.0 (TID 164). 3027 bytes result sent to driver +26/04/01 05:55:51 INFO TaskSetManager: Starting task 162.0 in stage 8.0 (TID 170) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:51 INFO TaskSetManager: Finished task 156.0 in stage 8.0 (TID 164) in 1683 ms on 10.0.0.133 (executor driver) (159/208) +26/04/01 05:55:51 INFO Executor: Running task 162.0 in stage 8.0 (TID 170) +26/04/01 05:55:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:51 INFO Executor: Finished task 159.0 in stage 8.0 (TID 167). 3027 bytes result sent to driver +26/04/01 05:55:51 INFO TaskSetManager: Starting task 163.0 in stage 8.0 (TID 171) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:51 INFO TaskSetManager: Finished task 159.0 in stage 8.0 (TID 167) in 1673 ms on 10.0.0.133 (executor driver) (160/208) +26/04/01 05:55:51 INFO Executor: Running task 163.0 in stage 8.0 (TID 171) +26/04/01 05:55:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:53 INFO Executor: Finished task 160.0 in stage 8.0 (TID 168). 3027 bytes result sent to driver +26/04/01 05:55:53 INFO Executor: Finished task 162.0 in stage 8.0 (TID 170). 2984 bytes result sent to driver +26/04/01 05:55:53 INFO TaskSetManager: Starting task 164.0 in stage 8.0 (TID 172) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:53 INFO Executor: Running task 164.0 in stage 8.0 (TID 172) +26/04/01 05:55:53 INFO TaskSetManager: Starting task 165.0 in stage 8.0 (TID 173) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:53 INFO Executor: Finished task 161.0 in stage 8.0 (TID 169). 2984 bytes result sent to driver +26/04/01 05:55:53 INFO Executor: Running task 165.0 in stage 8.0 (TID 173) +26/04/01 05:55:53 INFO TaskSetManager: Starting task 166.0 in stage 8.0 (TID 174) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:53 INFO TaskSetManager: Finished task 160.0 in stage 8.0 (TID 168) in 1676 ms on 10.0.0.133 (executor driver) (161/208) +26/04/01 05:55:53 INFO Executor: Running task 166.0 in stage 8.0 (TID 174) +26/04/01 05:55:53 INFO TaskSetManager: Finished task 162.0 in stage 8.0 (TID 170) in 1672 ms on 10.0.0.133 (executor driver) (162/208) +26/04/01 05:55:53 INFO TaskSetManager: Finished task 161.0 in stage 8.0 (TID 169) in 1673 ms on 10.0.0.133 (executor driver) (163/208) +26/04/01 05:55:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:53 INFO Executor: Finished task 163.0 in stage 8.0 (TID 171). 2984 bytes result sent to driver +26/04/01 05:55:53 INFO TaskSetManager: Starting task 167.0 in stage 8.0 (TID 175) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:53 INFO TaskSetManager: Finished task 163.0 in stage 8.0 (TID 171) in 1676 ms on 10.0.0.133 (executor driver) (164/208) +26/04/01 05:55:53 INFO Executor: Running task 167.0 in stage 8.0 (TID 175) +26/04/01 05:55:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:54 INFO Executor: Finished task 165.0 in stage 8.0 (TID 173). 3027 bytes result sent to driver +26/04/01 05:55:54 INFO TaskSetManager: Starting task 168.0 in stage 8.0 (TID 176) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:54 INFO Executor: Running task 168.0 in stage 8.0 (TID 176) +26/04/01 05:55:54 INFO TaskSetManager: Finished task 165.0 in stage 8.0 (TID 173) in 1673 ms on 10.0.0.133 (executor driver) (165/208) +26/04/01 05:55:54 INFO Executor: Finished task 164.0 in stage 8.0 (TID 172). 3027 bytes result sent to driver +26/04/01 05:55:54 INFO TaskSetManager: Starting task 169.0 in stage 8.0 (TID 177) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:54 INFO TaskSetManager: Finished task 164.0 in stage 8.0 (TID 172) in 1674 ms on 10.0.0.133 (executor driver) (166/208) +26/04/01 05:55:54 INFO Executor: Running task 169.0 in stage 8.0 (TID 177) +26/04/01 05:55:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:54 INFO Executor: Finished task 166.0 in stage 8.0 (TID 174). 2984 bytes result sent to driver +26/04/01 05:55:54 INFO TaskSetManager: Starting task 170.0 in stage 8.0 (TID 178) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:54 INFO TaskSetManager: Finished task 166.0 in stage 8.0 (TID 174) in 1679 ms on 10.0.0.133 (executor driver) (167/208) +26/04/01 05:55:54 INFO Executor: Running task 170.0 in stage 8.0 (TID 178) +26/04/01 05:55:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:54 INFO Executor: Finished task 167.0 in stage 8.0 (TID 175). 2984 bytes result sent to driver +26/04/01 05:55:54 INFO TaskSetManager: Starting task 171.0 in stage 8.0 (TID 179) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:54 INFO TaskSetManager: Finished task 167.0 in stage 8.0 (TID 175) in 1676 ms on 10.0.0.133 (executor driver) (168/208) +26/04/01 05:55:54 INFO Executor: Running task 171.0 in stage 8.0 (TID 179) +26/04/01 05:55:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:56 INFO Executor: Finished task 168.0 in stage 8.0 (TID 176). 3027 bytes result sent to driver +26/04/01 05:55:56 INFO TaskSetManager: Starting task 172.0 in stage 8.0 (TID 180) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:56 INFO Executor: Running task 172.0 in stage 8.0 (TID 180) +26/04/01 05:55:56 INFO TaskSetManager: Finished task 168.0 in stage 8.0 (TID 176) in 1678 ms on 10.0.0.133 (executor driver) (169/208) +26/04/01 05:55:56 INFO Executor: Finished task 169.0 in stage 8.0 (TID 177). 2984 bytes result sent to driver +26/04/01 05:55:56 INFO TaskSetManager: Starting task 173.0 in stage 8.0 (TID 181) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:56 INFO TaskSetManager: Finished task 169.0 in stage 8.0 (TID 177) in 1679 ms on 10.0.0.133 (executor driver) (170/208) +26/04/01 05:55:56 INFO Executor: Running task 173.0 in stage 8.0 (TID 181) +26/04/01 05:55:56 INFO Executor: Finished task 170.0 in stage 8.0 (TID 178). 2984 bytes result sent to driver +26/04/01 05:55:56 INFO TaskSetManager: Starting task 174.0 in stage 8.0 (TID 182) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:56 INFO TaskSetManager: Finished task 170.0 in stage 8.0 (TID 178) in 1675 ms on 10.0.0.133 (executor driver) (171/208) +26/04/01 05:55:56 INFO Executor: Running task 174.0 in stage 8.0 (TID 182) +26/04/01 05:55:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:56 INFO Executor: Finished task 171.0 in stage 8.0 (TID 179). 2984 bytes result sent to driver +26/04/01 05:55:56 INFO TaskSetManager: Starting task 175.0 in stage 8.0 (TID 183) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:56 INFO TaskSetManager: Finished task 171.0 in stage 8.0 (TID 179) in 1685 ms on 10.0.0.133 (executor driver) (172/208) +26/04/01 05:55:56 INFO Executor: Running task 175.0 in stage 8.0 (TID 183) +26/04/01 05:55:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:58 INFO Executor: Finished task 174.0 in stage 8.0 (TID 182). 2984 bytes result sent to driver +26/04/01 05:55:58 INFO Executor: Finished task 173.0 in stage 8.0 (TID 181). 2984 bytes result sent to driver +26/04/01 05:55:58 INFO TaskSetManager: Starting task 176.0 in stage 8.0 (TID 184) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:58 INFO TaskSetManager: Starting task 177.0 in stage 8.0 (TID 185) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:58 INFO Executor: Running task 176.0 in stage 8.0 (TID 184) +26/04/01 05:55:58 INFO TaskSetManager: Finished task 174.0 in stage 8.0 (TID 182) in 1675 ms on 10.0.0.133 (executor driver) (173/208) +26/04/01 05:55:58 INFO Executor: Running task 177.0 in stage 8.0 (TID 185) +26/04/01 05:55:58 INFO TaskSetManager: Finished task 173.0 in stage 8.0 (TID 181) in 1676 ms on 10.0.0.133 (executor driver) (174/208) +26/04/01 05:55:58 INFO Executor: Finished task 172.0 in stage 8.0 (TID 180). 3027 bytes result sent to driver +26/04/01 05:55:58 INFO TaskSetManager: Starting task 178.0 in stage 8.0 (TID 186) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:58 INFO TaskSetManager: Finished task 172.0 in stage 8.0 (TID 180) in 1681 ms on 10.0.0.133 (executor driver) (175/208) +26/04/01 05:55:58 INFO Executor: Running task 178.0 in stage 8.0 (TID 186) +26/04/01 05:55:58 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:58 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:58 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:55:58 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:58 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:58 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:58 INFO Executor: Finished task 175.0 in stage 8.0 (TID 183). 3027 bytes result sent to driver +26/04/01 05:55:58 INFO TaskSetManager: Starting task 179.0 in stage 8.0 (TID 187) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:55:58 INFO TaskSetManager: Finished task 175.0 in stage 8.0 (TID 183) in 1677 ms on 10.0.0.133 (executor driver) (176/208) +26/04/01 05:55:58 INFO Executor: Running task 179.0 in stage 8.0 (TID 187) +26/04/01 05:55:58 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:55:58 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:59 INFO Executor: Finished task 178.0 in stage 8.0 (TID 186). 2984 bytes result sent to driver +26/04/01 05:55:59 INFO Executor: Finished task 177.0 in stage 8.0 (TID 185). 3027 bytes result sent to driver +26/04/01 05:55:59 INFO TaskSetManager: Starting task 180.0 in stage 8.0 (TID 188) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:59 INFO Executor: Finished task 176.0 in stage 8.0 (TID 184). 3027 bytes result sent to driver +26/04/01 05:55:59 INFO Executor: Running task 180.0 in stage 8.0 (TID 188) +26/04/01 05:55:59 INFO TaskSetManager: Starting task 181.0 in stage 8.0 (TID 189) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:59 INFO Executor: Running task 181.0 in stage 8.0 (TID 189) +26/04/01 05:55:59 INFO TaskSetManager: Starting task 182.0 in stage 8.0 (TID 190) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:59 INFO TaskSetManager: Finished task 178.0 in stage 8.0 (TID 186) in 1675 ms on 10.0.0.133 (executor driver) (177/208) +26/04/01 05:55:59 INFO Executor: Running task 182.0 in stage 8.0 (TID 190) +26/04/01 05:55:59 INFO TaskSetManager: Finished task 177.0 in stage 8.0 (TID 185) in 1678 ms on 10.0.0.133 (executor driver) (178/208) +26/04/01 05:55:59 INFO TaskSetManager: Finished task 176.0 in stage 8.0 (TID 184) in 1678 ms on 10.0.0.133 (executor driver) (179/208) +26/04/01 05:55:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:55:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:55:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:55:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:55:59 INFO Executor: Finished task 179.0 in stage 8.0 (TID 187). 2984 bytes result sent to driver +26/04/01 05:55:59 INFO TaskSetManager: Starting task 183.0 in stage 8.0 (TID 191) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:55:59 INFO TaskSetManager: Finished task 179.0 in stage 8.0 (TID 187) in 1675 ms on 10.0.0.133 (executor driver) (180/208) +26/04/01 05:55:59 INFO Executor: Running task 183.0 in stage 8.0 (TID 191) +26/04/01 05:55:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:55:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:01 INFO Executor: Finished task 182.0 in stage 8.0 (TID 190). 2984 bytes result sent to driver +26/04/01 05:56:01 INFO TaskSetManager: Starting task 184.0 in stage 8.0 (TID 192) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:01 INFO TaskSetManager: Finished task 182.0 in stage 8.0 (TID 190) in 1666 ms on 10.0.0.133 (executor driver) (181/208) +26/04/01 05:56:01 INFO Executor: Running task 184.0 in stage 8.0 (TID 192) +26/04/01 05:56:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:01 INFO Executor: Finished task 180.0 in stage 8.0 (TID 188). 3027 bytes result sent to driver +26/04/01 05:56:01 INFO TaskSetManager: Starting task 185.0 in stage 8.0 (TID 193) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:01 INFO Executor: Finished task 181.0 in stage 8.0 (TID 189). 3027 bytes result sent to driver +26/04/01 05:56:01 INFO Executor: Running task 185.0 in stage 8.0 (TID 193) +26/04/01 05:56:01 INFO TaskSetManager: Starting task 186.0 in stage 8.0 (TID 194) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:01 INFO TaskSetManager: Finished task 180.0 in stage 8.0 (TID 188) in 1676 ms on 10.0.0.133 (executor driver) (182/208) +26/04/01 05:56:01 INFO Executor: Running task 186.0 in stage 8.0 (TID 194) +26/04/01 05:56:01 INFO TaskSetManager: Finished task 181.0 in stage 8.0 (TID 189) in 1676 ms on 10.0.0.133 (executor driver) (183/208) +26/04/01 05:56:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:01 INFO Executor: Finished task 183.0 in stage 8.0 (TID 191). 3027 bytes result sent to driver +26/04/01 05:56:01 INFO TaskSetManager: Starting task 187.0 in stage 8.0 (TID 195) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:01 INFO Executor: Running task 187.0 in stage 8.0 (TID 195) +26/04/01 05:56:01 INFO TaskSetManager: Finished task 183.0 in stage 8.0 (TID 191) in 1668 ms on 10.0.0.133 (executor driver) (184/208) +26/04/01 05:56:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:03 INFO Executor: Finished task 185.0 in stage 8.0 (TID 193). 2984 bytes result sent to driver +26/04/01 05:56:03 INFO TaskSetManager: Starting task 188.0 in stage 8.0 (TID 196) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:03 INFO TaskSetManager: Finished task 185.0 in stage 8.0 (TID 193) in 1673 ms on 10.0.0.133 (executor driver) (185/208) +26/04/01 05:56:03 INFO Executor: Running task 188.0 in stage 8.0 (TID 196) +26/04/01 05:56:03 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:03 INFO Executor: Finished task 186.0 in stage 8.0 (TID 194). 2984 bytes result sent to driver +26/04/01 05:56:03 INFO TaskSetManager: Starting task 189.0 in stage 8.0 (TID 197) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:03 INFO TaskSetManager: Finished task 186.0 in stage 8.0 (TID 194) in 1678 ms on 10.0.0.133 (executor driver) (186/208) +26/04/01 05:56:03 INFO Executor: Running task 189.0 in stage 8.0 (TID 197) +26/04/01 05:56:03 INFO Executor: Finished task 184.0 in stage 8.0 (TID 192). 3027 bytes result sent to driver +26/04/01 05:56:03 INFO TaskSetManager: Starting task 190.0 in stage 8.0 (TID 198) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:03 INFO TaskSetManager: Finished task 184.0 in stage 8.0 (TID 192) in 1689 ms on 10.0.0.133 (executor driver) (187/208) +26/04/01 05:56:03 INFO Executor: Running task 190.0 in stage 8.0 (TID 198) +26/04/01 05:56:03 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:03 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:03 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:03 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:03 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:03 INFO Executor: Finished task 187.0 in stage 8.0 (TID 195). 3027 bytes result sent to driver +26/04/01 05:56:03 INFO TaskSetManager: Starting task 191.0 in stage 8.0 (TID 199) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:03 INFO Executor: Running task 191.0 in stage 8.0 (TID 199) +26/04/01 05:56:03 INFO TaskSetManager: Finished task 187.0 in stage 8.0 (TID 195) in 1680 ms on 10.0.0.133 (executor driver) (188/208) +26/04/01 05:56:03 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:03 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:04 INFO Executor: Finished task 189.0 in stage 8.0 (TID 197). 2984 bytes result sent to driver +26/04/01 05:56:04 INFO Executor: Finished task 188.0 in stage 8.0 (TID 196). 3027 bytes result sent to driver +26/04/01 05:56:04 INFO TaskSetManager: Starting task 192.0 in stage 8.0 (TID 200) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:04 INFO Executor: Running task 192.0 in stage 8.0 (TID 200) +26/04/01 05:56:04 INFO TaskSetManager: Starting task 193.0 in stage 8.0 (TID 201) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:04 INFO Executor: Running task 193.0 in stage 8.0 (TID 201) +26/04/01 05:56:04 INFO TaskSetManager: Finished task 189.0 in stage 8.0 (TID 197) in 1674 ms on 10.0.0.133 (executor driver) (189/208) +26/04/01 05:56:04 INFO TaskSetManager: Finished task 188.0 in stage 8.0 (TID 196) in 1680 ms on 10.0.0.133 (executor driver) (190/208) +26/04/01 05:56:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 805306368-848563442, partition values: [empty row] +26/04/01 05:56:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 805306368-848649288, partition values: [empty row] +26/04/01 05:56:04 INFO Executor: Finished task 190.0 in stage 8.0 (TID 198). 2984 bytes result sent to driver +26/04/01 05:56:04 INFO TaskSetManager: Starting task 194.0 in stage 8.0 (TID 202) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:04 INFO Executor: Running task 194.0 in stage 8.0 (TID 202) +26/04/01 05:56:04 INFO TaskSetManager: Finished task 190.0 in stage 8.0 (TID 198) in 1677 ms on 10.0.0.133 (executor driver) (191/208) +26/04/01 05:56:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 805306368-848496433, partition values: [empty row] +26/04/01 05:56:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:04 INFO Executor: Finished task 191.0 in stage 8.0 (TID 199). 3027 bytes result sent to driver +26/04/01 05:56:04 INFO TaskSetManager: Starting task 195.0 in stage 8.0 (TID 203) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:04 INFO TaskSetManager: Finished task 191.0 in stage 8.0 (TID 199) in 1674 ms on 10.0.0.133 (executor driver) (192/208) +26/04/01 05:56:04 INFO Executor: Running task 195.0 in stage 8.0 (TID 203) +26/04/01 05:56:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 805306368-848463796, partition values: [empty row] +26/04/01 05:56:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 805306368-848521656, partition values: [empty row] +26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 805306368-848489219, partition values: [empty row] +26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 805306368-848594494, partition values: [empty row] +26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 805306368-848442878, partition values: [empty row] +26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:05 INFO Executor: Finished task 193.0 in stage 8.0 (TID 201). 2984 bytes result sent to driver +26/04/01 05:56:05 INFO TaskSetManager: Starting task 196.0 in stage 8.0 (TID 204) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:05 INFO TaskSetManager: Finished task 193.0 in stage 8.0 (TID 201) in 1045 ms on 10.0.0.133 (executor driver) (193/208) +26/04/01 05:56:05 INFO Executor: Running task 196.0 in stage 8.0 (TID 204) +26/04/01 05:56:05 INFO Executor: Finished task 194.0 in stage 8.0 (TID 202). 2984 bytes result sent to driver +26/04/01 05:56:05 INFO TaskSetManager: Starting task 197.0 in stage 8.0 (TID 205) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:05 INFO TaskSetManager: Finished task 194.0 in stage 8.0 (TID 202) in 1042 ms on 10.0.0.133 (executor driver) (194/208) +26/04/01 05:56:05 INFO Executor: Running task 197.0 in stage 8.0 (TID 205) +26/04/01 05:56:05 INFO Executor: Finished task 192.0 in stage 8.0 (TID 200). 3027 bytes result sent to driver +26/04/01 05:56:05 INFO TaskSetManager: Starting task 198.0 in stage 8.0 (TID 206) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:05 INFO TaskSetManager: Finished task 192.0 in stage 8.0 (TID 200) in 1047 ms on 10.0.0.133 (executor driver) (195/208) +26/04/01 05:56:05 INFO Executor: Running task 198.0 in stage 8.0 (TID 206) +26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 805306368-848432523, partition values: [empty row] +26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 805306368-848422348, partition values: [empty row] +26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 805306368-848399844, partition values: [empty row] +26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:05 INFO Executor: Finished task 195.0 in stage 8.0 (TID 203). 2984 bytes result sent to driver +26/04/01 05:56:05 INFO TaskSetManager: Starting task 199.0 in stage 8.0 (TID 207) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:05 INFO TaskSetManager: Finished task 195.0 in stage 8.0 (TID 203) in 1044 ms on 10.0.0.133 (executor driver) (196/208) +26/04/01 05:56:05 INFO Executor: Running task 199.0 in stage 8.0 (TID 207) +26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 805306368-848378208, partition values: [empty row] +26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 805306368-848411809, partition values: [empty row] +26/04/01 05:56:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 805306368-848379799, partition values: [empty row] +26/04/01 05:56:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 805306368-848423659, partition values: [empty row] +26/04/01 05:56:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 805306368-848348606, partition values: [empty row] +26/04/01 05:56:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:06 INFO Executor: Finished task 198.0 in stage 8.0 (TID 206). 2984 bytes result sent to driver +26/04/01 05:56:06 INFO TaskSetManager: Starting task 200.0 in stage 8.0 (TID 208) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9708 bytes) +26/04/01 05:56:06 INFO Executor: Running task 200.0 in stage 8.0 (TID 208) +26/04/01 05:56:06 INFO TaskSetManager: Finished task 198.0 in stage 8.0 (TID 206) in 1038 ms on 10.0.0.133 (executor driver) (197/208) +26/04/01 05:56:06 INFO Executor: Finished task 196.0 in stage 8.0 (TID 204). 3027 bytes result sent to driver +26/04/01 05:56:06 INFO TaskSetManager: Starting task 201.0 in stage 8.0 (TID 209) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:06 INFO TaskSetManager: Finished task 196.0 in stage 8.0 (TID 204) in 1042 ms on 10.0.0.133 (executor driver) (198/208) +26/04/01 05:56:06 INFO Executor: Running task 201.0 in stage 8.0 (TID 209) +26/04/01 05:56:06 INFO Executor: Finished task 197.0 in stage 8.0 (TID 205). 2984 bytes result sent to driver +26/04/01 05:56:06 INFO TaskSetManager: Starting task 202.0 in stage 8.0 (TID 210) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:06 INFO TaskSetManager: Finished task 197.0 in stage 8.0 (TID 205) in 1041 ms on 10.0.0.133 (executor driver) (199/208) +26/04/01 05:56:06 INFO Executor: Running task 202.0 in stage 8.0 (TID 210) +26/04/01 05:56:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 805306368-848344114, partition values: [empty row] +26/04/01 05:56:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 805306368-848329366, partition values: [empty row] +26/04/01 05:56:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 805306368-848328570, partition values: [empty row] +26/04/01 05:56:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:07 INFO Executor: Finished task 199.0 in stage 8.0 (TID 207). 2984 bytes result sent to driver +26/04/01 05:56:07 INFO TaskSetManager: Starting task 203.0 in stage 8.0 (TID 211) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:07 INFO Executor: Running task 203.0 in stage 8.0 (TID 211) +26/04/01 05:56:07 INFO TaskSetManager: Finished task 199.0 in stage 8.0 (TID 207) in 1043 ms on 10.0.0.133 (executor driver) (200/208) +26/04/01 05:56:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 805306368-848301737, partition values: [empty row] +26/04/01 05:56:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 805306368-848338051, partition values: [empty row] +26/04/01 05:56:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 805306368-848329327, partition values: [empty row] +26/04/01 05:56:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 805306368-848306153, partition values: [empty row] +26/04/01 05:56:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 805306368-848278425, partition values: [empty row] +26/04/01 05:56:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:08 INFO Executor: Finished task 200.0 in stage 8.0 (TID 208). 3027 bytes result sent to driver +26/04/01 05:56:08 INFO TaskSetManager: Starting task 204.0 in stage 8.0 (TID 212) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:08 INFO Executor: Running task 204.0 in stage 8.0 (TID 212) +26/04/01 05:56:08 INFO TaskSetManager: Finished task 200.0 in stage 8.0 (TID 208) in 1042 ms on 10.0.0.133 (executor driver) (201/208) +26/04/01 05:56:08 INFO Executor: Finished task 201.0 in stage 8.0 (TID 209). 2984 bytes result sent to driver +26/04/01 05:56:08 INFO TaskSetManager: Starting task 205.0 in stage 8.0 (TID 213) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:08 INFO Executor: Finished task 202.0 in stage 8.0 (TID 210). 2984 bytes result sent to driver +26/04/01 05:56:08 INFO TaskSetManager: Finished task 201.0 in stage 8.0 (TID 209) in 1043 ms on 10.0.0.133 (executor driver) (202/208) +26/04/01 05:56:08 INFO Executor: Running task 205.0 in stage 8.0 (TID 213) +26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 805306368-848263904, partition values: [empty row] +26/04/01 05:56:08 INFO TaskSetManager: Starting task 206.0 in stage 8.0 (TID 214) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:08 INFO TaskSetManager: Finished task 202.0 in stage 8.0 (TID 210) in 1044 ms on 10.0.0.133 (executor driver) (203/208) +26/04/01 05:56:08 INFO Executor: Running task 206.0 in stage 8.0 (TID 214) +26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 805306368-848235021, partition values: [empty row] +26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 805306368-848107384, partition values: [empty row] +26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:08 INFO Executor: Finished task 203.0 in stage 8.0 (TID 211). 2984 bytes result sent to driver +26/04/01 05:56:08 INFO TaskSetManager: Starting task 207.0 in stage 8.0 (TID 215) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:08 INFO TaskSetManager: Finished task 203.0 in stage 8.0 (TID 211) in 1041 ms on 10.0.0.133 (executor driver) (204/208) +26/04/01 05:56:08 INFO Executor: Running task 207.0 in stage 8.0 (TID 215) +26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 805306368-848005421, partition values: [empty row] +26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 805306368-848247851, partition values: [empty row] +26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 805306368-848050794, partition values: [empty row] +26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 805306368-848234776, partition values: [empty row] +26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 805306368-847699715, partition values: [empty row] +26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) +26/04/01 05:56:09 INFO Executor: Finished task 204.0 in stage 8.0 (TID 212). 2984 bytes result sent to driver +26/04/01 05:56:09 INFO TaskSetManager: Finished task 204.0 in stage 8.0 (TID 212) in 1042 ms on 10.0.0.133 (executor driver) (205/208) +26/04/01 05:56:09 INFO Executor: Finished task 205.0 in stage 8.0 (TID 213). 2984 bytes result sent to driver +26/04/01 05:56:09 INFO TaskSetManager: Finished task 205.0 in stage 8.0 (TID 213) in 1042 ms on 10.0.0.133 (executor driver) (206/208) +26/04/01 05:56:09 INFO Executor: Finished task 206.0 in stage 8.0 (TID 214). 2984 bytes result sent to driver +26/04/01 05:56:09 INFO TaskSetManager: Finished task 206.0 in stage 8.0 (TID 214) in 1044 ms on 10.0.0.133 (executor driver) (207/208) +26/04/01 05:56:09 INFO Executor: Finished task 207.0 in stage 8.0 (TID 215). 2984 bytes result sent to driver +26/04/01 05:56:09 INFO TaskSetManager: Finished task 207.0 in stage 8.0 (TID 215) in 1035 ms on 10.0.0.133 (executor driver) (208/208) +26/04/01 05:56:09 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool +26/04/01 05:56:09 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 85.502 s +26/04/01 05:56:09 INFO DAGScheduler: looking for newly runnable stages +26/04/01 05:56:09 INFO DAGScheduler: running: Set() +26/04/01 05:56:09 INFO DAGScheduler: waiting: Set() +26/04/01 05:56:09 INFO DAGScheduler: failed: Set() +26/04/01 05:56:09 INFO ShufflePartitionsUtil: For shuffle(0), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 05:56:09 INFO HashAggregateExec: spark.sql.codegen.aggregate.map.twolevel.enabled is set to true, but current version of codegened fast hashmap does not support this aggregate. +26/04/01 05:56:09 INFO CodeGenerator: Code generated in 13.867709 ms +26/04/01 05:56:09 INFO CodeGenerator: Code generated in 3.550125 ms +26/04/01 05:56:09 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:56:09 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 05:56:09 INFO DAGScheduler: Final stage: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:56:09 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 9) +26/04/01 05:56:09 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:09 INFO DAGScheduler: Submitting ResultStage 10 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:56:09 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 91.1 KiB, free 8.6 GiB) +26/04/01 05:56:09 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) +26/04/01 05:56:09 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:58187 (size: 35.0 KiB, free: 8.6 GiB) +26/04/01 05:56:09 INFO SparkContext: Created broadcast 10 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:09 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 10 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:56:09 INFO TaskSchedulerImpl: Adding task set 10.0 with 1 tasks resource profile 0 +26/04/01 05:56:09 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 216) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8999 bytes) +26/04/01 05:56:09 INFO Executor: Running task 0.0 in stage 10.0 (TID 216) +26/04/01 05:56:09 INFO ShuffleBlockFetcherIterator: Getting 208 (184.6 KiB) non-empty blocks including 208 (184.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms +26/04/01 05:56:09 INFO CodeGenerator: Code generated in 9.284042 ms +26/04/01 05:56:09 INFO Executor: Finished task 0.0 in stage 10.0 (TID 216). 5717 bytes result sent to driver +26/04/01 05:56:09 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 216) in 60 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:56:09 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool +26/04/01 05:56:09 INFO DAGScheduler: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.063 s +26/04/01 05:56:09 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:56:09 INFO TaskSchedulerImpl: Killing all running tasks in stage 10: Stage finished +26/04/01 05:56:09 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.067610 s +26/04/01 05:56:09 INFO DAGScheduler: Registering RDD 25 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 +26/04/01 05:56:09 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 05:56:09 INFO DAGScheduler: Final stage: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:56:09 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 11) +26/04/01 05:56:09 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:09 INFO DAGScheduler: Submitting ShuffleMapStage 12 (MapPartitionsRDD[25] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:56:09 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 91.8 KiB, free 8.6 GiB) +26/04/01 05:56:09 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 35.3 KiB, free 8.6 GiB) +26/04/01 05:56:09 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:58187 (size: 35.3 KiB, free: 8.6 GiB) +26/04/01 05:56:09 INFO SparkContext: Created broadcast 11 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:09 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 12 (MapPartitionsRDD[25] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:56:09 INFO TaskSchedulerImpl: Adding task set 12.0 with 1 tasks resource profile 0 +26/04/01 05:56:09 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 217) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8988 bytes) +26/04/01 05:56:09 INFO Executor: Running task 0.0 in stage 12.0 (TID 217) +26/04/01 05:56:09 INFO CodeGenerator: Code generated in 2.071125 ms +26/04/01 05:56:09 INFO ShuffleBlockFetcherIterator: Getting 208 (184.6 KiB) non-empty blocks including 208 (184.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:09 INFO Executor: Finished task 0.0 in stage 12.0 (TID 217). 5524 bytes result sent to driver +26/04/01 05:56:09 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 217) in 26 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:56:09 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool +26/04/01 05:56:09 INFO DAGScheduler: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.030 s +26/04/01 05:56:09 INFO DAGScheduler: looking for newly runnable stages +26/04/01 05:56:09 INFO DAGScheduler: running: Set() +26/04/01 05:56:09 INFO DAGScheduler: waiting: Set() +26/04/01 05:56:09 INFO DAGScheduler: failed: Set() +26/04/01 05:56:09 INFO ShufflePartitionsUtil: For shuffle(1), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 05:56:09 INFO CodeGenerator: Code generated in 3.744459 ms +26/04/01 05:56:09 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:56:09 INFO DAGScheduler: Got job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 05:56:09 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:56:09 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 14) +26/04/01 05:56:09 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:09 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:56:09 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 66.7 KiB, free 8.6 GiB) +26/04/01 05:56:09 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 27.3 KiB, free 8.6 GiB) +26/04/01 05:56:09 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:58187 (size: 27.3 KiB, free: 8.6 GiB) +26/04/01 05:56:09 INFO SparkContext: Created broadcast 12 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:09 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:56:09 INFO TaskSchedulerImpl: Adding task set 15.0 with 1 tasks resource profile 0 +26/04/01 05:56:09 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 218) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8999 bytes) +26/04/01 05:56:09 INFO Executor: Running task 0.0 in stage 15.0 (TID 218) +26/04/01 05:56:09 INFO ShuffleBlockFetcherIterator: Getting 1 (912.0 B) non-empty blocks including 1 (912.0 B) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:09 INFO CodeGenerator: Code generated in 2.812042 ms +26/04/01 05:56:09 INFO CodeGenerator: Code generated in 2.661958 ms +26/04/01 05:56:09 INFO Executor: Finished task 0.0 in stage 15.0 (TID 218). 7302 bytes result sent to driver +26/04/01 05:56:09 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 218) in 21 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:56:09 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool +26/04/01 05:56:09 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.024 s +26/04/01 05:56:09 INFO DAGScheduler: Job 11 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:56:09 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished +26/04/01 05:56:09 INFO DAGScheduler: Job 11 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.024247 s +26/04/01 05:56:09 INFO SparkContext: SparkContext is stopping with exitCode 0. +26/04/01 05:56:09 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! +26/04/01 05:56:09 INFO MemoryStore: MemoryStore cleared +26/04/01 05:56:09 INFO BlockManager: BlockManager stopped +26/04/01 05:56:09 INFO BlockManagerMaster: BlockManagerMaster stopped +26/04/01 05:56:09 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! +26/04/01 05:56:09 INFO SparkContext: Successfully stopped SparkContext +26/04/01 05:56:09 INFO ShutdownHookManager: Shutdown hook called +26/04/01 05:56:09 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-61f0e515-7a17-4e12-9e70-40ca151350fa +26/04/01 05:56:09 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-8e526fcf-8de7-4d28-9732-922dc858ca07/pyspark-260dfcd9-b64e-4cbd-bc76-c977a7a8a155 +26/04/01 05:56:09 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-8e526fcf-8de7-4d28-9732-922dc858ca07 + 89.94 real 363.21 user 10.80 sys + 2831646720 maximum resident set size + 0 average shared memory size + 0 average unshared data size + 0 average unshared stack size + 203207 page reclaims + 182 page faults + 0 swaps + 0 block input operations + 0 block output operations + 1015 messages sent + 1033 messages received + 1758 signals received + 16605 voluntary context switches + 539811 involuntary context switches + 7848607567173 instructions retired + 1350297036464 cycles elapsed + 2810317944 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.log b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.log new file mode 100644 index 0000000000..90aba2134d --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.log @@ -0,0 +1,330 @@ +Registering table customer from /opt/tpch/sf100/customer +Registering table lineitem from /opt/tpch/sf100/lineitem +Registering table nation from /opt/tpch/sf100/nation +Registering table orders from /opt/tpch/sf100/orders +Registering table part from /opt/tpch/sf100/part +Registering table partsupp from /opt/tpch/sf100/partsupp +Registering table region from /opt/tpch/sf100/region +Registering table supplier from /opt/tpch/sf100/supplier + +============================================================ +Starting iteration 1 of 1 +============================================================ + +Running query 5 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q5.sql +Executing: -- CometBench-H query 5 derived from TPC-H query 5 under the terms of the TPC Fair Use Policy. +-- TP... +== Physical Plan == +AdaptiveSparkPlan (44) ++- Sort (43) + +- Exchange (42) + +- HashAggregate (41) + +- Exchange (40) + +- HashAggregate (39) + +- Project (38) + +- BroadcastHashJoin Inner BuildRight (37) + :- Project (32) + : +- BroadcastHashJoin Inner BuildRight (31) + : :- Project (27) + : : +- SortMergeJoin Inner (26) + : : :- Sort (21) + : : : +- Exchange (20) + : : : +- Project (19) + : : : +- SortMergeJoin Inner (18) + : : : :- Sort (13) + : : : : +- Exchange (12) + : : : : +- Project (11) + : : : : +- SortMergeJoin Inner (10) + : : : : :- Sort (4) + : : : : : +- Exchange (3) + : : : : : +- Filter (2) + : : : : : +- Scan parquet (1) + : : : : +- Sort (9) + : : : : +- Exchange (8) + : : : : +- Project (7) + : : : : +- Filter (6) + : : : : +- Scan parquet (5) + : : : +- Sort (17) + : : : +- Exchange (16) + : : : +- Filter (15) + : : : +- Scan parquet (14) + : : +- Sort (25) + : : +- Exchange (24) + : : +- Filter (23) + : : +- Scan parquet (22) + : +- BroadcastExchange (30) + : +- Filter (29) + : +- Scan parquet (28) + +- BroadcastExchange (36) + +- Project (35) + +- Filter (34) + +- Scan parquet (33) + + +(1) Scan parquet +Output [2]: [c_custkey#0L, c_nationkey#3L] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/customer] +PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] +ReadSchema: struct + +(2) Filter +Input [2]: [c_custkey#0L, c_nationkey#3L] +Condition : (isnotnull(c_custkey#0L) AND isnotnull(c_nationkey#3L)) + +(3) Exchange +Input [2]: [c_custkey#0L, c_nationkey#3L] +Arguments: hashpartitioning(c_custkey#0L, 200), ENSURE_REQUIREMENTS, [plan_id=159] + +(4) Sort +Input [2]: [c_custkey#0L, c_nationkey#3L] +Arguments: [c_custkey#0L ASC NULLS FIRST], false, 0 + +(5) Scan parquet +Output [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/orders] +PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1994-01-01), LessThan(o_orderdate,1995-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] +ReadSchema: struct + +(6) Filter +Input [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] +Condition : ((((isnotnull(o_orderdate#60) AND (o_orderdate#60 >= 1994-01-01)) AND (o_orderdate#60 < 1995-01-01)) AND isnotnull(o_custkey#57L)) AND isnotnull(o_orderkey#56L)) + +(7) Project +Output [2]: [o_orderkey#56L, o_custkey#57L] +Input [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] + +(8) Exchange +Input [2]: [o_orderkey#56L, o_custkey#57L] +Arguments: hashpartitioning(o_custkey#57L, 200), ENSURE_REQUIREMENTS, [plan_id=160] + +(9) Sort +Input [2]: [o_orderkey#56L, o_custkey#57L] +Arguments: [o_custkey#57L ASC NULLS FIRST], false, 0 + +(10) SortMergeJoin +Left keys [1]: [c_custkey#0L] +Right keys [1]: [o_custkey#57L] +Join type: Inner +Join condition: None + +(11) Project +Output [2]: [c_nationkey#3L, o_orderkey#56L] +Input [4]: [c_custkey#0L, c_nationkey#3L, o_orderkey#56L, o_custkey#57L] + +(12) Exchange +Input [2]: [c_nationkey#3L, o_orderkey#56L] +Arguments: hashpartitioning(o_orderkey#56L, 200), ENSURE_REQUIREMENTS, [plan_id=167] + +(13) Sort +Input [2]: [c_nationkey#3L, o_orderkey#56L] +Arguments: [o_orderkey#56L ASC NULLS FIRST], false, 0 + +(14) Scan parquet +Output [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] +PushedFilters: [IsNotNull(l_orderkey), IsNotNull(l_suppkey)] +ReadSchema: struct + +(15) Filter +Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Condition : (isnotnull(l_orderkey#16L) AND isnotnull(l_suppkey#18L)) + +(16) Exchange +Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: hashpartitioning(l_orderkey#16L, 200), ENSURE_REQUIREMENTS, [plan_id=168] + +(17) Sort +Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: [l_orderkey#16L ASC NULLS FIRST], false, 0 + +(18) SortMergeJoin +Left keys [1]: [o_orderkey#56L] +Right keys [1]: [l_orderkey#16L] +Join type: Inner +Join condition: None + +(19) Project +Output [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Input [6]: [c_nationkey#3L, o_orderkey#56L, l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] + +(20) Exchange +Input [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: hashpartitioning(l_suppkey#18L, c_nationkey#3L, 200), ENSURE_REQUIREMENTS, [plan_id=175] + +(21) Sort +Input [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] +Arguments: [l_suppkey#18L ASC NULLS FIRST, c_nationkey#3L ASC NULLS FIRST], false, 0 + +(22) Scan parquet +Output [2]: [s_suppkey#108L, s_nationkey#111L] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/supplier] +PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] +ReadSchema: struct + +(23) Filter +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Condition : (isnotnull(s_suppkey#108L) AND isnotnull(s_nationkey#111L)) + +(24) Exchange +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: hashpartitioning(s_suppkey#108L, s_nationkey#111L, 200), ENSURE_REQUIREMENTS, [plan_id=176] + +(25) Sort +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: [s_suppkey#108L ASC NULLS FIRST, s_nationkey#111L ASC NULLS FIRST], false, 0 + +(26) SortMergeJoin +Left keys [2]: [l_suppkey#18L, c_nationkey#3L] +Right keys [2]: [s_suppkey#108L, s_nationkey#111L] +Join type: Inner +Join condition: None + +(27) Project +Output [3]: [l_extendedprice#21, l_discount#22, s_nationkey#111L] +Input [6]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22, s_suppkey#108L, s_nationkey#111L] + +(28) Scan parquet +Output [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/nation] +PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] +ReadSchema: struct + +(29) Filter +Input [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] +Condition : ((isnotnull(n_nationkey#48L) AND isnotnull(n_regionkey#50L)) AND might_contain(Subquery subquery#128, [id=#134], xxhash64(n_regionkey#50L, 42))) + +(30) BroadcastExchange +Input [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [plan_id=182] + +(31) BroadcastHashJoin +Left keys [1]: [s_nationkey#111L] +Right keys [1]: [n_nationkey#48L] +Join type: Inner +Join condition: None + +(32) Project +Output [4]: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L] +Input [6]: [l_extendedprice#21, l_discount#22, s_nationkey#111L, n_nationkey#48L, n_name#49, n_regionkey#50L] + +(33) Scan parquet +Output [2]: [r_regionkey#102L, r_name#103] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/region] +PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AFRICA), IsNotNull(r_regionkey)] +ReadSchema: struct + +(34) Filter +Input [2]: [r_regionkey#102L, r_name#103] +Condition : ((isnotnull(r_name#103) AND (r_name#103 = AFRICA)) AND isnotnull(r_regionkey#102L)) + +(35) Project +Output [1]: [r_regionkey#102L] +Input [2]: [r_regionkey#102L, r_name#103] + +(36) BroadcastExchange +Input [1]: [r_regionkey#102L] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [plan_id=186] + +(37) BroadcastHashJoin +Left keys [1]: [n_regionkey#50L] +Right keys [1]: [r_regionkey#102L] +Join type: Inner +Join condition: None + +(38) Project +Output [3]: [l_extendedprice#21, l_discount#22, n_name#49] +Input [5]: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L, r_regionkey#102L] + +(39) HashAggregate +Input [3]: [l_extendedprice#21, l_discount#22, n_name#49] +Keys [1]: [n_name#49] +Functions [1]: [partial_sum((l_extendedprice#21 * (1 - l_discount#22)))] +Aggregate Attributes [2]: [sum#129, isEmpty#130] +Results [3]: [n_name#49, sum#131, isEmpty#132] + +(40) Exchange +Input [3]: [n_name#49, sum#131, isEmpty#132] +Arguments: hashpartitioning(n_name#49, 200), ENSURE_REQUIREMENTS, [plan_id=191] + +(41) HashAggregate +Input [3]: [n_name#49, sum#131, isEmpty#132] +Keys [1]: [n_name#49] +Functions [1]: [sum((l_extendedprice#21 * (1 - l_discount#22)))] +Aggregate Attributes [1]: [sum((l_extendedprice#21 * (1 - l_discount#22)))#123] +Results [2]: [n_name#49, sum((l_extendedprice#21 * (1 - l_discount#22)))#123 AS revenue#122] + +(42) Exchange +Input [2]: [n_name#49, revenue#122] +Arguments: rangepartitioning(revenue#122 DESC NULLS LAST, 200), ENSURE_REQUIREMENTS, [plan_id=194] + +(43) Sort +Input [2]: [n_name#49, revenue#122] +Arguments: [revenue#122 DESC NULLS LAST], true, 0 + +(44) AdaptiveSparkPlan +Output [2]: [n_name#49, revenue#122] +Arguments: isFinalPlan=false + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 29 Hosting Expression = Subquery subquery#128, [id=#134] +AdaptiveSparkPlan (51) ++- ObjectHashAggregate (50) + +- Exchange (49) + +- ObjectHashAggregate (48) + +- Project (47) + +- Filter (46) + +- Scan parquet (45) + + +(45) Scan parquet +Output [2]: [r_regionkey#102L, r_name#103] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/region] +PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AFRICA), IsNotNull(r_regionkey)] +ReadSchema: struct + +(46) Filter +Input [2]: [r_regionkey#102L, r_name#103] +Condition : ((isnotnull(r_name#103) AND (r_name#103 = AFRICA)) AND isnotnull(r_regionkey#102L)) + +(47) Project +Output [1]: [r_regionkey#102L] +Input [2]: [r_regionkey#102L, r_name#103] + +(48) ObjectHashAggregate +Input [1]: [r_regionkey#102L] +Keys: [] +Functions [1]: [partial_bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)] +Aggregate Attributes [1]: [buf#133] +Results [1]: [buf#134] + +(49) Exchange +Input [1]: [buf#134] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=132] + +(50) ObjectHashAggregate +Input [1]: [buf#134] +Keys: [] +Functions [1]: [bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)] +Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)#126] +Results [1]: [bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)#126 AS bloomFilter#127] + +(51) AdaptiveSparkPlan +Output [1]: [bloomFilter#127] +Arguments: isFinalPlan=false + + + +Query 5 returned 5 rows, hash=f1e68aab4aa9d0988709357fc210d775 +Query 5 took 84.19 seconds + +Iteration 1 took 84.19 seconds + +Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/spark-offheap4g-q5-tpch-1775044656629.json diff --git a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.time b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.time new file mode 100644 index 0000000000..e276705a6c --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.time @@ -0,0 +1,4441 @@ +26/04/01 05:56:10 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) +26/04/01 05:56:10 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address +26/04/01 05:56:10 INFO SparkContext: Running Spark version 3.5.8 +26/04/01 05:56:10 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 05:56:10 INFO SparkContext: Java version 17.0.17 +26/04/01 05:56:10 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +26/04/01 05:56:10 INFO ResourceUtils: ============================================================== +26/04/01 05:56:10 INFO ResourceUtils: No custom resources configured for spark.driver. +26/04/01 05:56:10 INFO ResourceUtils: ============================================================== +26/04/01 05:56:10 INFO SparkContext: Submitted application: spark-offheap4g-q5 benchmark derived from tpch +26/04/01 05:56:10 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 4096, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) +26/04/01 05:56:10 INFO ResourceProfile: Limiting resource is cpu +26/04/01 05:56:10 INFO ResourceProfileManager: Added ResourceProfile id: 0 +26/04/01 05:56:10 INFO SecurityManager: Changing view acls to: andy +26/04/01 05:56:10 INFO SecurityManager: Changing modify acls to: andy +26/04/01 05:56:10 INFO SecurityManager: Changing view acls groups to: +26/04/01 05:56:10 INFO SecurityManager: Changing modify acls groups to: +26/04/01 05:56:10 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY +26/04/01 05:56:10 INFO Utils: Successfully started service 'sparkDriver' on port 58205. +26/04/01 05:56:10 INFO SparkEnv: Registering MapOutputTracker +26/04/01 05:56:10 INFO SparkEnv: Registering BlockManagerMaster +26/04/01 05:56:10 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information +26/04/01 05:56:10 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up +26/04/01 05:56:10 INFO SparkEnv: Registering BlockManagerMasterHeartbeat +26/04/01 05:56:10 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-3a98b41f-875a-442d-ba64-9adf3d2ffc14 +26/04/01 05:56:10 INFO MemoryStore: MemoryStore started with capacity 8.6 GiB +26/04/01 05:56:10 INFO SparkEnv: Registering OutputCommitCoordinator +26/04/01 05:56:10 INFO Executor: Starting executor ID driver on host 10.0.0.133 +26/04/01 05:56:10 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 05:56:10 INFO Executor: Java version 17.0.17 +26/04/01 05:56:10 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' +26/04/01 05:56:10 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@f860e41 for default. +26/04/01 05:56:10 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 58206. +26/04/01 05:56:10 INFO NettyBlockTransferService: Server created on 10.0.0.133:58206 +26/04/01 05:56:10 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy +26/04/01 05:56:10 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 58206, None) +26/04/01 05:56:10 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:58206 with 8.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 58206, None) +26/04/01 05:56:10 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 58206, None) +26/04/01 05:56:10 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 58206, None) +26/04/01 05:56:11 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. +26/04/01 05:56:11 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. +26/04/01 05:56:11 INFO InMemoryFileIndex: It took 18 ms to list leaf files for 1 paths. +26/04/01 05:56:11 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:56:11 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:56:11 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:56:11 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:56:11 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:11 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:56:11 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:56:11 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:56:11 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:11 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:11 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:56:11 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 +26/04/01 05:56:11 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) +26/04/01 05:56:11 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) +26/04/01 05:56:11 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 1998 bytes result sent to driver +26/04/01 05:56:11 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 118 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:56:11 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool +26/04/01 05:56:11 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.321 s +26/04/01 05:56:11 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:56:11 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished +26/04/01 05:56:11 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.332862 s +26/04/01 05:56:11 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. +26/04/01 05:56:12 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:56:12 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 +26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) +26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) +26/04/01 05:56:12 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2344 bytes result sent to driver +26/04/01 05:56:12 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 14 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool +26/04/01 05:56:12 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.023 s +26/04/01 05:56:12 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:56:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished +26/04/01 05:56:12 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.024672 s +26/04/01 05:56:12 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. +26/04/01 05:56:12 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:56:12 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 +26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) +26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) +26/04/01 05:56:12 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver +26/04/01 05:56:12 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 7 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool +26/04/01 05:56:12 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s +26/04/01 05:56:12 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:56:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished +26/04/01 05:56:12 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.017571 s +26/04/01 05:56:12 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. +26/04/01 05:56:12 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:56:12 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 +26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) +26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) +26/04/01 05:56:12 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver +26/04/01 05:56:12 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 7 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool +26/04/01 05:56:12 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s +26/04/01 05:56:12 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:56:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished +26/04/01 05:56:12 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.016001 s +26/04/01 05:56:12 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. +26/04/01 05:56:12 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:56:12 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 +26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9134 bytes) +26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) +26/04/01 05:56:12 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver +26/04/01 05:56:12 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool +26/04/01 05:56:12 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s +26/04/01 05:56:12 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:56:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished +26/04/01 05:56:12 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.014413 s +26/04/01 05:56:12 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 05:56:12 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:56:12 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 +26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) +26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) +26/04/01 05:56:12 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver +26/04/01 05:56:12 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 7 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool +26/04/01 05:56:12 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 05:56:12 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:56:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished +26/04/01 05:56:12 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.014465 s +26/04/01 05:56:12 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. +26/04/01 05:56:12 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:56:12 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 +26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) +26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) +26/04/01 05:56:12 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver +26/04/01 05:56:12 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool +26/04/01 05:56:12 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.012 s +26/04/01 05:56:12 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:56:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished +26/04/01 05:56:12 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.013261 s +26/04/01 05:56:12 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 05:56:12 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 05:56:12 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) +26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 +26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) +26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) +26/04/01 05:56:12 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver +26/04/01 05:56:12 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool +26/04/01 05:56:12 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.012 s +26/04/01 05:56:12 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:56:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished +26/04/01 05:56:12 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.013840 s +26/04/01 05:56:12 INFO FileSourceStrategy: Pushed Filters: IsNotNull(c_custkey),IsNotNull(c_nationkey) +26/04/01 05:56:12 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(c_custkey#0L),isnotnull(c_nationkey#3L) +26/04/01 05:56:12 INFO FileSourceStrategy: Pushed Filters: IsNotNull(o_orderdate),GreaterThanOrEqual(o_orderdate,1994-01-01),LessThan(o_orderdate,1995-01-01),IsNotNull(o_custkey),IsNotNull(o_orderkey) +26/04/01 05:56:12 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(o_orderdate#60),(o_orderdate#60 >= 1994-01-01),(o_orderdate#60 < 1995-01-01),isnotnull(o_custkey#57L),isnotnull(o_orderkey#56L) +26/04/01 05:56:12 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_orderkey),IsNotNull(l_suppkey) +26/04/01 05:56:12 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_orderkey#16L),isnotnull(l_suppkey#18L) +26/04/01 05:56:12 INFO FileSourceStrategy: Pushed Filters: IsNotNull(s_suppkey),IsNotNull(s_nationkey) +26/04/01 05:56:12 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(s_suppkey#108L),isnotnull(s_nationkey#111L) +26/04/01 05:56:12 INFO FileSourceStrategy: Pushed Filters: IsNotNull(n_nationkey),IsNotNull(n_regionkey) +26/04/01 05:56:12 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(n_nationkey#48L),isnotnull(n_regionkey#50L),might_contain(scalar-subquery#128 [], xxhash64(n_regionkey#50L, 42)) +26/04/01 05:56:12 INFO FileSourceStrategy: Pushed Filters: IsNotNull(r_name),EqualTo(r_name,AFRICA),IsNotNull(r_regionkey) +26/04/01 05:56:12 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(r_name#103),(r_name#103 = AFRICA),isnotnull(r_regionkey#102L) +26/04/01 05:56:12 INFO FileSourceStrategy: Pushed Filters: IsNotNull(r_name),EqualTo(r_name,AFRICA),IsNotNull(r_regionkey) +26/04/01 05:56:12 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(r_name#103),(r_name#103 = AFRICA),isnotnull(r_regionkey#102L) +26/04/01 05:56:12 INFO CodeGenerator: Code generated in 78.90275 ms +26/04/01 05:56:12 INFO CodeGenerator: Code generated in 79.198875 ms +26/04/01 05:56:12 INFO CodeGenerator: Code generated in 78.94675 ms +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:58206 (size: 35.0 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:58206 (size: 35.0 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:56:12 INFO SparkContext: Created broadcast 10 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:58206 (size: 35.0 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 9 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:56:12 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 05:56:12 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 05:56:12 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 05:56:12 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO DAGScheduler: Registering RDD 28 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 +26/04/01 05:56:12 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 16 output partitions +26/04/01 05:56:12 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:12 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:56:12 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 17.7 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:58206 (size: 7.9 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 11 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:12 INFO DAGScheduler: Submitting 16 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 8.0 with 16 tasks resource profile 0 +26/04/01 05:56:12 INFO CodeGenerator: Code generated in 10.717584 ms +26/04/01 05:56:12 INFO DAGScheduler: Registering RDD 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 +26/04/01 05:56:12 INFO DAGScheduler: Got map stage job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 05:56:12 INFO DAGScheduler: Final stage: ShuffleMapStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:12 INFO DAGScheduler: Submitting ShuffleMapStage 9 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:12 INFO TaskSetManager: Starting task 1.0 in stage 8.0 (TID 9) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:12 INFO TaskSetManager: Starting task 2.0 in stage 8.0 (TID 10) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 201.9 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO TaskSetManager: Starting task 3.0 in stage 8.0 (TID 11) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) +26/04/01 05:56:12 INFO Executor: Running task 1.0 in stage 8.0 (TID 9) +26/04/01 05:56:12 INFO Executor: Running task 2.0 in stage 8.0 (TID 10) +26/04/01 05:56:12 INFO Executor: Running task 3.0 in stage 8.0 (TID 11) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 35.2 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:58206 (size: 35.2 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 12 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:56:12 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_13 stored as values in memory (estimated size 31.4 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_13_piece0 stored as bytes in memory (estimated size 14.0 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_13_piece0 in memory on 10.0.0.133:58206 (size: 14.0 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 13 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 9 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 9.0 with 1 tasks resource profile 0 +26/04/01 05:56:12 INFO DAGScheduler: Got job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 10 (MapPartitionsRDD[25] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_14 stored as values in memory (estimated size 15.6 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_14_piece0 stored as bytes in memory (estimated size 6.7 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_14_piece0 in memory on 10.0.0.133:58206 (size: 6.7 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 14 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 10 (MapPartitionsRDD[25] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 10.0 with 1 tasks resource profile 0 +26/04/01 05:56:12 INFO DAGScheduler: Registering RDD 32 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 2 +26/04/01 05:56:12 INFO DAGScheduler: Got map stage job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 64 output partitions +26/04/01 05:56:12 INFO DAGScheduler: Final stage: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:12 INFO DAGScheduler: Submitting ShuffleMapStage 11 (MapPartitionsRDD[32] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:56:12 INFO CodeGenerator: Code generated in 7.381917 ms +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_15 stored as values in memory (estimated size 202.1 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_16 stored as values in memory (estimated size 19.2 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_16_piece0 stored as bytes in memory (estimated size 8.3 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_16_piece0 in memory on 10.0.0.133:58206 (size: 8.3 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 16 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:12 INFO DAGScheduler: Submitting 64 missing tasks from ShuffleMapStage 11 (MapPartitionsRDD[32] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 11.0 with 64 tasks resource profile 0 +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_15_piece0 stored as bytes in memory (estimated size 35.2 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_15_piece0 in memory on 10.0.0.133:58206 (size: 35.2 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO CodeGenerator: Code generated in 4.947959 ms +26/04/01 05:56:12 INFO SparkContext: Created broadcast 15 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:56:12 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 05:56:12 INFO DAGScheduler: Registering RDD 36 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 3 +26/04/01 05:56:12 INFO DAGScheduler: Got map stage job 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions +26/04/01 05:56:12 INFO DAGScheduler: Final stage: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:12 INFO DAGScheduler: Submitting ShuffleMapStage 12 (MapPartitionsRDD[36] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_17 stored as values in memory (estimated size 19.2 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_17_piece0 stored as bytes in memory (estimated size 8.3 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_17_piece0 in memory on 10.0.0.133:58206 (size: 8.3 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 17 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:12 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 12 (MapPartitionsRDD[36] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 12.0 with 208 tasks resource profile 0 +26/04/01 05:56:12 INFO CodeGenerator: Code generated in 5.984708 ms +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_18 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO CodeGenerator: Code generated in 6.960333 ms +26/04/01 05:56:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.28.parquet, range: 0-43421251, partition values: [empty row] +26/04/01 05:56:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.32.parquet, range: 0-43435385, partition values: [empty row] +26/04/01 05:56:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.1.parquet, range: 0-43414283, partition values: [empty row] +26/04/01 05:56:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.24.parquet, range: 0-43412266, partition values: [empty row] +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_18_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_18_piece0 in memory on 10.0.0.133:58206 (size: 35.0 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 18 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:56:12 INFO FileSourceScanExec: Planning scan with bin packing, max size: 55920126 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 05:56:12 INFO DAGScheduler: Registering RDD 40 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 4 +26/04/01 05:56:12 INFO DAGScheduler: Got map stage job 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 4 output partitions +26/04/01 05:56:12 INFO DAGScheduler: Final stage: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:12 INFO DAGScheduler: Submitting ShuffleMapStage 13 (MapPartitionsRDD[40] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_19 stored as values in memory (estimated size 17.8 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO MemoryStore: Block broadcast_19_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 8.6 GiB) +26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_19_piece0 in memory on 10.0.0.133:58206 (size: 7.9 KiB, free: 8.6 GiB) +26/04/01 05:56:12 INFO SparkContext: Created broadcast 19 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:12 INFO DAGScheduler: Submitting 4 missing tasks from ShuffleMapStage 13 (MapPartitionsRDD[40] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3)) +26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 13.0 with 4 tasks resource profile 0 +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 05:56:13 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 05:56:13 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 05:56:13 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.7.parquet, range: 0-43422275, partition values: [empty row] +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.26.parquet, range: 0-43415615, partition values: [empty row] +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.22.parquet, range: 0-43412913, partition values: [empty row] +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.15.parquet, range: 0-43410292, partition values: [empty row] +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO Executor: Finished task 3.0 in stage 8.0 (TID 11). 2294 bytes result sent to driver +26/04/01 05:56:13 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 2294 bytes result sent to driver +26/04/01 05:56:13 INFO Executor: Finished task 1.0 in stage 8.0 (TID 9). 2294 bytes result sent to driver +26/04/01 05:56:13 INFO Executor: Finished task 2.0 in stage 8.0 (TID 10). 2294 bytes result sent to driver +26/04/01 05:56:13 INFO TaskSetManager: Starting task 4.0 in stage 8.0 (TID 12) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:13 INFO Executor: Running task 4.0 in stage 8.0 (TID 12) +26/04/01 05:56:13 INFO TaskSetManager: Starting task 5.0 in stage 8.0 (TID 13) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:13 INFO TaskSetManager: Finished task 3.0 in stage 8.0 (TID 11) in 669 ms on 10.0.0.133 (executor driver) (1/16) +26/04/01 05:56:13 INFO TaskSetManager: Starting task 6.0 in stage 8.0 (TID 14) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:13 INFO Executor: Running task 5.0 in stage 8.0 (TID 13) +26/04/01 05:56:13 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 672 ms on 10.0.0.133 (executor driver) (2/16) +26/04/01 05:56:13 INFO Executor: Running task 6.0 in stage 8.0 (TID 14) +26/04/01 05:56:13 INFO TaskSetManager: Finished task 1.0 in stage 8.0 (TID 9) in 670 ms on 10.0.0.133 (executor driver) (3/16) +26/04/01 05:56:13 INFO TaskSetManager: Starting task 7.0 in stage 8.0 (TID 15) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:13 INFO Executor: Running task 7.0 in stage 8.0 (TID 15) +26/04/01 05:56:13 INFO TaskSetManager: Finished task 2.0 in stage 8.0 (TID 10) in 670 ms on 10.0.0.133 (executor driver) (4/16) +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.9.parquet, range: 0-43409639, partition values: [empty row] +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.18.parquet, range: 0-43409175, partition values: [empty row] +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.30.parquet, range: 0-43407009, partition values: [empty row] +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.11.parquet, range: 0-43405290, partition values: [empty row] +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.5.parquet, range: 0-43408011, partition values: [empty row] +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.31.parquet, range: 0-43405268, partition values: [empty row] +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.20.parquet, range: 0-43409316, partition values: [empty row] +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.8.parquet, range: 0-43406367, partition values: [empty row] +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO Executor: Finished task 5.0 in stage 8.0 (TID 13). 2294 bytes result sent to driver +26/04/01 05:56:13 INFO Executor: Finished task 7.0 in stage 8.0 (TID 15). 2294 bytes result sent to driver +26/04/01 05:56:13 INFO TaskSetManager: Starting task 8.0 in stage 8.0 (TID 16) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:13 INFO Executor: Running task 8.0 in stage 8.0 (TID 16) +26/04/01 05:56:13 INFO TaskSetManager: Starting task 9.0 in stage 8.0 (TID 17) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:13 INFO TaskSetManager: Finished task 5.0 in stage 8.0 (TID 13) in 328 ms on 10.0.0.133 (executor driver) (5/16) +26/04/01 05:56:13 INFO TaskSetManager: Finished task 7.0 in stage 8.0 (TID 15) in 327 ms on 10.0.0.133 (executor driver) (6/16) +26/04/01 05:56:13 INFO Executor: Running task 9.0 in stage 8.0 (TID 17) +26/04/01 05:56:13 INFO Executor: Finished task 4.0 in stage 8.0 (TID 12). 2294 bytes result sent to driver +26/04/01 05:56:13 INFO TaskSetManager: Starting task 10.0 in stage 8.0 (TID 18) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:13 INFO Executor: Running task 10.0 in stage 8.0 (TID 18) +26/04/01 05:56:13 INFO TaskSetManager: Finished task 4.0 in stage 8.0 (TID 12) in 331 ms on 10.0.0.133 (executor driver) (7/16) +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.2.parquet, range: 0-43405025, partition values: [empty row] +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.19.parquet, range: 0-43404330, partition values: [empty row] +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.13.parquet, range: 0-43401020, partition values: [empty row] +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO Executor: Finished task 6.0 in stage 8.0 (TID 14). 2294 bytes result sent to driver +26/04/01 05:56:13 INFO TaskSetManager: Starting task 11.0 in stage 8.0 (TID 19) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:13 INFO TaskSetManager: Finished task 6.0 in stage 8.0 (TID 14) in 337 ms on 10.0.0.133 (executor driver) (8/16) +26/04/01 05:56:13 INFO Executor: Running task 11.0 in stage 8.0 (TID 19) +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.4.parquet, range: 0-43397542, partition values: [empty row] +26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.23.parquet, range: 0-43398227, partition values: [empty row] +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.10.parquet, range: 0-43404438, partition values: [empty row] +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.6.parquet, range: 0-43402993, partition values: [empty row] +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.21.parquet, range: 0-43396532, partition values: [empty row] +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:14 INFO Executor: Finished task 10.0 in stage 8.0 (TID 18). 2294 bytes result sent to driver +26/04/01 05:56:14 INFO Executor: Finished task 8.0 in stage 8.0 (TID 16). 2294 bytes result sent to driver +26/04/01 05:56:14 INFO TaskSetManager: Starting task 12.0 in stage 8.0 (TID 20) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:14 INFO TaskSetManager: Finished task 10.0 in stage 8.0 (TID 18) in 230 ms on 10.0.0.133 (executor driver) (9/16) +26/04/01 05:56:14 INFO Executor: Running task 12.0 in stage 8.0 (TID 20) +26/04/01 05:56:14 INFO TaskSetManager: Starting task 13.0 in stage 8.0 (TID 21) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.12.parquet, range: 0-43396359, partition values: [empty row] +26/04/01 05:56:14 INFO TaskSetManager: Finished task 8.0 in stage 8.0 (TID 16) in 236 ms on 10.0.0.133 (executor driver) (10/16) +26/04/01 05:56:14 INFO Executor: Running task 13.0 in stage 8.0 (TID 21) +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.29.parquet, range: 0-43391813, partition values: [empty row] +26/04/01 05:56:14 INFO Executor: Finished task 9.0 in stage 8.0 (TID 17). 2294 bytes result sent to driver +26/04/01 05:56:14 INFO TaskSetManager: Starting task 14.0 in stage 8.0 (TID 22) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:14 INFO TaskSetManager: Finished task 9.0 in stage 8.0 (TID 17) in 238 ms on 10.0.0.133 (executor driver) (11/16) +26/04/01 05:56:14 INFO Executor: Running task 14.0 in stage 8.0 (TID 22) +26/04/01 05:56:14 INFO Executor: Finished task 11.0 in stage 8.0 (TID 19). 2251 bytes result sent to driver +26/04/01 05:56:14 INFO TaskSetManager: Starting task 15.0 in stage 8.0 (TID 23) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.16.parquet, range: 0-43385525, partition values: [empty row] +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:14 INFO TaskSetManager: Finished task 11.0 in stage 8.0 (TID 19) in 231 ms on 10.0.0.133 (executor driver) (12/16) +26/04/01 05:56:14 INFO Executor: Running task 15.0 in stage 8.0 (TID 23) +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.25.parquet, range: 0-43376403, partition values: [empty row] +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.17.parquet, range: 0-43381052, partition values: [empty row] +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.3.parquet, range: 0-43385934, partition values: [empty row] +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.27.parquet, range: 0-43393057, partition values: [empty row] +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.14.parquet, range: 0-43368555, partition values: [empty row] +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) +26/04/01 05:56:14 INFO Executor: Finished task 12.0 in stage 8.0 (TID 20). 2251 bytes result sent to driver +26/04/01 05:56:14 INFO TaskSetManager: Starting task 0.0 in stage 9.0 (TID 24) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:14 INFO Executor: Running task 0.0 in stage 9.0 (TID 24) +26/04/01 05:56:14 INFO TaskSetManager: Finished task 12.0 in stage 8.0 (TID 20) in 217 ms on 10.0.0.133 (executor driver) (13/16) +26/04/01 05:56:14 INFO Executor: Finished task 15.0 in stage 8.0 (TID 23). 2294 bytes result sent to driver +26/04/01 05:56:14 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 25) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9608 bytes) +26/04/01 05:56:14 INFO Executor: Finished task 13.0 in stage 8.0 (TID 21). 2251 bytes result sent to driver +26/04/01 05:56:14 INFO Executor: Running task 0.0 in stage 10.0 (TID 25) +26/04/01 05:56:14 INFO TaskSetManager: Starting task 0.0 in stage 11.0 (TID 26) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:14 INFO TaskSetManager: Finished task 15.0 in stage 8.0 (TID 23) in 214 ms on 10.0.0.133 (executor driver) (14/16) +26/04/01 05:56:14 INFO Executor: Running task 0.0 in stage 11.0 (TID 26) +26/04/01 05:56:14 INFO TaskSetManager: Finished task 13.0 in stage 8.0 (TID 21) in 222 ms on 10.0.0.133 (executor driver) (15/16) +26/04/01 05:56:14 INFO Executor: Finished task 14.0 in stage 8.0 (TID 22). 2294 bytes result sent to driver +26/04/01 05:56:14 INFO TaskSetManager: Starting task 1.0 in stage 11.0 (TID 27) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:14 INFO Executor: Running task 1.0 in stage 11.0 (TID 27) +26/04/01 05:56:14 INFO TaskSetManager: Finished task 14.0 in stage 8.0 (TID 22) in 218 ms on 10.0.0.133 (executor driver) (16/16) +26/04/01 05:56:14 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool +26/04/01 05:56:14 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 1.464 s +26/04/01 05:56:14 INFO DAGScheduler: looking for newly runnable stages +26/04/01 05:56:14 INFO DAGScheduler: running: Set(ShuffleMapStage 12, ShuffleMapStage 9, ShuffleMapStage 13, ResultStage 10, ShuffleMapStage 11) +26/04/01 05:56:14 INFO CodeGenerator: Code generated in 5.694291 ms +26/04/01 05:56:14 INFO DAGScheduler: waiting: Set() +26/04/01 05:56:14 INFO DAGScheduler: failed: Set() +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/region/region.1.parquet, range: 0-1227, partition values: [empty row] +26/04/01 05:56:14 INFO CodeGenerator: Code generated in 6.989792 ms +26/04/01 05:56:14 INFO CodeGenerator: Code generated in 5.905167 ms +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/region/region.1.parquet, range: 0-1227, partition values: [empty row] +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(r_name, null), eq(r_name, Binary{"AFRICA"})), noteq(r_regionkey, null)) +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(r_name, null), eq(r_name, Binary{"AFRICA"})), noteq(r_regionkey, null)) +26/04/01 05:56:14 INFO CodeGenerator: Code generated in 4.1855 ms +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.7.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.13.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:14 INFO Executor: Finished task 0.0 in stage 10.0 (TID 25). 1848 bytes result sent to driver +26/04/01 05:56:14 INFO TaskSetManager: Starting task 2.0 in stage 11.0 (TID 28) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:14 INFO Executor: Running task 2.0 in stage 11.0 (TID 28) +26/04/01 05:56:14 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 25) in 21 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:56:14 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool +26/04/01 05:56:14 INFO DAGScheduler: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 1.455 s +26/04/01 05:56:14 INFO DAGScheduler: Job 10 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:56:14 INFO TaskSchedulerImpl: Killing all running tasks in stage 10: Stage finished +26/04/01 05:56:14 INFO DAGScheduler: Job 10 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 1.479527 s +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.25.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:14 INFO CodeGenerator: Code generated in 2.792583 ms +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:14 INFO CodeGenerator: Code generated in 2.404583 ms +26/04/01 05:56:14 INFO MemoryStore: Block broadcast_20 stored as values in memory (estimated size 1024.0 KiB, free 8.6 GiB) +26/04/01 05:56:14 INFO MemoryStore: Block broadcast_20_piece0 stored as bytes in memory (estimated size 185.0 B, free 8.6 GiB) +26/04/01 05:56:14 INFO BlockManagerInfo: Added broadcast_20_piece0 in memory on 10.0.0.133:58206 (size: 185.0 B, free: 8.6 GiB) +26/04/01 05:56:14 INFO SparkContext: Created broadcast 20 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:56:14 INFO CodeGenerator: Code generated in 2.952166 ms +26/04/01 05:56:14 INFO CodeGenerator: Code generated in 4.225375 ms +26/04/01 05:56:14 INFO Executor: Finished task 0.0 in stage 9.0 (TID 24). 2580 bytes result sent to driver +26/04/01 05:56:14 INFO TaskSetManager: Starting task 3.0 in stage 11.0 (TID 29) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:14 INFO Executor: Running task 3.0 in stage 11.0 (TID 29) +26/04/01 05:56:14 INFO TaskSetManager: Finished task 0.0 in stage 9.0 (TID 24) in 56 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:56:14 INFO TaskSchedulerImpl: Removed TaskSet 9.0, whose tasks have all completed, from pool +26/04/01 05:56:14 INFO DAGScheduler: ShuffleMapStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 1.502 s +26/04/01 05:56:14 INFO DAGScheduler: looking for newly runnable stages +26/04/01 05:56:14 INFO DAGScheduler: running: Set(ShuffleMapStage 12, ShuffleMapStage 13, ShuffleMapStage 11) +26/04/01 05:56:14 INFO DAGScheduler: waiting: Set() +26/04/01 05:56:14 INFO DAGScheduler: failed: Set() +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.24.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:14 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:56:14 INFO DAGScheduler: Got job 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 05:56:14 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:56:14 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 14) +26/04/01 05:56:14 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:14 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[43] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:56:14 INFO MemoryStore: Block broadcast_21 stored as values in memory (estimated size 32.0 KiB, free 8.6 GiB) +26/04/01 05:56:14 INFO MemoryStore: Block broadcast_21_piece0 stored as bytes in memory (estimated size 14.9 KiB, free 8.6 GiB) +26/04/01 05:56:14 INFO BlockManagerInfo: Added broadcast_21_piece0 in memory on 10.0.0.133:58206 (size: 14.9 KiB, free: 8.6 GiB) +26/04/01 05:56:14 INFO SparkContext: Created broadcast 21 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:14 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 15 (MapPartitionsRDD[43] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:56:14 INFO TaskSchedulerImpl: Adding task set 15.0 with 1 tasks resource profile 0 +26/04/01 05:56:14 INFO Executor: Finished task 3.0 in stage 11.0 (TID 29). 2294 bytes result sent to driver +26/04/01 05:56:14 INFO TaskSetManager: Starting task 4.0 in stage 11.0 (TID 30) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:14 INFO TaskSetManager: Finished task 3.0 in stage 11.0 (TID 29) in 305 ms on 10.0.0.133 (executor driver) (1/64) +26/04/01 05:56:14 INFO Executor: Running task 4.0 in stage 11.0 (TID 30) +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.12.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:14 INFO Executor: Finished task 2.0 in stage 11.0 (TID 28). 2251 bytes result sent to driver +26/04/01 05:56:14 INFO TaskSetManager: Starting task 5.0 in stage 11.0 (TID 31) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:14 INFO TaskSetManager: Finished task 2.0 in stage 11.0 (TID 28) in 338 ms on 10.0.0.133 (executor driver) (2/64) +26/04/01 05:56:14 INFO Executor: Running task 5.0 in stage 11.0 (TID 31) +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.6.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:14 INFO Executor: Finished task 1.0 in stage 11.0 (TID 27). 2251 bytes result sent to driver +26/04/01 05:56:14 INFO TaskSetManager: Starting task 6.0 in stage 11.0 (TID 32) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:14 INFO Executor: Running task 6.0 in stage 11.0 (TID 32) +26/04/01 05:56:14 INFO TaskSetManager: Finished task 1.0 in stage 11.0 (TID 27) in 361 ms on 10.0.0.133 (executor driver) (3/64) +26/04/01 05:56:14 INFO Executor: Finished task 0.0 in stage 11.0 (TID 26). 2251 bytes result sent to driver +26/04/01 05:56:14 INFO TaskSetManager: Starting task 7.0 in stage 11.0 (TID 33) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:14 INFO TaskSetManager: Finished task 0.0 in stage 11.0 (TID 26) in 364 ms on 10.0.0.133 (executor driver) (4/64) +26/04/01 05:56:14 INFO Executor: Running task 7.0 in stage 11.0 (TID 33) +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.19.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.26.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:14 INFO Executor: Finished task 4.0 in stage 11.0 (TID 30). 2251 bytes result sent to driver +26/04/01 05:56:14 INFO TaskSetManager: Starting task 8.0 in stage 11.0 (TID 34) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:15 INFO TaskSetManager: Finished task 4.0 in stage 11.0 (TID 30) in 260 ms on 10.0.0.133 (executor driver) (5/64) +26/04/01 05:56:15 INFO Executor: Running task 8.0 in stage 11.0 (TID 34) +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.4.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO Executor: Finished task 6.0 in stage 11.0 (TID 32). 2251 bytes result sent to driver +26/04/01 05:56:15 INFO TaskSetManager: Starting task 9.0 in stage 11.0 (TID 35) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:15 INFO Executor: Running task 9.0 in stage 11.0 (TID 35) +26/04/01 05:56:15 INFO TaskSetManager: Finished task 6.0 in stage 11.0 (TID 32) in 255 ms on 10.0.0.133 (executor driver) (6/64) +26/04/01 05:56:15 INFO Executor: Finished task 5.0 in stage 11.0 (TID 31). 2251 bytes result sent to driver +26/04/01 05:56:15 INFO TaskSetManager: Starting task 10.0 in stage 11.0 (TID 36) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:15 INFO TaskSetManager: Finished task 5.0 in stage 11.0 (TID 31) in 261 ms on 10.0.0.133 (executor driver) (7/64) +26/04/01 05:56:15 INFO Executor: Finished task 7.0 in stage 11.0 (TID 33). 2251 bytes result sent to driver +26/04/01 05:56:15 INFO Executor: Running task 10.0 in stage 11.0 (TID 36) +26/04/01 05:56:15 INFO TaskSetManager: Starting task 11.0 in stage 11.0 (TID 37) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.10.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO TaskSetManager: Finished task 7.0 in stage 11.0 (TID 33) in 256 ms on 10.0.0.133 (executor driver) (8/64) +26/04/01 05:56:15 INFO Executor: Running task 11.0 in stage 11.0 (TID 37) +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.5.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.11.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO Executor: Finished task 8.0 in stage 11.0 (TID 34). 2294 bytes result sent to driver +26/04/01 05:56:15 INFO TaskSetManager: Starting task 12.0 in stage 11.0 (TID 38) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:15 INFO TaskSetManager: Finished task 8.0 in stage 11.0 (TID 34) in 251 ms on 10.0.0.133 (executor driver) (9/64) +26/04/01 05:56:15 INFO Executor: Running task 12.0 in stage 11.0 (TID 38) +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.18.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO Executor: Finished task 10.0 in stage 11.0 (TID 36). 2251 bytes result sent to driver +26/04/01 05:56:15 INFO TaskSetManager: Starting task 13.0 in stage 11.0 (TID 39) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:15 INFO TaskSetManager: Finished task 10.0 in stage 11.0 (TID 36) in 258 ms on 10.0.0.133 (executor driver) (10/64) +26/04/01 05:56:15 INFO Executor: Running task 13.0 in stage 11.0 (TID 39) +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.27.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO Executor: Finished task 11.0 in stage 11.0 (TID 37). 2251 bytes result sent to driver +26/04/01 05:56:15 INFO TaskSetManager: Starting task 14.0 in stage 11.0 (TID 40) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:15 INFO Executor: Finished task 9.0 in stage 11.0 (TID 35). 2251 bytes result sent to driver +26/04/01 05:56:15 INFO TaskSetManager: Finished task 11.0 in stage 11.0 (TID 37) in 261 ms on 10.0.0.133 (executor driver) (11/64) +26/04/01 05:56:15 INFO Executor: Running task 14.0 in stage 11.0 (TID 40) +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO TaskSetManager: Starting task 15.0 in stage 11.0 (TID 41) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:15 INFO Executor: Running task 15.0 in stage 11.0 (TID 41) +26/04/01 05:56:15 INFO TaskSetManager: Finished task 9.0 in stage 11.0 (TID 35) in 264 ms on 10.0.0.133 (executor driver) (12/64) +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.32.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.22.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO Executor: Finished task 12.0 in stage 11.0 (TID 38). 2251 bytes result sent to driver +26/04/01 05:56:15 INFO TaskSetManager: Starting task 16.0 in stage 11.0 (TID 42) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:15 INFO TaskSetManager: Finished task 12.0 in stage 11.0 (TID 38) in 248 ms on 10.0.0.133 (executor driver) (13/64) +26/04/01 05:56:15 INFO Executor: Running task 16.0 in stage 11.0 (TID 42) +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.9.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO Executor: Finished task 13.0 in stage 11.0 (TID 39). 2251 bytes result sent to driver +26/04/01 05:56:15 INFO TaskSetManager: Starting task 17.0 in stage 11.0 (TID 43) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:15 INFO TaskSetManager: Finished task 13.0 in stage 11.0 (TID 39) in 251 ms on 10.0.0.133 (executor driver) (14/64) +26/04/01 05:56:15 INFO Executor: Running task 17.0 in stage 11.0 (TID 43) +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.14.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO Executor: Finished task 14.0 in stage 11.0 (TID 40). 2251 bytes result sent to driver +26/04/01 05:56:15 INFO TaskSetManager: Starting task 18.0 in stage 11.0 (TID 44) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:15 INFO TaskSetManager: Finished task 14.0 in stage 11.0 (TID 40) in 251 ms on 10.0.0.133 (executor driver) (15/64) +26/04/01 05:56:15 INFO Executor: Finished task 15.0 in stage 11.0 (TID 41). 2251 bytes result sent to driver +26/04/01 05:56:15 INFO Executor: Running task 18.0 in stage 11.0 (TID 44) +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO TaskSetManager: Starting task 19.0 in stage 11.0 (TID 45) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:15 INFO TaskSetManager: Finished task 15.0 in stage 11.0 (TID 41) in 250 ms on 10.0.0.133 (executor driver) (16/64) +26/04/01 05:56:15 INFO Executor: Running task 19.0 in stage 11.0 (TID 45) +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.1.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.15.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO Executor: Finished task 16.0 in stage 11.0 (TID 42). 2251 bytes result sent to driver +26/04/01 05:56:15 INFO TaskSetManager: Starting task 20.0 in stage 11.0 (TID 46) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:15 INFO Executor: Running task 20.0 in stage 11.0 (TID 46) +26/04/01 05:56:15 INFO TaskSetManager: Finished task 16.0 in stage 11.0 (TID 42) in 256 ms on 10.0.0.133 (executor driver) (17/64) +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.23.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO Executor: Finished task 17.0 in stage 11.0 (TID 43). 2251 bytes result sent to driver +26/04/01 05:56:15 INFO TaskSetManager: Starting task 21.0 in stage 11.0 (TID 47) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:15 INFO TaskSetManager: Finished task 17.0 in stage 11.0 (TID 43) in 262 ms on 10.0.0.133 (executor driver) (18/64) +26/04/01 05:56:15 INFO Executor: Running task 21.0 in stage 11.0 (TID 47) +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.8.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO Executor: Finished task 19.0 in stage 11.0 (TID 45). 2251 bytes result sent to driver +26/04/01 05:56:15 INFO TaskSetManager: Starting task 22.0 in stage 11.0 (TID 48) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:15 INFO Executor: Running task 22.0 in stage 11.0 (TID 48) +26/04/01 05:56:15 INFO TaskSetManager: Finished task 19.0 in stage 11.0 (TID 45) in 262 ms on 10.0.0.133 (executor driver) (19/64) +26/04/01 05:56:15 INFO Executor: Finished task 18.0 in stage 11.0 (TID 44). 2251 bytes result sent to driver +26/04/01 05:56:15 INFO TaskSetManager: Starting task 23.0 in stage 11.0 (TID 49) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:15 INFO TaskSetManager: Finished task 18.0 in stage 11.0 (TID 44) in 262 ms on 10.0.0.133 (executor driver) (20/64) +26/04/01 05:56:15 INFO Executor: Running task 23.0 in stage 11.0 (TID 49) +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.28.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.17.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:15 INFO Executor: Finished task 20.0 in stage 11.0 (TID 46). 2251 bytes result sent to driver +26/04/01 05:56:15 INFO TaskSetManager: Starting task 24.0 in stage 11.0 (TID 50) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:15 INFO TaskSetManager: Finished task 20.0 in stage 11.0 (TID 46) in 222 ms on 10.0.0.133 (executor driver) (21/64) +26/04/01 05:56:15 INFO Executor: Running task 24.0 in stage 11.0 (TID 50) +26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.3.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO Executor: Finished task 22.0 in stage 11.0 (TID 48). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO TaskSetManager: Starting task 25.0 in stage 11.0 (TID 51) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:16 INFO Executor: Finished task 21.0 in stage 11.0 (TID 47). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO Executor: Running task 25.0 in stage 11.0 (TID 51) +26/04/01 05:56:16 INFO TaskSetManager: Starting task 26.0 in stage 11.0 (TID 52) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 22.0 in stage 11.0 (TID 48) in 238 ms on 10.0.0.133 (executor driver) (22/64) +26/04/01 05:56:16 INFO Executor: Running task 26.0 in stage 11.0 (TID 52) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 21.0 in stage 11.0 (TID 47) in 241 ms on 10.0.0.133 (executor driver) (23/64) +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.21.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.31.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:16 INFO Executor: Finished task 23.0 in stage 11.0 (TID 49). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO TaskSetManager: Starting task 27.0 in stage 11.0 (TID 53) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 23.0 in stage 11.0 (TID 49) in 239 ms on 10.0.0.133 (executor driver) (24/64) +26/04/01 05:56:16 INFO Executor: Running task 27.0 in stage 11.0 (TID 53) +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.20.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO Executor: Finished task 24.0 in stage 11.0 (TID 50). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO TaskSetManager: Starting task 28.0 in stage 11.0 (TID 54) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 24.0 in stage 11.0 (TID 50) in 229 ms on 10.0.0.133 (executor driver) (25/64) +26/04/01 05:56:16 INFO Executor: Running task 28.0 in stage 11.0 (TID 54) +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.30.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO Executor: Finished task 26.0 in stage 11.0 (TID 52). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO Executor: Finished task 27.0 in stage 11.0 (TID 53). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO TaskSetManager: Starting task 29.0 in stage 11.0 (TID 55) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 26.0 in stage 11.0 (TID 52) in 248 ms on 10.0.0.133 (executor driver) (26/64) +26/04/01 05:56:16 INFO Executor: Running task 29.0 in stage 11.0 (TID 55) +26/04/01 05:56:16 INFO TaskSetManager: Starting task 30.0 in stage 11.0 (TID 56) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 27.0 in stage 11.0 (TID 53) in 247 ms on 10.0.0.133 (executor driver) (27/64) +26/04/01 05:56:16 INFO Executor: Finished task 25.0 in stage 11.0 (TID 51). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO Executor: Running task 30.0 in stage 11.0 (TID 56) +26/04/01 05:56:16 INFO TaskSetManager: Starting task 31.0 in stage 11.0 (TID 57) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 25.0 in stage 11.0 (TID 51) in 249 ms on 10.0.0.133 (executor driver) (28/64) +26/04/01 05:56:16 INFO Executor: Running task 31.0 in stage 11.0 (TID 57) +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.29.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.16.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.2.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO Executor: Finished task 28.0 in stage 11.0 (TID 54). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO TaskSetManager: Starting task 32.0 in stage 11.0 (TID 58) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:16 INFO Executor: Running task 32.0 in stage 11.0 (TID 58) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 28.0 in stage 11.0 (TID 54) in 223 ms on 10.0.0.133 (executor driver) (29/64) +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.11.parquet, range: 134217728-233581632, partition values: [empty row] +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO Executor: Finished task 31.0 in stage 11.0 (TID 57). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO TaskSetManager: Starting task 33.0 in stage 11.0 (TID 59) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 31.0 in stage 11.0 (TID 57) in 250 ms on 10.0.0.133 (executor driver) (30/64) +26/04/01 05:56:16 INFO Executor: Running task 33.0 in stage 11.0 (TID 59) +26/04/01 05:56:16 INFO Executor: Finished task 30.0 in stage 11.0 (TID 56). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO TaskSetManager: Starting task 34.0 in stage 11.0 (TID 60) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.23.parquet, range: 134217728-233580600, partition values: [empty row] +26/04/01 05:56:16 INFO TaskSetManager: Finished task 30.0 in stage 11.0 (TID 56) in 252 ms on 10.0.0.133 (executor driver) (31/64) +26/04/01 05:56:16 INFO Executor: Running task 34.0 in stage 11.0 (TID 60) +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.30.parquet, range: 134217728-233577812, partition values: [empty row] +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO Executor: Finished task 29.0 in stage 11.0 (TID 55). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO TaskSetManager: Starting task 35.0 in stage 11.0 (TID 61) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 29.0 in stage 11.0 (TID 55) in 255 ms on 10.0.0.133 (executor driver) (32/64) +26/04/01 05:56:16 INFO Executor: Running task 35.0 in stage 11.0 (TID 61) +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.7.parquet, range: 134217728-233566955, partition values: [empty row] +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO Executor: Finished task 32.0 in stage 11.0 (TID 58). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO TaskSetManager: Starting task 36.0 in stage 11.0 (TID 62) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 32.0 in stage 11.0 (TID 58) in 169 ms on 10.0.0.133 (executor driver) (33/64) +26/04/01 05:56:16 INFO Executor: Running task 36.0 in stage 11.0 (TID 62) +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.12.parquet, range: 134217728-233565571, partition values: [empty row] +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO Executor: Finished task 34.0 in stage 11.0 (TID 60). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO TaskSetManager: Starting task 37.0 in stage 11.0 (TID 63) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:16 INFO Executor: Running task 37.0 in stage 11.0 (TID 63) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 34.0 in stage 11.0 (TID 60) in 195 ms on 10.0.0.133 (executor driver) (34/64) +26/04/01 05:56:16 INFO Executor: Finished task 33.0 in stage 11.0 (TID 59). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO TaskSetManager: Starting task 38.0 in stage 11.0 (TID 64) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:16 INFO Executor: Running task 38.0 in stage 11.0 (TID 64) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 33.0 in stage 11.0 (TID 59) in 199 ms on 10.0.0.133 (executor driver) (35/64) +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.16.parquet, range: 134217728-233564116, partition values: [empty row] +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.32.parquet, range: 134217728-233563384, partition values: [empty row] +26/04/01 05:56:16 INFO Executor: Finished task 35.0 in stage 11.0 (TID 61). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO TaskSetManager: Starting task 39.0 in stage 11.0 (TID 65) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:16 INFO Executor: Running task 39.0 in stage 11.0 (TID 65) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 35.0 in stage 11.0 (TID 61) in 196 ms on 10.0.0.133 (executor driver) (36/64) +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.14.parquet, range: 134217728-233562716, partition values: [empty row] +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO Executor: Finished task 36.0 in stage 11.0 (TID 62). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO TaskSetManager: Starting task 40.0 in stage 11.0 (TID 66) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 36.0 in stage 11.0 (TID 62) in 175 ms on 10.0.0.133 (executor driver) (37/64) +26/04/01 05:56:16 INFO Executor: Running task 40.0 in stage 11.0 (TID 66) +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.5.parquet, range: 134217728-233556822, partition values: [empty row] +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO Executor: Finished task 37.0 in stage 11.0 (TID 63). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO TaskSetManager: Starting task 41.0 in stage 11.0 (TID 67) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 37.0 in stage 11.0 (TID 63) in 203 ms on 10.0.0.133 (executor driver) (38/64) +26/04/01 05:56:16 INFO Executor: Running task 41.0 in stage 11.0 (TID 67) +26/04/01 05:56:16 INFO Executor: Finished task 39.0 in stage 11.0 (TID 65). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO TaskSetManager: Starting task 42.0 in stage 11.0 (TID 68) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:16 INFO Executor: Finished task 38.0 in stage 11.0 (TID 64). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO Executor: Running task 42.0 in stage 11.0 (TID 68) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 39.0 in stage 11.0 (TID 65) in 200 ms on 10.0.0.133 (executor driver) (39/64) +26/04/01 05:56:16 INFO TaskSetManager: Starting task 43.0 in stage 11.0 (TID 69) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 38.0 in stage 11.0 (TID 64) in 203 ms on 10.0.0.133 (executor driver) (40/64) +26/04/01 05:56:16 INFO Executor: Running task 43.0 in stage 11.0 (TID 69) +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.27.parquet, range: 134217728-233556807, partition values: [empty row] +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.19.parquet, range: 134217728-233556337, partition values: [empty row] +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.1.parquet, range: 134217728-233553544, partition values: [empty row] +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:16 INFO Executor: Finished task 40.0 in stage 11.0 (TID 66). 2251 bytes result sent to driver +26/04/01 05:56:16 INFO TaskSetManager: Starting task 44.0 in stage 11.0 (TID 70) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:16 INFO Executor: Running task 44.0 in stage 11.0 (TID 70) +26/04/01 05:56:16 INFO TaskSetManager: Finished task 40.0 in stage 11.0 (TID 66) in 185 ms on 10.0.0.133 (executor driver) (41/64) +26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.6.parquet, range: 134217728-233549987, partition values: [empty row] +26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO Executor: Finished task 43.0 in stage 11.0 (TID 69). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO TaskSetManager: Starting task 45.0 in stage 11.0 (TID 71) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:17 INFO Executor: Running task 45.0 in stage 11.0 (TID 71) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 43.0 in stage 11.0 (TID 69) in 196 ms on 10.0.0.133 (executor driver) (42/64) +26/04/01 05:56:17 INFO Executor: Finished task 42.0 in stage 11.0 (TID 68). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO TaskSetManager: Starting task 46.0 in stage 11.0 (TID 72) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 42.0 in stage 11.0 (TID 68) in 198 ms on 10.0.0.133 (executor driver) (43/64) +26/04/01 05:56:17 INFO Executor: Running task 46.0 in stage 11.0 (TID 72) +26/04/01 05:56:17 INFO Executor: Finished task 41.0 in stage 11.0 (TID 67). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO TaskSetManager: Starting task 47.0 in stage 11.0 (TID 73) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 41.0 in stage 11.0 (TID 67) in 200 ms on 10.0.0.133 (executor driver) (44/64) +26/04/01 05:56:17 INFO Executor: Running task 47.0 in stage 11.0 (TID 73) +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.21.parquet, range: 134217728-233548593, partition values: [empty row] +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.28.parquet, range: 134217728-233545450, partition values: [empty row] +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.4.parquet, range: 134217728-233545407, partition values: [empty row] +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO Executor: Finished task 44.0 in stage 11.0 (TID 70). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO TaskSetManager: Starting task 48.0 in stage 11.0 (TID 74) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 44.0 in stage 11.0 (TID 70) in 177 ms on 10.0.0.133 (executor driver) (45/64) +26/04/01 05:56:17 INFO Executor: Running task 48.0 in stage 11.0 (TID 74) +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.29.parquet, range: 134217728-233542751, partition values: [empty row] +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO Executor: Finished task 46.0 in stage 11.0 (TID 72). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO TaskSetManager: Starting task 49.0 in stage 11.0 (TID 75) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 46.0 in stage 11.0 (TID 72) in 202 ms on 10.0.0.133 (executor driver) (46/64) +26/04/01 05:56:17 INFO Executor: Running task 49.0 in stage 11.0 (TID 75) +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.24.parquet, range: 134217728-233541467, partition values: [empty row] +26/04/01 05:56:17 INFO Executor: Finished task 45.0 in stage 11.0 (TID 71). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO TaskSetManager: Starting task 50.0 in stage 11.0 (TID 76) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:17 INFO Executor: Running task 50.0 in stage 11.0 (TID 76) +26/04/01 05:56:17 INFO Executor: Finished task 48.0 in stage 11.0 (TID 74). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO TaskSetManager: Finished task 45.0 in stage 11.0 (TID 71) in 206 ms on 10.0.0.133 (executor driver) (47/64) +26/04/01 05:56:17 INFO TaskSetManager: Starting task 51.0 in stage 11.0 (TID 77) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 48.0 in stage 11.0 (TID 74) in 187 ms on 10.0.0.133 (executor driver) (48/64) +26/04/01 05:56:17 INFO Executor: Running task 51.0 in stage 11.0 (TID 77) +26/04/01 05:56:17 INFO Executor: Finished task 47.0 in stage 11.0 (TID 73). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO TaskSetManager: Starting task 52.0 in stage 11.0 (TID 78) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 47.0 in stage 11.0 (TID 73) in 206 ms on 10.0.0.133 (executor driver) (49/64) +26/04/01 05:56:17 INFO Executor: Running task 52.0 in stage 11.0 (TID 78) +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.20.parquet, range: 134217728-233539473, partition values: [empty row] +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.26.parquet, range: 134217728-233538804, partition values: [empty row] +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.10.parquet, range: 134217728-233538598, partition values: [empty row] +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO Executor: Finished task 50.0 in stage 11.0 (TID 76). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO Executor: Finished task 51.0 in stage 11.0 (TID 77). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO TaskSetManager: Starting task 53.0 in stage 11.0 (TID 79) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:17 INFO Executor: Running task 53.0 in stage 11.0 (TID 79) +26/04/01 05:56:17 INFO TaskSetManager: Starting task 54.0 in stage 11.0 (TID 80) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:17 INFO Executor: Finished task 49.0 in stage 11.0 (TID 75). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO Executor: Running task 54.0 in stage 11.0 (TID 80) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 50.0 in stage 11.0 (TID 76) in 225 ms on 10.0.0.133 (executor driver) (50/64) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 51.0 in stage 11.0 (TID 77) in 226 ms on 10.0.0.133 (executor driver) (51/64) +26/04/01 05:56:17 INFO TaskSetManager: Starting task 55.0 in stage 11.0 (TID 81) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 49.0 in stage 11.0 (TID 75) in 229 ms on 10.0.0.133 (executor driver) (52/64) +26/04/01 05:56:17 INFO Executor: Running task 55.0 in stage 11.0 (TID 81) +26/04/01 05:56:17 INFO Executor: Finished task 52.0 in stage 11.0 (TID 78). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO TaskSetManager: Starting task 56.0 in stage 11.0 (TID 82) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.13.parquet, range: 134217728-233537986, partition values: [empty row] +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.22.parquet, range: 134217728-233536356, partition values: [empty row] +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.8.parquet, range: 134217728-233536463, partition values: [empty row] +26/04/01 05:56:17 INFO TaskSetManager: Finished task 52.0 in stage 11.0 (TID 78) in 226 ms on 10.0.0.133 (executor driver) (53/64) +26/04/01 05:56:17 INFO Executor: Running task 56.0 in stage 11.0 (TID 82) +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.9.parquet, range: 134217728-233535871, partition values: [empty row] +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO Executor: Finished task 55.0 in stage 11.0 (TID 81). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO Executor: Finished task 54.0 in stage 11.0 (TID 80). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO TaskSetManager: Starting task 57.0 in stage 11.0 (TID 83) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:17 INFO Executor: Running task 57.0 in stage 11.0 (TID 83) +26/04/01 05:56:17 INFO TaskSetManager: Starting task 58.0 in stage 11.0 (TID 84) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 55.0 in stage 11.0 (TID 81) in 204 ms on 10.0.0.133 (executor driver) (54/64) +26/04/01 05:56:17 INFO Executor: Running task 58.0 in stage 11.0 (TID 84) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 54.0 in stage 11.0 (TID 80) in 205 ms on 10.0.0.133 (executor driver) (55/64) +26/04/01 05:56:17 INFO Executor: Finished task 53.0 in stage 11.0 (TID 79). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO Executor: Finished task 56.0 in stage 11.0 (TID 82). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO TaskSetManager: Starting task 59.0 in stage 11.0 (TID 85) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 53.0 in stage 11.0 (TID 79) in 206 ms on 10.0.0.133 (executor driver) (56/64) +26/04/01 05:56:17 INFO Executor: Running task 59.0 in stage 11.0 (TID 85) +26/04/01 05:56:17 INFO TaskSetManager: Starting task 60.0 in stage 11.0 (TID 86) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:17 INFO Executor: Running task 60.0 in stage 11.0 (TID 86) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 56.0 in stage 11.0 (TID 82) in 204 ms on 10.0.0.133 (executor driver) (57/64) +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.17.parquet, range: 134217728-233534488, partition values: [empty row] +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.31.parquet, range: 134217728-233535690, partition values: [empty row] +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.18.parquet, range: 134217728-233534883, partition values: [empty row] +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.15.parquet, range: 134217728-233533044, partition values: [empty row] +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO Executor: Finished task 57.0 in stage 11.0 (TID 83). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO TaskSetManager: Starting task 61.0 in stage 11.0 (TID 87) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9598 bytes) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 57.0 in stage 11.0 (TID 83) in 206 ms on 10.0.0.133 (executor driver) (58/64) +26/04/01 05:56:17 INFO Executor: Running task 61.0 in stage 11.0 (TID 87) +26/04/01 05:56:17 INFO Executor: Finished task 60.0 in stage 11.0 (TID 86). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO TaskSetManager: Starting task 62.0 in stage 11.0 (TID 88) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 60.0 in stage 11.0 (TID 86) in 206 ms on 10.0.0.133 (executor driver) (59/64) +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.25.parquet, range: 134217728-233532173, partition values: [empty row] +26/04/01 05:56:17 INFO Executor: Finished task 58.0 in stage 11.0 (TID 84). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO Executor: Running task 62.0 in stage 11.0 (TID 88) +26/04/01 05:56:17 INFO TaskSetManager: Starting task 63.0 in stage 11.0 (TID 89) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9597 bytes) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 58.0 in stage 11.0 (TID 84) in 208 ms on 10.0.0.133 (executor driver) (60/64) +26/04/01 05:56:17 INFO Executor: Running task 63.0 in stage 11.0 (TID 89) +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.3.parquet, range: 134217728-233524702, partition values: [empty row] +26/04/01 05:56:17 INFO Executor: Finished task 59.0 in stage 11.0 (TID 85). 2251 bytes result sent to driver +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.2.parquet, range: 134217728-233511585, partition values: [empty row] +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 90) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:17 INFO TaskSetManager: Finished task 59.0 in stage 11.0 (TID 85) in 209 ms on 10.0.0.133 (executor driver) (61/64) +26/04/01 05:56:17 INFO Executor: Running task 0.0 in stage 12.0 (TID 90) +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) +26/04/01 05:56:17 INFO CodeGenerator: Code generated in 4.42975 ms +26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:18 INFO Executor: Finished task 61.0 in stage 11.0 (TID 87). 2251 bytes result sent to driver +26/04/01 05:56:18 INFO Executor: Finished task 62.0 in stage 11.0 (TID 88). 2251 bytes result sent to driver +26/04/01 05:56:18 INFO TaskSetManager: Starting task 1.0 in stage 12.0 (TID 91) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:18 INFO Executor: Running task 1.0 in stage 12.0 (TID 91) +26/04/01 05:56:18 INFO TaskSetManager: Starting task 2.0 in stage 12.0 (TID 92) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:18 INFO TaskSetManager: Finished task 61.0 in stage 11.0 (TID 87) in 211 ms on 10.0.0.133 (executor driver) (62/64) +26/04/01 05:56:18 INFO Executor: Running task 2.0 in stage 12.0 (TID 92) +26/04/01 05:56:18 INFO TaskSetManager: Finished task 62.0 in stage 11.0 (TID 88) in 209 ms on 10.0.0.133 (executor driver) (63/64) +26/04/01 05:56:18 INFO Executor: Finished task 63.0 in stage 11.0 (TID 89). 2251 bytes result sent to driver +26/04/01 05:56:18 INFO TaskSetManager: Starting task 3.0 in stage 12.0 (TID 93) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:18 INFO Executor: Running task 3.0 in stage 12.0 (TID 93) +26/04/01 05:56:18 INFO TaskSetManager: Finished task 63.0 in stage 11.0 (TID 89) in 210 ms on 10.0.0.133 (executor driver) (64/64) +26/04/01 05:56:18 INFO TaskSchedulerImpl: Removed TaskSet 11.0, whose tasks have all completed, from pool +26/04/01 05:56:18 INFO DAGScheduler: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 5.209 s +26/04/01 05:56:18 INFO DAGScheduler: looking for newly runnable stages +26/04/01 05:56:18 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 12, ShuffleMapStage 13) +26/04/01 05:56:18 INFO DAGScheduler: waiting: Set() +26/04/01 05:56:18 INFO DAGScheduler: failed: Set() +26/04/01 05:56:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:18 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:18 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:18 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:18 INFO ShufflePartitionsUtil: For shuffle(0, 2), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 05:56:18 INFO CodeGenerator: Code generated in 9.73425 ms +26/04/01 05:56:18 INFO CodeGenerator: Code generated in 4.046291 ms +26/04/01 05:56:18 INFO CodeGenerator: Code generated in 7.183792 ms +26/04/01 05:56:18 INFO DAGScheduler: Registering RDD 50 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 5 +26/04/01 05:56:18 INFO DAGScheduler: Got map stage job 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 7 output partitions +26/04/01 05:56:18 INFO DAGScheduler: Final stage: ShuffleMapStage 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:56:18 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 16, ShuffleMapStage 17) +26/04/01 05:56:18 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:18 INFO DAGScheduler: Submitting ShuffleMapStage 18 (MapPartitionsRDD[50] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:56:18 INFO MemoryStore: Block broadcast_22 stored as values in memory (estimated size 47.7 KiB, free 8.6 GiB) +26/04/01 05:56:18 INFO MemoryStore: Block broadcast_22_piece0 stored as bytes in memory (estimated size 22.2 KiB, free 8.6 GiB) +26/04/01 05:56:18 INFO BlockManagerInfo: Added broadcast_22_piece0 in memory on 10.0.0.133:58206 (size: 22.2 KiB, free: 8.6 GiB) +26/04/01 05:56:18 INFO SparkContext: Created broadcast 22 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:18 INFO DAGScheduler: Submitting 7 missing tasks from ShuffleMapStage 18 (MapPartitionsRDD[50] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6)) +26/04/01 05:56:18 INFO TaskSchedulerImpl: Adding task set 18.0 with 7 tasks resource profile 0 +26/04/01 05:56:18 INFO Executor: Finished task 0.0 in stage 12.0 (TID 90). 2251 bytes result sent to driver +26/04/01 05:56:18 INFO TaskSetManager: Starting task 4.0 in stage 12.0 (TID 94) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:18 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 90) in 774 ms on 10.0.0.133 (executor driver) (1/208) +26/04/01 05:56:18 INFO Executor: Running task 4.0 in stage 12.0 (TID 94) +26/04/01 05:56:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:18 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:18 INFO Executor: Finished task 3.0 in stage 12.0 (TID 93). 2251 bytes result sent to driver +26/04/01 05:56:18 INFO TaskSetManager: Starting task 5.0 in stage 12.0 (TID 95) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:18 INFO TaskSetManager: Finished task 3.0 in stage 12.0 (TID 93) in 805 ms on 10.0.0.133 (executor driver) (2/208) +26/04/01 05:56:18 INFO Executor: Running task 5.0 in stage 12.0 (TID 95) +26/04/01 05:56:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:18 INFO Executor: Finished task 2.0 in stage 12.0 (TID 92). 2251 bytes result sent to driver +26/04/01 05:56:18 INFO TaskSetManager: Starting task 6.0 in stage 12.0 (TID 96) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:18 INFO TaskSetManager: Finished task 2.0 in stage 12.0 (TID 92) in 809 ms on 10.0.0.133 (executor driver) (3/208) +26/04/01 05:56:18 INFO Executor: Running task 6.0 in stage 12.0 (TID 96) +26/04/01 05:56:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:18 INFO Executor: Finished task 1.0 in stage 12.0 (TID 91). 2251 bytes result sent to driver +26/04/01 05:56:18 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:18 INFO TaskSetManager: Starting task 7.0 in stage 12.0 (TID 97) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:18 INFO TaskSetManager: Finished task 1.0 in stage 12.0 (TID 91) in 811 ms on 10.0.0.133 (executor driver) (4/208) +26/04/01 05:56:18 INFO Executor: Running task 7.0 in stage 12.0 (TID 97) +26/04/01 05:56:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:18 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:18 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:19 INFO Executor: Finished task 4.0 in stage 12.0 (TID 94). 2251 bytes result sent to driver +26/04/01 05:56:19 INFO TaskSetManager: Starting task 8.0 in stage 12.0 (TID 98) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:19 INFO TaskSetManager: Finished task 4.0 in stage 12.0 (TID 94) in 821 ms on 10.0.0.133 (executor driver) (5/208) +26/04/01 05:56:19 INFO Executor: Running task 8.0 in stage 12.0 (TID 98) +26/04/01 05:56:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:19 INFO BlockManagerInfo: Removed broadcast_11_piece0 on 10.0.0.133:58206 in memory (size: 7.9 KiB, free: 8.6 GiB) +26/04/01 05:56:19 INFO BlockManagerInfo: Removed broadcast_16_piece0 on 10.0.0.133:58206 in memory (size: 8.3 KiB, free: 8.6 GiB) +26/04/01 05:56:19 INFO BlockManagerInfo: Removed broadcast_13_piece0 on 10.0.0.133:58206 in memory (size: 14.0 KiB, free: 8.6 GiB) +26/04/01 05:56:19 INFO BlockManagerInfo: Removed broadcast_14_piece0 on 10.0.0.133:58206 in memory (size: 6.7 KiB, free: 8.6 GiB) +26/04/01 05:56:19 INFO Executor: Finished task 6.0 in stage 12.0 (TID 96). 2251 bytes result sent to driver +26/04/01 05:56:19 INFO Executor: Finished task 7.0 in stage 12.0 (TID 97). 2251 bytes result sent to driver +26/04/01 05:56:19 INFO TaskSetManager: Starting task 9.0 in stage 12.0 (TID 99) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:19 INFO TaskSetManager: Finished task 6.0 in stage 12.0 (TID 96) in 908 ms on 10.0.0.133 (executor driver) (6/208) +26/04/01 05:56:19 INFO Executor: Running task 9.0 in stage 12.0 (TID 99) +26/04/01 05:56:19 INFO TaskSetManager: Starting task 10.0 in stage 12.0 (TID 100) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:19 INFO TaskSetManager: Finished task 7.0 in stage 12.0 (TID 97) in 906 ms on 10.0.0.133 (executor driver) (7/208) +26/04/01 05:56:19 INFO Executor: Running task 10.0 in stage 12.0 (TID 100) +26/04/01 05:56:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:19 INFO Executor: Finished task 5.0 in stage 12.0 (TID 95). 2251 bytes result sent to driver +26/04/01 05:56:19 INFO TaskSetManager: Starting task 11.0 in stage 12.0 (TID 101) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:19 INFO TaskSetManager: Finished task 5.0 in stage 12.0 (TID 95) in 934 ms on 10.0.0.133 (executor driver) (8/208) +26/04/01 05:56:19 INFO Executor: Running task 11.0 in stage 12.0 (TID 101) +26/04/01 05:56:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:20 INFO Executor: Finished task 8.0 in stage 12.0 (TID 98). 2251 bytes result sent to driver +26/04/01 05:56:20 INFO TaskSetManager: Starting task 12.0 in stage 12.0 (TID 102) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:20 INFO TaskSetManager: Finished task 8.0 in stage 12.0 (TID 98) in 848 ms on 10.0.0.133 (executor driver) (9/208) +26/04/01 05:56:20 INFO Executor: Running task 12.0 in stage 12.0 (TID 102) +26/04/01 05:56:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:20 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:20 INFO Executor: Finished task 11.0 in stage 12.0 (TID 101). 2251 bytes result sent to driver +26/04/01 05:56:20 INFO TaskSetManager: Starting task 13.0 in stage 12.0 (TID 103) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:20 INFO TaskSetManager: Finished task 11.0 in stage 12.0 (TID 101) in 763 ms on 10.0.0.133 (executor driver) (10/208) +26/04/01 05:56:20 INFO Executor: Running task 13.0 in stage 12.0 (TID 103) +26/04/01 05:56:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:20 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:20 INFO Executor: Finished task 9.0 in stage 12.0 (TID 99). 2251 bytes result sent to driver +26/04/01 05:56:20 INFO TaskSetManager: Starting task 14.0 in stage 12.0 (TID 104) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:20 INFO TaskSetManager: Finished task 9.0 in stage 12.0 (TID 99) in 794 ms on 10.0.0.133 (executor driver) (11/208) +26/04/01 05:56:20 INFO Executor: Running task 14.0 in stage 12.0 (TID 104) +26/04/01 05:56:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:20 INFO Executor: Finished task 10.0 in stage 12.0 (TID 100). 2251 bytes result sent to driver +26/04/01 05:56:20 INFO TaskSetManager: Starting task 15.0 in stage 12.0 (TID 105) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:20 INFO Executor: Running task 15.0 in stage 12.0 (TID 105) +26/04/01 05:56:20 INFO TaskSetManager: Finished task 10.0 in stage 12.0 (TID 100) in 796 ms on 10.0.0.133 (executor driver) (12/208) +26/04/01 05:56:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:20 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:20 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:21 INFO Executor: Finished task 12.0 in stage 12.0 (TID 102). 2294 bytes result sent to driver +26/04/01 05:56:21 INFO TaskSetManager: Starting task 16.0 in stage 12.0 (TID 106) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:21 INFO TaskSetManager: Finished task 12.0 in stage 12.0 (TID 102) in 748 ms on 10.0.0.133 (executor driver) (13/208) +26/04/01 05:56:21 INFO Executor: Running task 16.0 in stage 12.0 (TID 106) +26/04/01 05:56:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:21 INFO Executor: Finished task 15.0 in stage 12.0 (TID 105). 2251 bytes result sent to driver +26/04/01 05:56:21 INFO Executor: Finished task 14.0 in stage 12.0 (TID 104). 2251 bytes result sent to driver +26/04/01 05:56:21 INFO TaskSetManager: Starting task 17.0 in stage 12.0 (TID 107) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:21 INFO Executor: Running task 17.0 in stage 12.0 (TID 107) +26/04/01 05:56:21 INFO TaskSetManager: Starting task 18.0 in stage 12.0 (TID 108) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:21 INFO TaskSetManager: Finished task 15.0 in stage 12.0 (TID 105) in 887 ms on 10.0.0.133 (executor driver) (14/208) +26/04/01 05:56:21 INFO Executor: Running task 18.0 in stage 12.0 (TID 108) +26/04/01 05:56:21 INFO TaskSetManager: Finished task 14.0 in stage 12.0 (TID 104) in 889 ms on 10.0.0.133 (executor driver) (15/208) +26/04/01 05:56:21 INFO Executor: Finished task 13.0 in stage 12.0 (TID 103). 2251 bytes result sent to driver +26/04/01 05:56:21 INFO TaskSetManager: Starting task 19.0 in stage 12.0 (TID 109) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:21 INFO TaskSetManager: Finished task 13.0 in stage 12.0 (TID 103) in 899 ms on 10.0.0.133 (executor driver) (16/208) +26/04/01 05:56:21 INFO Executor: Running task 19.0 in stage 12.0 (TID 109) +26/04/01 05:56:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:21 INFO Executor: Finished task 16.0 in stage 12.0 (TID 106). 2251 bytes result sent to driver +26/04/01 05:56:21 INFO TaskSetManager: Starting task 20.0 in stage 12.0 (TID 110) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:21 INFO TaskSetManager: Finished task 16.0 in stage 12.0 (TID 106) in 760 ms on 10.0.0.133 (executor driver) (17/208) +26/04/01 05:56:21 INFO Executor: Running task 20.0 in stage 12.0 (TID 110) +26/04/01 05:56:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:22 INFO Executor: Finished task 17.0 in stage 12.0 (TID 107). 2251 bytes result sent to driver +26/04/01 05:56:22 INFO TaskSetManager: Starting task 21.0 in stage 12.0 (TID 111) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:22 INFO TaskSetManager: Finished task 17.0 in stage 12.0 (TID 107) in 771 ms on 10.0.0.133 (executor driver) (18/208) +26/04/01 05:56:22 INFO Executor: Running task 21.0 in stage 12.0 (TID 111) +26/04/01 05:56:22 INFO Executor: Finished task 18.0 in stage 12.0 (TID 108). 2251 bytes result sent to driver +26/04/01 05:56:22 INFO TaskSetManager: Starting task 22.0 in stage 12.0 (TID 112) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:22 INFO TaskSetManager: Finished task 18.0 in stage 12.0 (TID 108) in 771 ms on 10.0.0.133 (executor driver) (19/208) +26/04/01 05:56:22 INFO Executor: Running task 22.0 in stage 12.0 (TID 112) +26/04/01 05:56:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:22 INFO Executor: Finished task 19.0 in stage 12.0 (TID 109). 2251 bytes result sent to driver +26/04/01 05:56:22 INFO TaskSetManager: Starting task 23.0 in stage 12.0 (TID 113) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:22 INFO TaskSetManager: Finished task 19.0 in stage 12.0 (TID 109) in 780 ms on 10.0.0.133 (executor driver) (20/208) +26/04/01 05:56:22 INFO Executor: Running task 23.0 in stage 12.0 (TID 113) +26/04/01 05:56:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:22 INFO Executor: Finished task 20.0 in stage 12.0 (TID 110). 2251 bytes result sent to driver +26/04/01 05:56:22 INFO TaskSetManager: Starting task 24.0 in stage 12.0 (TID 114) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:22 INFO TaskSetManager: Finished task 20.0 in stage 12.0 (TID 110) in 754 ms on 10.0.0.133 (executor driver) (21/208) +26/04/01 05:56:22 INFO Executor: Running task 24.0 in stage 12.0 (TID 114) +26/04/01 05:56:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:23 INFO Executor: Finished task 22.0 in stage 12.0 (TID 112). 2251 bytes result sent to driver +26/04/01 05:56:23 INFO TaskSetManager: Starting task 25.0 in stage 12.0 (TID 115) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:23 INFO TaskSetManager: Finished task 22.0 in stage 12.0 (TID 112) in 754 ms on 10.0.0.133 (executor driver) (22/208) +26/04/01 05:56:23 INFO Executor: Running task 25.0 in stage 12.0 (TID 115) +26/04/01 05:56:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:23 INFO Executor: Finished task 21.0 in stage 12.0 (TID 111). 2251 bytes result sent to driver +26/04/01 05:56:23 INFO TaskSetManager: Starting task 26.0 in stage 12.0 (TID 116) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:23 INFO TaskSetManager: Finished task 21.0 in stage 12.0 (TID 111) in 757 ms on 10.0.0.133 (executor driver) (23/208) +26/04/01 05:56:23 INFO Executor: Running task 26.0 in stage 12.0 (TID 116) +26/04/01 05:56:23 INFO Executor: Finished task 23.0 in stage 12.0 (TID 113). 2251 bytes result sent to driver +26/04/01 05:56:23 INFO TaskSetManager: Starting task 27.0 in stage 12.0 (TID 117) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:23 INFO TaskSetManager: Finished task 23.0 in stage 12.0 (TID 113) in 747 ms on 10.0.0.133 (executor driver) (24/208) +26/04/01 05:56:23 INFO Executor: Running task 27.0 in stage 12.0 (TID 117) +26/04/01 05:56:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:23 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:23 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:23 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:23 INFO Executor: Finished task 24.0 in stage 12.0 (TID 114). 2251 bytes result sent to driver +26/04/01 05:56:23 INFO TaskSetManager: Starting task 28.0 in stage 12.0 (TID 118) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:23 INFO TaskSetManager: Finished task 24.0 in stage 12.0 (TID 114) in 735 ms on 10.0.0.133 (executor driver) (25/208) +26/04/01 05:56:23 INFO Executor: Running task 28.0 in stage 12.0 (TID 118) +26/04/01 05:56:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:23 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:23 INFO Executor: Finished task 26.0 in stage 12.0 (TID 116). 2251 bytes result sent to driver +26/04/01 05:56:23 INFO TaskSetManager: Starting task 29.0 in stage 12.0 (TID 119) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:23 INFO TaskSetManager: Finished task 26.0 in stage 12.0 (TID 116) in 789 ms on 10.0.0.133 (executor driver) (26/208) +26/04/01 05:56:23 INFO Executor: Running task 29.0 in stage 12.0 (TID 119) +26/04/01 05:56:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:23 INFO Executor: Finished task 27.0 in stage 12.0 (TID 117). 2251 bytes result sent to driver +26/04/01 05:56:23 INFO TaskSetManager: Starting task 30.0 in stage 12.0 (TID 120) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:23 INFO TaskSetManager: Finished task 27.0 in stage 12.0 (TID 117) in 791 ms on 10.0.0.133 (executor driver) (27/208) +26/04/01 05:56:23 INFO Executor: Running task 30.0 in stage 12.0 (TID 120) +26/04/01 05:56:23 INFO Executor: Finished task 25.0 in stage 12.0 (TID 115). 2251 bytes result sent to driver +26/04/01 05:56:23 INFO TaskSetManager: Starting task 31.0 in stage 12.0 (TID 121) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:23 INFO TaskSetManager: Finished task 25.0 in stage 12.0 (TID 115) in 795 ms on 10.0.0.133 (executor driver) (28/208) +26/04/01 05:56:23 INFO Executor: Running task 31.0 in stage 12.0 (TID 121) +26/04/01 05:56:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:23 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:23 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:23 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:24 INFO Executor: Finished task 28.0 in stage 12.0 (TID 118). 2251 bytes result sent to driver +26/04/01 05:56:24 INFO TaskSetManager: Starting task 32.0 in stage 12.0 (TID 122) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:24 INFO Executor: Running task 32.0 in stage 12.0 (TID 122) +26/04/01 05:56:24 INFO TaskSetManager: Finished task 28.0 in stage 12.0 (TID 118) in 767 ms on 10.0.0.133 (executor driver) (29/208) +26/04/01 05:56:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:24 INFO Executor: Finished task 31.0 in stage 12.0 (TID 121). 2251 bytes result sent to driver +26/04/01 05:56:24 INFO TaskSetManager: Starting task 33.0 in stage 12.0 (TID 123) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:24 INFO TaskSetManager: Finished task 31.0 in stage 12.0 (TID 121) in 743 ms on 10.0.0.133 (executor driver) (30/208) +26/04/01 05:56:24 INFO Executor: Running task 33.0 in stage 12.0 (TID 123) +26/04/01 05:56:24 INFO Executor: Finished task 30.0 in stage 12.0 (TID 120). 2251 bytes result sent to driver +26/04/01 05:56:24 INFO TaskSetManager: Starting task 34.0 in stage 12.0 (TID 124) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:24 INFO TaskSetManager: Finished task 30.0 in stage 12.0 (TID 120) in 744 ms on 10.0.0.133 (executor driver) (31/208) +26/04/01 05:56:24 INFO Executor: Running task 34.0 in stage 12.0 (TID 124) +26/04/01 05:56:24 INFO Executor: Finished task 29.0 in stage 12.0 (TID 119). 2251 bytes result sent to driver +26/04/01 05:56:24 INFO TaskSetManager: Starting task 35.0 in stage 12.0 (TID 125) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:24 INFO TaskSetManager: Finished task 29.0 in stage 12.0 (TID 119) in 747 ms on 10.0.0.133 (executor driver) (32/208) +26/04/01 05:56:24 INFO Executor: Running task 35.0 in stage 12.0 (TID 125) +26/04/01 05:56:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:24 INFO Executor: Finished task 32.0 in stage 12.0 (TID 122). 2251 bytes result sent to driver +26/04/01 05:56:24 INFO TaskSetManager: Starting task 36.0 in stage 12.0 (TID 126) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:24 INFO TaskSetManager: Finished task 32.0 in stage 12.0 (TID 122) in 719 ms on 10.0.0.133 (executor driver) (33/208) +26/04/01 05:56:24 INFO Executor: Running task 36.0 in stage 12.0 (TID 126) +26/04/01 05:56:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:25 INFO Executor: Finished task 35.0 in stage 12.0 (TID 125). 2251 bytes result sent to driver +26/04/01 05:56:25 INFO TaskSetManager: Starting task 37.0 in stage 12.0 (TID 127) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:25 INFO Executor: Finished task 34.0 in stage 12.0 (TID 124). 2251 bytes result sent to driver +26/04/01 05:56:25 INFO TaskSetManager: Finished task 35.0 in stage 12.0 (TID 125) in 742 ms on 10.0.0.133 (executor driver) (34/208) +26/04/01 05:56:25 INFO Executor: Running task 37.0 in stage 12.0 (TID 127) +26/04/01 05:56:25 INFO TaskSetManager: Starting task 38.0 in stage 12.0 (TID 128) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:25 INFO TaskSetManager: Finished task 34.0 in stage 12.0 (TID 124) in 743 ms on 10.0.0.133 (executor driver) (35/208) +26/04/01 05:56:25 INFO Executor: Running task 38.0 in stage 12.0 (TID 128) +26/04/01 05:56:25 INFO Executor: Finished task 33.0 in stage 12.0 (TID 123). 2251 bytes result sent to driver +26/04/01 05:56:25 INFO TaskSetManager: Starting task 39.0 in stage 12.0 (TID 129) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:25 INFO Executor: Running task 39.0 in stage 12.0 (TID 129) +26/04/01 05:56:25 INFO TaskSetManager: Finished task 33.0 in stage 12.0 (TID 123) in 745 ms on 10.0.0.133 (executor driver) (36/208) +26/04/01 05:56:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:25 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:25 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:25 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:25 INFO Executor: Finished task 36.0 in stage 12.0 (TID 126). 2251 bytes result sent to driver +26/04/01 05:56:25 INFO TaskSetManager: Starting task 40.0 in stage 12.0 (TID 130) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:25 INFO Executor: Running task 40.0 in stage 12.0 (TID 130) +26/04/01 05:56:25 INFO TaskSetManager: Finished task 36.0 in stage 12.0 (TID 126) in 757 ms on 10.0.0.133 (executor driver) (37/208) +26/04/01 05:56:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:25 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:26 INFO Executor: Finished task 37.0 in stage 12.0 (TID 127). 2251 bytes result sent to driver +26/04/01 05:56:26 INFO TaskSetManager: Starting task 41.0 in stage 12.0 (TID 131) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:26 INFO Executor: Running task 41.0 in stage 12.0 (TID 131) +26/04/01 05:56:26 INFO TaskSetManager: Finished task 37.0 in stage 12.0 (TID 127) in 769 ms on 10.0.0.133 (executor driver) (38/208) +26/04/01 05:56:26 INFO Executor: Finished task 39.0 in stage 12.0 (TID 129). 2251 bytes result sent to driver +26/04/01 05:56:26 INFO TaskSetManager: Starting task 42.0 in stage 12.0 (TID 132) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:26 INFO Executor: Running task 42.0 in stage 12.0 (TID 132) +26/04/01 05:56:26 INFO TaskSetManager: Finished task 39.0 in stage 12.0 (TID 129) in 769 ms on 10.0.0.133 (executor driver) (39/208) +26/04/01 05:56:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:26 INFO Executor: Finished task 38.0 in stage 12.0 (TID 128). 2251 bytes result sent to driver +26/04/01 05:56:26 INFO TaskSetManager: Starting task 43.0 in stage 12.0 (TID 133) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:26 INFO TaskSetManager: Finished task 38.0 in stage 12.0 (TID 128) in 771 ms on 10.0.0.133 (executor driver) (40/208) +26/04/01 05:56:26 INFO Executor: Running task 43.0 in stage 12.0 (TID 133) +26/04/01 05:56:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:26 INFO Executor: Finished task 40.0 in stage 12.0 (TID 130). 2251 bytes result sent to driver +26/04/01 05:56:26 INFO TaskSetManager: Starting task 44.0 in stage 12.0 (TID 134) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:26 INFO TaskSetManager: Finished task 40.0 in stage 12.0 (TID 130) in 703 ms on 10.0.0.133 (executor driver) (41/208) +26/04/01 05:56:26 INFO Executor: Running task 44.0 in stage 12.0 (TID 134) +26/04/01 05:56:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:26 INFO Executor: Finished task 42.0 in stage 12.0 (TID 132). 2251 bytes result sent to driver +26/04/01 05:56:26 INFO TaskSetManager: Starting task 45.0 in stage 12.0 (TID 135) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:26 INFO TaskSetManager: Finished task 42.0 in stage 12.0 (TID 132) in 747 ms on 10.0.0.133 (executor driver) (42/208) +26/04/01 05:56:26 INFO Executor: Running task 45.0 in stage 12.0 (TID 135) +26/04/01 05:56:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:26 INFO Executor: Finished task 43.0 in stage 12.0 (TID 133). 2251 bytes result sent to driver +26/04/01 05:56:26 INFO TaskSetManager: Starting task 46.0 in stage 12.0 (TID 136) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:26 INFO TaskSetManager: Finished task 43.0 in stage 12.0 (TID 133) in 776 ms on 10.0.0.133 (executor driver) (43/208) +26/04/01 05:56:26 INFO Executor: Running task 46.0 in stage 12.0 (TID 136) +26/04/01 05:56:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:26 INFO Executor: Finished task 41.0 in stage 12.0 (TID 131). 2251 bytes result sent to driver +26/04/01 05:56:26 INFO TaskSetManager: Starting task 47.0 in stage 12.0 (TID 137) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:26 INFO TaskSetManager: Finished task 41.0 in stage 12.0 (TID 131) in 789 ms on 10.0.0.133 (executor driver) (44/208) +26/04/01 05:56:26 INFO Executor: Running task 47.0 in stage 12.0 (TID 137) +26/04/01 05:56:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:27 INFO Executor: Finished task 44.0 in stage 12.0 (TID 134). 2251 bytes result sent to driver +26/04/01 05:56:27 INFO TaskSetManager: Starting task 48.0 in stage 12.0 (TID 138) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:27 INFO Executor: Running task 48.0 in stage 12.0 (TID 138) +26/04/01 05:56:27 INFO TaskSetManager: Finished task 44.0 in stage 12.0 (TID 134) in 737 ms on 10.0.0.133 (executor driver) (45/208) +26/04/01 05:56:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:27 INFO Executor: Finished task 45.0 in stage 12.0 (TID 135). 2251 bytes result sent to driver +26/04/01 05:56:27 INFO TaskSetManager: Starting task 49.0 in stage 12.0 (TID 139) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:27 INFO TaskSetManager: Finished task 45.0 in stage 12.0 (TID 135) in 726 ms on 10.0.0.133 (executor driver) (46/208) +26/04/01 05:56:27 INFO Executor: Running task 49.0 in stage 12.0 (TID 139) +26/04/01 05:56:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:27 INFO Executor: Finished task 46.0 in stage 12.0 (TID 136). 2251 bytes result sent to driver +26/04/01 05:56:27 INFO TaskSetManager: Starting task 50.0 in stage 12.0 (TID 140) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:27 INFO Executor: Running task 50.0 in stage 12.0 (TID 140) +26/04/01 05:56:27 INFO TaskSetManager: Finished task 46.0 in stage 12.0 (TID 136) in 788 ms on 10.0.0.133 (executor driver) (47/208) +26/04/01 05:56:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:27 INFO Executor: Finished task 47.0 in stage 12.0 (TID 137). 2251 bytes result sent to driver +26/04/01 05:56:27 INFO TaskSetManager: Starting task 51.0 in stage 12.0 (TID 141) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:27 INFO TaskSetManager: Finished task 47.0 in stage 12.0 (TID 137) in 822 ms on 10.0.0.133 (executor driver) (48/208) +26/04/01 05:56:27 INFO Executor: Running task 51.0 in stage 12.0 (TID 141) +26/04/01 05:56:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:27 INFO Executor: Finished task 48.0 in stage 12.0 (TID 138). 2251 bytes result sent to driver +26/04/01 05:56:27 INFO TaskSetManager: Starting task 52.0 in stage 12.0 (TID 142) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:27 INFO TaskSetManager: Finished task 48.0 in stage 12.0 (TID 138) in 760 ms on 10.0.0.133 (executor driver) (49/208) +26/04/01 05:56:27 INFO Executor: Running task 52.0 in stage 12.0 (TID 142) +26/04/01 05:56:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:28 INFO Executor: Finished task 49.0 in stage 12.0 (TID 139). 2251 bytes result sent to driver +26/04/01 05:56:28 INFO TaskSetManager: Starting task 53.0 in stage 12.0 (TID 143) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:28 INFO TaskSetManager: Finished task 49.0 in stage 12.0 (TID 139) in 732 ms on 10.0.0.133 (executor driver) (50/208) +26/04/01 05:56:28 INFO Executor: Running task 53.0 in stage 12.0 (TID 143) +26/04/01 05:56:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:28 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:28 INFO Executor: Finished task 50.0 in stage 12.0 (TID 140). 2251 bytes result sent to driver +26/04/01 05:56:28 INFO TaskSetManager: Starting task 54.0 in stage 12.0 (TID 144) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:28 INFO TaskSetManager: Finished task 50.0 in stage 12.0 (TID 140) in 742 ms on 10.0.0.133 (executor driver) (51/208) +26/04/01 05:56:28 INFO Executor: Running task 54.0 in stage 12.0 (TID 144) +26/04/01 05:56:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:28 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:28 INFO Executor: Finished task 51.0 in stage 12.0 (TID 141). 2251 bytes result sent to driver +26/04/01 05:56:28 INFO TaskSetManager: Starting task 55.0 in stage 12.0 (TID 145) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:28 INFO TaskSetManager: Finished task 51.0 in stage 12.0 (TID 141) in 725 ms on 10.0.0.133 (executor driver) (52/208) +26/04/01 05:56:28 INFO Executor: Running task 55.0 in stage 12.0 (TID 145) +26/04/01 05:56:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:28 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:28 INFO Executor: Finished task 52.0 in stage 12.0 (TID 142). 2251 bytes result sent to driver +26/04/01 05:56:28 INFO TaskSetManager: Starting task 56.0 in stage 12.0 (TID 146) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:28 INFO TaskSetManager: Finished task 52.0 in stage 12.0 (TID 142) in 738 ms on 10.0.0.133 (executor driver) (53/208) +26/04/01 05:56:28 INFO Executor: Running task 56.0 in stage 12.0 (TID 146) +26/04/01 05:56:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:28 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:29 INFO Executor: Finished task 53.0 in stage 12.0 (TID 143). 2251 bytes result sent to driver +26/04/01 05:56:29 INFO TaskSetManager: Starting task 57.0 in stage 12.0 (TID 147) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:29 INFO Executor: Running task 57.0 in stage 12.0 (TID 147) +26/04/01 05:56:29 INFO TaskSetManager: Finished task 53.0 in stage 12.0 (TID 143) in 727 ms on 10.0.0.133 (executor driver) (54/208) +26/04/01 05:56:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:29 INFO Executor: Finished task 54.0 in stage 12.0 (TID 144). 2251 bytes result sent to driver +26/04/01 05:56:29 INFO TaskSetManager: Starting task 58.0 in stage 12.0 (TID 148) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:29 INFO TaskSetManager: Finished task 54.0 in stage 12.0 (TID 144) in 774 ms on 10.0.0.133 (executor driver) (55/208) +26/04/01 05:56:29 INFO Executor: Running task 58.0 in stage 12.0 (TID 148) +26/04/01 05:56:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:29 INFO Executor: Finished task 55.0 in stage 12.0 (TID 145). 2251 bytes result sent to driver +26/04/01 05:56:29 INFO TaskSetManager: Starting task 59.0 in stage 12.0 (TID 149) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:29 INFO TaskSetManager: Finished task 55.0 in stage 12.0 (TID 145) in 756 ms on 10.0.0.133 (executor driver) (56/208) +26/04/01 05:56:29 INFO Executor: Running task 59.0 in stage 12.0 (TID 149) +26/04/01 05:56:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:29 INFO Executor: Finished task 56.0 in stage 12.0 (TID 146). 2251 bytes result sent to driver +26/04/01 05:56:29 INFO TaskSetManager: Starting task 60.0 in stage 12.0 (TID 150) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:29 INFO TaskSetManager: Finished task 56.0 in stage 12.0 (TID 146) in 767 ms on 10.0.0.133 (executor driver) (57/208) +26/04/01 05:56:29 INFO Executor: Running task 60.0 in stage 12.0 (TID 150) +26/04/01 05:56:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:29 INFO Executor: Finished task 57.0 in stage 12.0 (TID 147). 2251 bytes result sent to driver +26/04/01 05:56:29 INFO TaskSetManager: Starting task 61.0 in stage 12.0 (TID 151) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:29 INFO TaskSetManager: Finished task 57.0 in stage 12.0 (TID 147) in 757 ms on 10.0.0.133 (executor driver) (58/208) +26/04/01 05:56:29 INFO Executor: Running task 61.0 in stage 12.0 (TID 151) +26/04/01 05:56:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:29 INFO Executor: Finished task 58.0 in stage 12.0 (TID 148). 2251 bytes result sent to driver +26/04/01 05:56:29 INFO TaskSetManager: Starting task 62.0 in stage 12.0 (TID 152) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:29 INFO TaskSetManager: Finished task 58.0 in stage 12.0 (TID 148) in 761 ms on 10.0.0.133 (executor driver) (59/208) +26/04/01 05:56:29 INFO Executor: Running task 62.0 in stage 12.0 (TID 152) +26/04/01 05:56:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:30 INFO Executor: Finished task 59.0 in stage 12.0 (TID 149). 2251 bytes result sent to driver +26/04/01 05:56:30 INFO TaskSetManager: Starting task 63.0 in stage 12.0 (TID 153) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:30 INFO TaskSetManager: Finished task 59.0 in stage 12.0 (TID 149) in 782 ms on 10.0.0.133 (executor driver) (60/208) +26/04/01 05:56:30 INFO Executor: Running task 63.0 in stage 12.0 (TID 153) +26/04/01 05:56:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:30 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:30 INFO Executor: Finished task 60.0 in stage 12.0 (TID 150). 2251 bytes result sent to driver +26/04/01 05:56:30 INFO TaskSetManager: Starting task 64.0 in stage 12.0 (TID 154) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:30 INFO TaskSetManager: Finished task 60.0 in stage 12.0 (TID 150) in 765 ms on 10.0.0.133 (executor driver) (61/208) +26/04/01 05:56:30 INFO Executor: Running task 64.0 in stage 12.0 (TID 154) +26/04/01 05:56:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:30 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:30 INFO Executor: Finished task 61.0 in stage 12.0 (TID 151). 2251 bytes result sent to driver +26/04/01 05:56:30 INFO TaskSetManager: Starting task 65.0 in stage 12.0 (TID 155) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:30 INFO TaskSetManager: Finished task 61.0 in stage 12.0 (TID 151) in 755 ms on 10.0.0.133 (executor driver) (62/208) +26/04/01 05:56:30 INFO Executor: Running task 65.0 in stage 12.0 (TID 155) +26/04/01 05:56:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:30 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:30 INFO Executor: Finished task 62.0 in stage 12.0 (TID 152). 2251 bytes result sent to driver +26/04/01 05:56:30 INFO TaskSetManager: Starting task 66.0 in stage 12.0 (TID 156) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:30 INFO TaskSetManager: Finished task 62.0 in stage 12.0 (TID 152) in 760 ms on 10.0.0.133 (executor driver) (63/208) +26/04/01 05:56:30 INFO Executor: Running task 66.0 in stage 12.0 (TID 156) +26/04/01 05:56:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:30 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:30 INFO Executor: Finished task 63.0 in stage 12.0 (TID 153). 2251 bytes result sent to driver +26/04/01 05:56:30 INFO TaskSetManager: Starting task 67.0 in stage 12.0 (TID 157) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:30 INFO TaskSetManager: Finished task 63.0 in stage 12.0 (TID 153) in 742 ms on 10.0.0.133 (executor driver) (64/208) +26/04/01 05:56:30 INFO Executor: Running task 67.0 in stage 12.0 (TID 157) +26/04/01 05:56:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:30 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:30 INFO Executor: Finished task 64.0 in stage 12.0 (TID 154). 2251 bytes result sent to driver +26/04/01 05:56:30 INFO TaskSetManager: Starting task 68.0 in stage 12.0 (TID 158) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:30 INFO TaskSetManager: Finished task 64.0 in stage 12.0 (TID 154) in 726 ms on 10.0.0.133 (executor driver) (65/208) +26/04/01 05:56:30 INFO Executor: Running task 68.0 in stage 12.0 (TID 158) +26/04/01 05:56:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:30 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:31 INFO Executor: Finished task 65.0 in stage 12.0 (TID 155). 2251 bytes result sent to driver +26/04/01 05:56:31 INFO TaskSetManager: Starting task 69.0 in stage 12.0 (TID 159) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:31 INFO Executor: Running task 69.0 in stage 12.0 (TID 159) +26/04/01 05:56:31 INFO TaskSetManager: Finished task 65.0 in stage 12.0 (TID 155) in 721 ms on 10.0.0.133 (executor driver) (66/208) +26/04/01 05:56:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:31 INFO Executor: Finished task 66.0 in stage 12.0 (TID 156). 2251 bytes result sent to driver +26/04/01 05:56:31 INFO TaskSetManager: Starting task 70.0 in stage 12.0 (TID 160) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:31 INFO TaskSetManager: Finished task 66.0 in stage 12.0 (TID 156) in 754 ms on 10.0.0.133 (executor driver) (67/208) +26/04/01 05:56:31 INFO Executor: Running task 70.0 in stage 12.0 (TID 160) +26/04/01 05:56:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:31 INFO Executor: Finished task 67.0 in stage 12.0 (TID 157). 2251 bytes result sent to driver +26/04/01 05:56:31 INFO TaskSetManager: Starting task 71.0 in stage 12.0 (TID 161) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:31 INFO TaskSetManager: Finished task 67.0 in stage 12.0 (TID 157) in 772 ms on 10.0.0.133 (executor driver) (68/208) +26/04/01 05:56:31 INFO Executor: Running task 71.0 in stage 12.0 (TID 161) +26/04/01 05:56:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:31 INFO Executor: Finished task 68.0 in stage 12.0 (TID 158). 2251 bytes result sent to driver +26/04/01 05:56:31 INFO TaskSetManager: Starting task 72.0 in stage 12.0 (TID 162) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:31 INFO TaskSetManager: Finished task 68.0 in stage 12.0 (TID 158) in 776 ms on 10.0.0.133 (executor driver) (69/208) +26/04/01 05:56:31 INFO Executor: Running task 72.0 in stage 12.0 (TID 162) +26/04/01 05:56:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:32 INFO Executor: Finished task 69.0 in stage 12.0 (TID 159). 2251 bytes result sent to driver +26/04/01 05:56:32 INFO TaskSetManager: Starting task 73.0 in stage 12.0 (TID 163) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:32 INFO TaskSetManager: Finished task 69.0 in stage 12.0 (TID 159) in 769 ms on 10.0.0.133 (executor driver) (70/208) +26/04/01 05:56:32 INFO Executor: Running task 73.0 in stage 12.0 (TID 163) +26/04/01 05:56:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:32 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:32 INFO Executor: Finished task 71.0 in stage 12.0 (TID 161). 2251 bytes result sent to driver +26/04/01 05:56:32 INFO TaskSetManager: Starting task 74.0 in stage 12.0 (TID 164) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:32 INFO TaskSetManager: Finished task 71.0 in stage 12.0 (TID 161) in 722 ms on 10.0.0.133 (executor driver) (71/208) +26/04/01 05:56:32 INFO Executor: Running task 74.0 in stage 12.0 (TID 164) +26/04/01 05:56:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:32 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:32 INFO Executor: Finished task 72.0 in stage 12.0 (TID 162). 2251 bytes result sent to driver +26/04/01 05:56:32 INFO TaskSetManager: Starting task 75.0 in stage 12.0 (TID 165) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:32 INFO TaskSetManager: Finished task 72.0 in stage 12.0 (TID 162) in 724 ms on 10.0.0.133 (executor driver) (72/208) +26/04/01 05:56:32 INFO Executor: Running task 75.0 in stage 12.0 (TID 165) +26/04/01 05:56:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:32 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:32 INFO Executor: Finished task 70.0 in stage 12.0 (TID 160). 2251 bytes result sent to driver +26/04/01 05:56:32 INFO TaskSetManager: Starting task 76.0 in stage 12.0 (TID 166) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:32 INFO TaskSetManager: Finished task 70.0 in stage 12.0 (TID 160) in 854 ms on 10.0.0.133 (executor driver) (73/208) +26/04/01 05:56:32 INFO Executor: Running task 76.0 in stage 12.0 (TID 166) +26/04/01 05:56:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:32 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:32 INFO Executor: Finished task 73.0 in stage 12.0 (TID 163). 2251 bytes result sent to driver +26/04/01 05:56:32 INFO TaskSetManager: Starting task 77.0 in stage 12.0 (TID 167) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:32 INFO TaskSetManager: Finished task 73.0 in stage 12.0 (TID 163) in 736 ms on 10.0.0.133 (executor driver) (74/208) +26/04/01 05:56:32 INFO Executor: Running task 77.0 in stage 12.0 (TID 167) +26/04/01 05:56:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:32 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:32 INFO Executor: Finished task 74.0 in stage 12.0 (TID 164). 2251 bytes result sent to driver +26/04/01 05:56:32 INFO TaskSetManager: Starting task 78.0 in stage 12.0 (TID 168) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:32 INFO TaskSetManager: Finished task 74.0 in stage 12.0 (TID 164) in 716 ms on 10.0.0.133 (executor driver) (75/208) +26/04/01 05:56:32 INFO Executor: Running task 78.0 in stage 12.0 (TID 168) +26/04/01 05:56:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:32 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:33 INFO Executor: Finished task 75.0 in stage 12.0 (TID 165). 2251 bytes result sent to driver +26/04/01 05:56:33 INFO TaskSetManager: Starting task 79.0 in stage 12.0 (TID 169) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:33 INFO TaskSetManager: Finished task 75.0 in stage 12.0 (TID 165) in 736 ms on 10.0.0.133 (executor driver) (76/208) +26/04/01 05:56:33 INFO Executor: Running task 79.0 in stage 12.0 (TID 169) +26/04/01 05:56:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:33 INFO Executor: Finished task 76.0 in stage 12.0 (TID 166). 2251 bytes result sent to driver +26/04/01 05:56:33 INFO TaskSetManager: Starting task 80.0 in stage 12.0 (TID 170) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:33 INFO TaskSetManager: Finished task 76.0 in stage 12.0 (TID 166) in 730 ms on 10.0.0.133 (executor driver) (77/208) +26/04/01 05:56:33 INFO Executor: Running task 80.0 in stage 12.0 (TID 170) +26/04/01 05:56:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:33 INFO Executor: Finished task 77.0 in stage 12.0 (TID 167). 2251 bytes result sent to driver +26/04/01 05:56:33 INFO TaskSetManager: Starting task 81.0 in stage 12.0 (TID 171) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:33 INFO TaskSetManager: Finished task 77.0 in stage 12.0 (TID 167) in 710 ms on 10.0.0.133 (executor driver) (78/208) +26/04/01 05:56:33 INFO Executor: Running task 81.0 in stage 12.0 (TID 171) +26/04/01 05:56:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:33 INFO Executor: Finished task 78.0 in stage 12.0 (TID 168). 2251 bytes result sent to driver +26/04/01 05:56:33 INFO TaskSetManager: Starting task 82.0 in stage 12.0 (TID 172) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:33 INFO TaskSetManager: Finished task 78.0 in stage 12.0 (TID 168) in 733 ms on 10.0.0.133 (executor driver) (79/208) +26/04/01 05:56:33 INFO Executor: Running task 82.0 in stage 12.0 (TID 172) +26/04/01 05:56:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:33 INFO Executor: Finished task 80.0 in stage 12.0 (TID 170). 2251 bytes result sent to driver +26/04/01 05:56:33 INFO TaskSetManager: Starting task 83.0 in stage 12.0 (TID 173) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:33 INFO TaskSetManager: Finished task 80.0 in stage 12.0 (TID 170) in 720 ms on 10.0.0.133 (executor driver) (80/208) +26/04/01 05:56:33 INFO Executor: Running task 83.0 in stage 12.0 (TID 173) +26/04/01 05:56:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:33 INFO Executor: Finished task 79.0 in stage 12.0 (TID 169). 2251 bytes result sent to driver +26/04/01 05:56:33 INFO TaskSetManager: Starting task 84.0 in stage 12.0 (TID 174) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:33 INFO Executor: Running task 84.0 in stage 12.0 (TID 174) +26/04/01 05:56:33 INFO TaskSetManager: Finished task 79.0 in stage 12.0 (TID 169) in 737 ms on 10.0.0.133 (executor driver) (81/208) +26/04/01 05:56:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:34 INFO Executor: Finished task 81.0 in stage 12.0 (TID 171). 2251 bytes result sent to driver +26/04/01 05:56:34 INFO TaskSetManager: Starting task 85.0 in stage 12.0 (TID 175) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:34 INFO TaskSetManager: Finished task 81.0 in stage 12.0 (TID 171) in 728 ms on 10.0.0.133 (executor driver) (82/208) +26/04/01 05:56:34 INFO Executor: Running task 85.0 in stage 12.0 (TID 175) +26/04/01 05:56:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:34 INFO Executor: Finished task 82.0 in stage 12.0 (TID 172). 2251 bytes result sent to driver +26/04/01 05:56:34 INFO TaskSetManager: Starting task 86.0 in stage 12.0 (TID 176) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:34 INFO TaskSetManager: Finished task 82.0 in stage 12.0 (TID 172) in 697 ms on 10.0.0.133 (executor driver) (83/208) +26/04/01 05:56:34 INFO Executor: Running task 86.0 in stage 12.0 (TID 176) +26/04/01 05:56:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:34 INFO Executor: Finished task 83.0 in stage 12.0 (TID 173). 2251 bytes result sent to driver +26/04/01 05:56:34 INFO TaskSetManager: Starting task 87.0 in stage 12.0 (TID 177) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:34 INFO TaskSetManager: Finished task 83.0 in stage 12.0 (TID 173) in 707 ms on 10.0.0.133 (executor driver) (84/208) +26/04/01 05:56:34 INFO Executor: Running task 87.0 in stage 12.0 (TID 177) +26/04/01 05:56:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:34 INFO Executor: Finished task 84.0 in stage 12.0 (TID 174). 2251 bytes result sent to driver +26/04/01 05:56:34 INFO TaskSetManager: Starting task 88.0 in stage 12.0 (TID 178) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:34 INFO Executor: Running task 88.0 in stage 12.0 (TID 178) +26/04/01 05:56:34 INFO TaskSetManager: Finished task 84.0 in stage 12.0 (TID 174) in 708 ms on 10.0.0.133 (executor driver) (85/208) +26/04/01 05:56:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:34 INFO Executor: Finished task 85.0 in stage 12.0 (TID 175). 2251 bytes result sent to driver +26/04/01 05:56:34 INFO TaskSetManager: Starting task 89.0 in stage 12.0 (TID 179) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:34 INFO TaskSetManager: Finished task 85.0 in stage 12.0 (TID 175) in 701 ms on 10.0.0.133 (executor driver) (86/208) +26/04/01 05:56:34 INFO Executor: Running task 89.0 in stage 12.0 (TID 179) +26/04/01 05:56:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:35 INFO Executor: Finished task 86.0 in stage 12.0 (TID 176). 2251 bytes result sent to driver +26/04/01 05:56:35 INFO TaskSetManager: Starting task 90.0 in stage 12.0 (TID 180) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:35 INFO TaskSetManager: Finished task 86.0 in stage 12.0 (TID 176) in 710 ms on 10.0.0.133 (executor driver) (87/208) +26/04/01 05:56:35 INFO Executor: Running task 90.0 in stage 12.0 (TID 180) +26/04/01 05:56:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:35 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:35 INFO Executor: Finished task 87.0 in stage 12.0 (TID 177). 2251 bytes result sent to driver +26/04/01 05:56:35 INFO TaskSetManager: Starting task 91.0 in stage 12.0 (TID 181) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:35 INFO TaskSetManager: Finished task 87.0 in stage 12.0 (TID 177) in 735 ms on 10.0.0.133 (executor driver) (88/208) +26/04/01 05:56:35 INFO Executor: Running task 91.0 in stage 12.0 (TID 181) +26/04/01 05:56:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:35 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:35 INFO Executor: Finished task 88.0 in stage 12.0 (TID 178). 2251 bytes result sent to driver +26/04/01 05:56:35 INFO TaskSetManager: Starting task 92.0 in stage 12.0 (TID 182) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:35 INFO Executor: Running task 92.0 in stage 12.0 (TID 182) +26/04/01 05:56:35 INFO TaskSetManager: Finished task 88.0 in stage 12.0 (TID 178) in 782 ms on 10.0.0.133 (executor driver) (89/208) +26/04/01 05:56:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:35 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:35 INFO Executor: Finished task 89.0 in stage 12.0 (TID 179). 2251 bytes result sent to driver +26/04/01 05:56:35 INFO TaskSetManager: Starting task 93.0 in stage 12.0 (TID 183) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:35 INFO Executor: Running task 93.0 in stage 12.0 (TID 183) +26/04/01 05:56:35 INFO TaskSetManager: Finished task 89.0 in stage 12.0 (TID 179) in 743 ms on 10.0.0.133 (executor driver) (90/208) +26/04/01 05:56:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:35 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:35 INFO Executor: Finished task 90.0 in stage 12.0 (TID 180). 2251 bytes result sent to driver +26/04/01 05:56:35 INFO TaskSetManager: Starting task 94.0 in stage 12.0 (TID 184) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:35 INFO TaskSetManager: Finished task 90.0 in stage 12.0 (TID 180) in 720 ms on 10.0.0.133 (executor driver) (91/208) +26/04/01 05:56:35 INFO Executor: Running task 94.0 in stage 12.0 (TID 184) +26/04/01 05:56:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:35 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:35 INFO Executor: Finished task 91.0 in stage 12.0 (TID 181). 2251 bytes result sent to driver +26/04/01 05:56:35 INFO TaskSetManager: Starting task 95.0 in stage 12.0 (TID 185) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:35 INFO Executor: Running task 95.0 in stage 12.0 (TID 185) +26/04/01 05:56:35 INFO TaskSetManager: Finished task 91.0 in stage 12.0 (TID 181) in 745 ms on 10.0.0.133 (executor driver) (92/208) +26/04/01 05:56:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:35 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:36 INFO Executor: Finished task 92.0 in stage 12.0 (TID 182). 2251 bytes result sent to driver +26/04/01 05:56:36 INFO TaskSetManager: Starting task 96.0 in stage 12.0 (TID 186) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:36 INFO TaskSetManager: Finished task 92.0 in stage 12.0 (TID 182) in 723 ms on 10.0.0.133 (executor driver) (93/208) +26/04/01 05:56:36 INFO Executor: Running task 96.0 in stage 12.0 (TID 186) +26/04/01 05:56:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:36 INFO Executor: Finished task 93.0 in stage 12.0 (TID 183). 2251 bytes result sent to driver +26/04/01 05:56:36 INFO TaskSetManager: Starting task 97.0 in stage 12.0 (TID 187) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:36 INFO TaskSetManager: Finished task 93.0 in stage 12.0 (TID 183) in 714 ms on 10.0.0.133 (executor driver) (94/208) +26/04/01 05:56:36 INFO Executor: Running task 97.0 in stage 12.0 (TID 187) +26/04/01 05:56:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:36 INFO Executor: Finished task 94.0 in stage 12.0 (TID 184). 2251 bytes result sent to driver +26/04/01 05:56:36 INFO TaskSetManager: Starting task 98.0 in stage 12.0 (TID 188) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:36 INFO TaskSetManager: Finished task 94.0 in stage 12.0 (TID 184) in 700 ms on 10.0.0.133 (executor driver) (95/208) +26/04/01 05:56:36 INFO Executor: Running task 98.0 in stage 12.0 (TID 188) +26/04/01 05:56:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:36 INFO Executor: Finished task 95.0 in stage 12.0 (TID 185). 2251 bytes result sent to driver +26/04/01 05:56:36 INFO TaskSetManager: Starting task 99.0 in stage 12.0 (TID 189) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:36 INFO TaskSetManager: Finished task 95.0 in stage 12.0 (TID 185) in 720 ms on 10.0.0.133 (executor driver) (96/208) +26/04/01 05:56:36 INFO Executor: Running task 99.0 in stage 12.0 (TID 189) +26/04/01 05:56:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:36 INFO Executor: Finished task 96.0 in stage 12.0 (TID 186). 2251 bytes result sent to driver +26/04/01 05:56:36 INFO TaskSetManager: Starting task 100.0 in stage 12.0 (TID 190) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:36 INFO TaskSetManager: Finished task 96.0 in stage 12.0 (TID 186) in 747 ms on 10.0.0.133 (executor driver) (97/208) +26/04/01 05:56:36 INFO Executor: Running task 100.0 in stage 12.0 (TID 190) +26/04/01 05:56:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:37 INFO Executor: Finished task 97.0 in stage 12.0 (TID 187). 2251 bytes result sent to driver +26/04/01 05:56:37 INFO TaskSetManager: Starting task 101.0 in stage 12.0 (TID 191) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:37 INFO TaskSetManager: Finished task 97.0 in stage 12.0 (TID 187) in 764 ms on 10.0.0.133 (executor driver) (98/208) +26/04/01 05:56:37 INFO Executor: Running task 101.0 in stage 12.0 (TID 191) +26/04/01 05:56:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:37 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:37 INFO Executor: Finished task 98.0 in stage 12.0 (TID 188). 2251 bytes result sent to driver +26/04/01 05:56:37 INFO TaskSetManager: Starting task 102.0 in stage 12.0 (TID 192) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:37 INFO TaskSetManager: Finished task 98.0 in stage 12.0 (TID 188) in 806 ms on 10.0.0.133 (executor driver) (99/208) +26/04/01 05:56:37 INFO Executor: Running task 102.0 in stage 12.0 (TID 192) +26/04/01 05:56:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:37 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:37 INFO Executor: Finished task 99.0 in stage 12.0 (TID 189). 2251 bytes result sent to driver +26/04/01 05:56:37 INFO TaskSetManager: Starting task 103.0 in stage 12.0 (TID 193) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:37 INFO TaskSetManager: Finished task 99.0 in stage 12.0 (TID 189) in 762 ms on 10.0.0.133 (executor driver) (100/208) +26/04/01 05:56:37 INFO Executor: Running task 103.0 in stage 12.0 (TID 193) +26/04/01 05:56:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:37 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:37 INFO Executor: Finished task 100.0 in stage 12.0 (TID 190). 2251 bytes result sent to driver +26/04/01 05:56:37 INFO TaskSetManager: Starting task 104.0 in stage 12.0 (TID 194) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:37 INFO TaskSetManager: Finished task 100.0 in stage 12.0 (TID 190) in 752 ms on 10.0.0.133 (executor driver) (101/208) +26/04/01 05:56:37 INFO Executor: Running task 104.0 in stage 12.0 (TID 194) +26/04/01 05:56:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:37 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:37 INFO Executor: Finished task 101.0 in stage 12.0 (TID 191). 2251 bytes result sent to driver +26/04/01 05:56:37 INFO TaskSetManager: Starting task 105.0 in stage 12.0 (TID 195) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:37 INFO TaskSetManager: Finished task 101.0 in stage 12.0 (TID 191) in 756 ms on 10.0.0.133 (executor driver) (102/208) +26/04/01 05:56:37 INFO Executor: Running task 105.0 in stage 12.0 (TID 195) +26/04/01 05:56:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:37 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:38 INFO Executor: Finished task 102.0 in stage 12.0 (TID 192). 2251 bytes result sent to driver +26/04/01 05:56:38 INFO TaskSetManager: Starting task 106.0 in stage 12.0 (TID 196) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:38 INFO TaskSetManager: Finished task 102.0 in stage 12.0 (TID 192) in 740 ms on 10.0.0.133 (executor driver) (103/208) +26/04/01 05:56:38 INFO Executor: Running task 106.0 in stage 12.0 (TID 196) +26/04/01 05:56:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:38 INFO Executor: Finished task 103.0 in stage 12.0 (TID 193). 2251 bytes result sent to driver +26/04/01 05:56:38 INFO TaskSetManager: Starting task 107.0 in stage 12.0 (TID 197) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:38 INFO TaskSetManager: Finished task 103.0 in stage 12.0 (TID 193) in 728 ms on 10.0.0.133 (executor driver) (104/208) +26/04/01 05:56:38 INFO Executor: Running task 107.0 in stage 12.0 (TID 197) +26/04/01 05:56:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:38 INFO Executor: Finished task 104.0 in stage 12.0 (TID 194). 2251 bytes result sent to driver +26/04/01 05:56:38 INFO TaskSetManager: Starting task 108.0 in stage 12.0 (TID 198) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:38 INFO TaskSetManager: Finished task 104.0 in stage 12.0 (TID 194) in 739 ms on 10.0.0.133 (executor driver) (105/208) +26/04/01 05:56:38 INFO Executor: Running task 108.0 in stage 12.0 (TID 198) +26/04/01 05:56:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:38 INFO Executor: Finished task 105.0 in stage 12.0 (TID 195). 2251 bytes result sent to driver +26/04/01 05:56:38 INFO TaskSetManager: Starting task 109.0 in stage 12.0 (TID 199) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:38 INFO TaskSetManager: Finished task 105.0 in stage 12.0 (TID 195) in 762 ms on 10.0.0.133 (executor driver) (106/208) +26/04/01 05:56:38 INFO Executor: Running task 109.0 in stage 12.0 (TID 199) +26/04/01 05:56:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:38 INFO Executor: Finished task 106.0 in stage 12.0 (TID 196). 2251 bytes result sent to driver +26/04/01 05:56:38 INFO TaskSetManager: Starting task 110.0 in stage 12.0 (TID 200) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:38 INFO TaskSetManager: Finished task 106.0 in stage 12.0 (TID 196) in 770 ms on 10.0.0.133 (executor driver) (107/208) +26/04/01 05:56:38 INFO Executor: Running task 110.0 in stage 12.0 (TID 200) +26/04/01 05:56:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:38 INFO Executor: Finished task 107.0 in stage 12.0 (TID 197). 2251 bytes result sent to driver +26/04/01 05:56:38 INFO TaskSetManager: Starting task 111.0 in stage 12.0 (TID 201) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:38 INFO TaskSetManager: Finished task 107.0 in stage 12.0 (TID 197) in 806 ms on 10.0.0.133 (executor driver) (108/208) +26/04/01 05:56:38 INFO Executor: Running task 111.0 in stage 12.0 (TID 201) +26/04/01 05:56:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:39 INFO Executor: Finished task 108.0 in stage 12.0 (TID 198). 2251 bytes result sent to driver +26/04/01 05:56:39 INFO TaskSetManager: Starting task 112.0 in stage 12.0 (TID 202) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:39 INFO TaskSetManager: Finished task 108.0 in stage 12.0 (TID 198) in 788 ms on 10.0.0.133 (executor driver) (109/208) +26/04/01 05:56:39 INFO Executor: Running task 112.0 in stage 12.0 (TID 202) +26/04/01 05:56:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:39 INFO Executor: Finished task 109.0 in stage 12.0 (TID 199). 2251 bytes result sent to driver +26/04/01 05:56:39 INFO TaskSetManager: Starting task 113.0 in stage 12.0 (TID 203) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:39 INFO TaskSetManager: Finished task 109.0 in stage 12.0 (TID 199) in 765 ms on 10.0.0.133 (executor driver) (110/208) +26/04/01 05:56:39 INFO Executor: Running task 113.0 in stage 12.0 (TID 203) +26/04/01 05:56:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:39 INFO Executor: Finished task 110.0 in stage 12.0 (TID 200). 2251 bytes result sent to driver +26/04/01 05:56:39 INFO TaskSetManager: Starting task 114.0 in stage 12.0 (TID 204) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:39 INFO TaskSetManager: Finished task 110.0 in stage 12.0 (TID 200) in 742 ms on 10.0.0.133 (executor driver) (111/208) +26/04/01 05:56:39 INFO Executor: Running task 114.0 in stage 12.0 (TID 204) +26/04/01 05:56:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:39 INFO Executor: Finished task 111.0 in stage 12.0 (TID 201). 2251 bytes result sent to driver +26/04/01 05:56:39 INFO TaskSetManager: Starting task 115.0 in stage 12.0 (TID 205) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:39 INFO TaskSetManager: Finished task 111.0 in stage 12.0 (TID 201) in 700 ms on 10.0.0.133 (executor driver) (112/208) +26/04/01 05:56:39 INFO Executor: Running task 115.0 in stage 12.0 (TID 205) +26/04/01 05:56:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:39 INFO Executor: Finished task 112.0 in stage 12.0 (TID 202). 2251 bytes result sent to driver +26/04/01 05:56:39 INFO TaskSetManager: Starting task 116.0 in stage 12.0 (TID 206) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:39 INFO TaskSetManager: Finished task 112.0 in stage 12.0 (TID 202) in 736 ms on 10.0.0.133 (executor driver) (113/208) +26/04/01 05:56:39 INFO Executor: Running task 116.0 in stage 12.0 (TID 206) +26/04/01 05:56:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:40 INFO Executor: Finished task 113.0 in stage 12.0 (TID 203). 2251 bytes result sent to driver +26/04/01 05:56:40 INFO TaskSetManager: Starting task 117.0 in stage 12.0 (TID 207) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:40 INFO TaskSetManager: Finished task 113.0 in stage 12.0 (TID 203) in 727 ms on 10.0.0.133 (executor driver) (114/208) +26/04/01 05:56:40 INFO Executor: Running task 117.0 in stage 12.0 (TID 207) +26/04/01 05:56:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:40 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:40 INFO Executor: Finished task 114.0 in stage 12.0 (TID 204). 2251 bytes result sent to driver +26/04/01 05:56:40 INFO TaskSetManager: Starting task 118.0 in stage 12.0 (TID 208) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:40 INFO TaskSetManager: Finished task 114.0 in stage 12.0 (TID 204) in 731 ms on 10.0.0.133 (executor driver) (115/208) +26/04/01 05:56:40 INFO Executor: Running task 118.0 in stage 12.0 (TID 208) +26/04/01 05:56:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:40 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:40 INFO Executor: Finished task 115.0 in stage 12.0 (TID 205). 2251 bytes result sent to driver +26/04/01 05:56:40 INFO TaskSetManager: Starting task 119.0 in stage 12.0 (TID 209) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:40 INFO TaskSetManager: Finished task 115.0 in stage 12.0 (TID 205) in 736 ms on 10.0.0.133 (executor driver) (116/208) +26/04/01 05:56:40 INFO Executor: Running task 119.0 in stage 12.0 (TID 209) +26/04/01 05:56:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:40 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:40 INFO Executor: Finished task 116.0 in stage 12.0 (TID 206). 2251 bytes result sent to driver +26/04/01 05:56:40 INFO TaskSetManager: Starting task 120.0 in stage 12.0 (TID 210) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:40 INFO TaskSetManager: Finished task 116.0 in stage 12.0 (TID 206) in 724 ms on 10.0.0.133 (executor driver) (117/208) +26/04/01 05:56:40 INFO Executor: Running task 120.0 in stage 12.0 (TID 210) +26/04/01 05:56:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:40 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:40 INFO Executor: Finished task 117.0 in stage 12.0 (TID 207). 2251 bytes result sent to driver +26/04/01 05:56:40 INFO TaskSetManager: Starting task 121.0 in stage 12.0 (TID 211) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:40 INFO Executor: Running task 121.0 in stage 12.0 (TID 211) +26/04/01 05:56:40 INFO TaskSetManager: Finished task 117.0 in stage 12.0 (TID 207) in 752 ms on 10.0.0.133 (executor driver) (118/208) +26/04/01 05:56:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:40 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:41 INFO Executor: Finished task 118.0 in stage 12.0 (TID 208). 2251 bytes result sent to driver +26/04/01 05:56:41 INFO TaskSetManager: Starting task 122.0 in stage 12.0 (TID 212) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:41 INFO TaskSetManager: Finished task 118.0 in stage 12.0 (TID 208) in 750 ms on 10.0.0.133 (executor driver) (119/208) +26/04/01 05:56:41 INFO Executor: Running task 122.0 in stage 12.0 (TID 212) +26/04/01 05:56:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:41 INFO Executor: Finished task 119.0 in stage 12.0 (TID 209). 2251 bytes result sent to driver +26/04/01 05:56:41 INFO TaskSetManager: Starting task 123.0 in stage 12.0 (TID 213) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:41 INFO Executor: Running task 123.0 in stage 12.0 (TID 213) +26/04/01 05:56:41 INFO TaskSetManager: Finished task 119.0 in stage 12.0 (TID 209) in 725 ms on 10.0.0.133 (executor driver) (120/208) +26/04/01 05:56:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:41 INFO Executor: Finished task 120.0 in stage 12.0 (TID 210). 2251 bytes result sent to driver +26/04/01 05:56:41 INFO TaskSetManager: Starting task 124.0 in stage 12.0 (TID 214) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:41 INFO TaskSetManager: Finished task 120.0 in stage 12.0 (TID 210) in 736 ms on 10.0.0.133 (executor driver) (121/208) +26/04/01 05:56:41 INFO Executor: Running task 124.0 in stage 12.0 (TID 214) +26/04/01 05:56:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:41 INFO Executor: Finished task 121.0 in stage 12.0 (TID 211). 2251 bytes result sent to driver +26/04/01 05:56:41 INFO TaskSetManager: Starting task 125.0 in stage 12.0 (TID 215) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:41 INFO TaskSetManager: Finished task 121.0 in stage 12.0 (TID 211) in 705 ms on 10.0.0.133 (executor driver) (122/208) +26/04/01 05:56:41 INFO Executor: Running task 125.0 in stage 12.0 (TID 215) +26/04/01 05:56:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:41 INFO Executor: Finished task 122.0 in stage 12.0 (TID 212). 2251 bytes result sent to driver +26/04/01 05:56:41 INFO TaskSetManager: Starting task 126.0 in stage 12.0 (TID 216) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:41 INFO Executor: Running task 126.0 in stage 12.0 (TID 216) +26/04/01 05:56:41 INFO TaskSetManager: Finished task 122.0 in stage 12.0 (TID 212) in 704 ms on 10.0.0.133 (executor driver) (123/208) +26/04/01 05:56:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:41 INFO Executor: Finished task 123.0 in stage 12.0 (TID 213). 2251 bytes result sent to driver +26/04/01 05:56:41 INFO TaskSetManager: Starting task 127.0 in stage 12.0 (TID 217) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:41 INFO TaskSetManager: Finished task 123.0 in stage 12.0 (TID 213) in 699 ms on 10.0.0.133 (executor driver) (124/208) +26/04/01 05:56:41 INFO Executor: Running task 127.0 in stage 12.0 (TID 217) +26/04/01 05:56:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:41 INFO Executor: Finished task 124.0 in stage 12.0 (TID 214). 2251 bytes result sent to driver +26/04/01 05:56:41 INFO TaskSetManager: Starting task 128.0 in stage 12.0 (TID 218) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:41 INFO TaskSetManager: Finished task 124.0 in stage 12.0 (TID 214) in 714 ms on 10.0.0.133 (executor driver) (125/208) +26/04/01 05:56:41 INFO Executor: Running task 128.0 in stage 12.0 (TID 218) +26/04/01 05:56:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:42 INFO Executor: Finished task 125.0 in stage 12.0 (TID 215). 2251 bytes result sent to driver +26/04/01 05:56:42 INFO TaskSetManager: Starting task 129.0 in stage 12.0 (TID 219) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:42 INFO TaskSetManager: Finished task 125.0 in stage 12.0 (TID 215) in 710 ms on 10.0.0.133 (executor driver) (126/208) +26/04/01 05:56:42 INFO Executor: Running task 129.0 in stage 12.0 (TID 219) +26/04/01 05:56:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:42 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:42 INFO Executor: Finished task 126.0 in stage 12.0 (TID 216). 2251 bytes result sent to driver +26/04/01 05:56:42 INFO TaskSetManager: Starting task 130.0 in stage 12.0 (TID 220) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:42 INFO TaskSetManager: Finished task 126.0 in stage 12.0 (TID 216) in 717 ms on 10.0.0.133 (executor driver) (127/208) +26/04/01 05:56:42 INFO Executor: Running task 130.0 in stage 12.0 (TID 220) +26/04/01 05:56:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:42 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:42 INFO Executor: Finished task 127.0 in stage 12.0 (TID 217). 2251 bytes result sent to driver +26/04/01 05:56:42 INFO TaskSetManager: Starting task 131.0 in stage 12.0 (TID 221) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:42 INFO Executor: Running task 131.0 in stage 12.0 (TID 221) +26/04/01 05:56:42 INFO TaskSetManager: Finished task 127.0 in stage 12.0 (TID 217) in 739 ms on 10.0.0.133 (executor driver) (128/208) +26/04/01 05:56:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:42 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:42 INFO Executor: Finished task 128.0 in stage 12.0 (TID 218). 2251 bytes result sent to driver +26/04/01 05:56:42 INFO TaskSetManager: Starting task 132.0 in stage 12.0 (TID 222) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:42 INFO TaskSetManager: Finished task 128.0 in stage 12.0 (TID 218) in 710 ms on 10.0.0.133 (executor driver) (129/208) +26/04/01 05:56:42 INFO Executor: Running task 132.0 in stage 12.0 (TID 222) +26/04/01 05:56:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:42 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:43 INFO Executor: Finished task 129.0 in stage 12.0 (TID 219). 2251 bytes result sent to driver +26/04/01 05:56:43 INFO TaskSetManager: Starting task 133.0 in stage 12.0 (TID 223) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:43 INFO TaskSetManager: Finished task 129.0 in stage 12.0 (TID 219) in 730 ms on 10.0.0.133 (executor driver) (130/208) +26/04/01 05:56:43 INFO Executor: Running task 133.0 in stage 12.0 (TID 223) +26/04/01 05:56:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:43 INFO Executor: Finished task 130.0 in stage 12.0 (TID 220). 2251 bytes result sent to driver +26/04/01 05:56:43 INFO TaskSetManager: Starting task 134.0 in stage 12.0 (TID 224) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:43 INFO TaskSetManager: Finished task 130.0 in stage 12.0 (TID 220) in 702 ms on 10.0.0.133 (executor driver) (131/208) +26/04/01 05:56:43 INFO Executor: Running task 134.0 in stage 12.0 (TID 224) +26/04/01 05:56:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:43 INFO Executor: Finished task 131.0 in stage 12.0 (TID 221). 2251 bytes result sent to driver +26/04/01 05:56:43 INFO TaskSetManager: Starting task 135.0 in stage 12.0 (TID 225) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:43 INFO TaskSetManager: Finished task 131.0 in stage 12.0 (TID 221) in 696 ms on 10.0.0.133 (executor driver) (132/208) +26/04/01 05:56:43 INFO Executor: Running task 135.0 in stage 12.0 (TID 225) +26/04/01 05:56:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:43 INFO Executor: Finished task 132.0 in stage 12.0 (TID 222). 2251 bytes result sent to driver +26/04/01 05:56:43 INFO TaskSetManager: Starting task 136.0 in stage 12.0 (TID 226) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:43 INFO TaskSetManager: Finished task 132.0 in stage 12.0 (TID 222) in 709 ms on 10.0.0.133 (executor driver) (133/208) +26/04/01 05:56:43 INFO Executor: Running task 136.0 in stage 12.0 (TID 226) +26/04/01 05:56:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:43 INFO Executor: Finished task 133.0 in stage 12.0 (TID 223). 2251 bytes result sent to driver +26/04/01 05:56:43 INFO TaskSetManager: Starting task 137.0 in stage 12.0 (TID 227) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:43 INFO TaskSetManager: Finished task 133.0 in stage 12.0 (TID 223) in 714 ms on 10.0.0.133 (executor driver) (134/208) +26/04/01 05:56:43 INFO Executor: Running task 137.0 in stage 12.0 (TID 227) +26/04/01 05:56:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:43 INFO Executor: Finished task 134.0 in stage 12.0 (TID 224). 2251 bytes result sent to driver +26/04/01 05:56:43 INFO TaskSetManager: Starting task 138.0 in stage 12.0 (TID 228) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:43 INFO TaskSetManager: Finished task 134.0 in stage 12.0 (TID 224) in 708 ms on 10.0.0.133 (executor driver) (135/208) +26/04/01 05:56:43 INFO Executor: Running task 138.0 in stage 12.0 (TID 228) +26/04/01 05:56:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:44 INFO Executor: Finished task 135.0 in stage 12.0 (TID 225). 2251 bytes result sent to driver +26/04/01 05:56:44 INFO TaskSetManager: Starting task 139.0 in stage 12.0 (TID 229) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:44 INFO TaskSetManager: Finished task 135.0 in stage 12.0 (TID 225) in 708 ms on 10.0.0.133 (executor driver) (136/208) +26/04/01 05:56:44 INFO Executor: Running task 139.0 in stage 12.0 (TID 229) +26/04/01 05:56:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:44 INFO Executor: Finished task 136.0 in stage 12.0 (TID 226). 2251 bytes result sent to driver +26/04/01 05:56:44 INFO TaskSetManager: Starting task 140.0 in stage 12.0 (TID 230) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:44 INFO TaskSetManager: Finished task 136.0 in stage 12.0 (TID 226) in 744 ms on 10.0.0.133 (executor driver) (137/208) +26/04/01 05:56:44 INFO Executor: Running task 140.0 in stage 12.0 (TID 230) +26/04/01 05:56:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:44 INFO Executor: Finished task 137.0 in stage 12.0 (TID 227). 2251 bytes result sent to driver +26/04/01 05:56:44 INFO TaskSetManager: Starting task 141.0 in stage 12.0 (TID 231) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:44 INFO Executor: Running task 141.0 in stage 12.0 (TID 231) +26/04/01 05:56:44 INFO TaskSetManager: Finished task 137.0 in stage 12.0 (TID 227) in 730 ms on 10.0.0.133 (executor driver) (138/208) +26/04/01 05:56:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:44 INFO Executor: Finished task 138.0 in stage 12.0 (TID 228). 2251 bytes result sent to driver +26/04/01 05:56:44 INFO TaskSetManager: Starting task 142.0 in stage 12.0 (TID 232) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:44 INFO TaskSetManager: Finished task 138.0 in stage 12.0 (TID 228) in 745 ms on 10.0.0.133 (executor driver) (139/208) +26/04/01 05:56:44 INFO Executor: Running task 142.0 in stage 12.0 (TID 232) +26/04/01 05:56:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:44 INFO Executor: Finished task 139.0 in stage 12.0 (TID 229). 2251 bytes result sent to driver +26/04/01 05:56:44 INFO TaskSetManager: Starting task 143.0 in stage 12.0 (TID 233) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:44 INFO Executor: Running task 143.0 in stage 12.0 (TID 233) +26/04/01 05:56:44 INFO TaskSetManager: Finished task 139.0 in stage 12.0 (TID 229) in 714 ms on 10.0.0.133 (executor driver) (140/208) +26/04/01 05:56:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:44 INFO Executor: Finished task 140.0 in stage 12.0 (TID 230). 2251 bytes result sent to driver +26/04/01 05:56:44 INFO TaskSetManager: Starting task 144.0 in stage 12.0 (TID 234) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:44 INFO TaskSetManager: Finished task 140.0 in stage 12.0 (TID 230) in 714 ms on 10.0.0.133 (executor driver) (141/208) +26/04/01 05:56:44 INFO Executor: Running task 144.0 in stage 12.0 (TID 234) +26/04/01 05:56:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:45 INFO Executor: Finished task 141.0 in stage 12.0 (TID 231). 2251 bytes result sent to driver +26/04/01 05:56:45 INFO TaskSetManager: Starting task 145.0 in stage 12.0 (TID 235) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:45 INFO TaskSetManager: Finished task 141.0 in stage 12.0 (TID 231) in 723 ms on 10.0.0.133 (executor driver) (142/208) +26/04/01 05:56:45 INFO Executor: Running task 145.0 in stage 12.0 (TID 235) +26/04/01 05:56:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:45 INFO Executor: Finished task 142.0 in stage 12.0 (TID 232). 2251 bytes result sent to driver +26/04/01 05:56:45 INFO TaskSetManager: Starting task 146.0 in stage 12.0 (TID 236) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:45 INFO TaskSetManager: Finished task 142.0 in stage 12.0 (TID 232) in 733 ms on 10.0.0.133 (executor driver) (143/208) +26/04/01 05:56:45 INFO Executor: Running task 146.0 in stage 12.0 (TID 236) +26/04/01 05:56:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:45 INFO Executor: Finished task 143.0 in stage 12.0 (TID 233). 2251 bytes result sent to driver +26/04/01 05:56:45 INFO TaskSetManager: Starting task 147.0 in stage 12.0 (TID 237) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:45 INFO TaskSetManager: Finished task 143.0 in stage 12.0 (TID 233) in 739 ms on 10.0.0.133 (executor driver) (144/208) +26/04/01 05:56:45 INFO Executor: Running task 147.0 in stage 12.0 (TID 237) +26/04/01 05:56:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:45 INFO Executor: Finished task 144.0 in stage 12.0 (TID 234). 2251 bytes result sent to driver +26/04/01 05:56:45 INFO TaskSetManager: Starting task 148.0 in stage 12.0 (TID 238) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:45 INFO TaskSetManager: Finished task 144.0 in stage 12.0 (TID 234) in 717 ms on 10.0.0.133 (executor driver) (145/208) +26/04/01 05:56:45 INFO Executor: Running task 148.0 in stage 12.0 (TID 238) +26/04/01 05:56:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:45 INFO Executor: Finished task 145.0 in stage 12.0 (TID 235). 2251 bytes result sent to driver +26/04/01 05:56:45 INFO TaskSetManager: Starting task 149.0 in stage 12.0 (TID 239) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:45 INFO TaskSetManager: Finished task 145.0 in stage 12.0 (TID 235) in 745 ms on 10.0.0.133 (executor driver) (146/208) +26/04/01 05:56:45 INFO Executor: Running task 149.0 in stage 12.0 (TID 239) +26/04/01 05:56:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:46 INFO Executor: Finished task 146.0 in stage 12.0 (TID 236). 2251 bytes result sent to driver +26/04/01 05:56:46 INFO TaskSetManager: Starting task 150.0 in stage 12.0 (TID 240) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:46 INFO TaskSetManager: Finished task 146.0 in stage 12.0 (TID 236) in 737 ms on 10.0.0.133 (executor driver) (147/208) +26/04/01 05:56:46 INFO Executor: Running task 150.0 in stage 12.0 (TID 240) +26/04/01 05:56:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:46 INFO Executor: Finished task 147.0 in stage 12.0 (TID 237). 2251 bytes result sent to driver +26/04/01 05:56:46 INFO TaskSetManager: Starting task 151.0 in stage 12.0 (TID 241) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:46 INFO TaskSetManager: Finished task 147.0 in stage 12.0 (TID 237) in 744 ms on 10.0.0.133 (executor driver) (148/208) +26/04/01 05:56:46 INFO Executor: Running task 151.0 in stage 12.0 (TID 241) +26/04/01 05:56:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:46 INFO Executor: Finished task 148.0 in stage 12.0 (TID 238). 2251 bytes result sent to driver +26/04/01 05:56:46 INFO TaskSetManager: Starting task 152.0 in stage 12.0 (TID 242) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:46 INFO TaskSetManager: Finished task 148.0 in stage 12.0 (TID 238) in 723 ms on 10.0.0.133 (executor driver) (149/208) +26/04/01 05:56:46 INFO Executor: Running task 152.0 in stage 12.0 (TID 242) +26/04/01 05:56:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:46 INFO Executor: Finished task 149.0 in stage 12.0 (TID 239). 2251 bytes result sent to driver +26/04/01 05:56:46 INFO TaskSetManager: Starting task 153.0 in stage 12.0 (TID 243) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:46 INFO TaskSetManager: Finished task 149.0 in stage 12.0 (TID 239) in 704 ms on 10.0.0.133 (executor driver) (150/208) +26/04/01 05:56:46 INFO Executor: Running task 153.0 in stage 12.0 (TID 243) +26/04/01 05:56:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:46 INFO Executor: Finished task 150.0 in stage 12.0 (TID 240). 2251 bytes result sent to driver +26/04/01 05:56:46 INFO TaskSetManager: Starting task 154.0 in stage 12.0 (TID 244) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:46 INFO TaskSetManager: Finished task 150.0 in stage 12.0 (TID 240) in 704 ms on 10.0.0.133 (executor driver) (151/208) +26/04/01 05:56:46 INFO Executor: Running task 154.0 in stage 12.0 (TID 244) +26/04/01 05:56:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:46 INFO Executor: Finished task 151.0 in stage 12.0 (TID 241). 2251 bytes result sent to driver +26/04/01 05:56:46 INFO TaskSetManager: Starting task 155.0 in stage 12.0 (TID 245) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:46 INFO TaskSetManager: Finished task 151.0 in stage 12.0 (TID 241) in 699 ms on 10.0.0.133 (executor driver) (152/208) +26/04/01 05:56:46 INFO Executor: Running task 155.0 in stage 12.0 (TID 245) +26/04/01 05:56:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:46 INFO Executor: Finished task 152.0 in stage 12.0 (TID 242). 2251 bytes result sent to driver +26/04/01 05:56:46 INFO TaskSetManager: Starting task 156.0 in stage 12.0 (TID 246) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:46 INFO TaskSetManager: Finished task 152.0 in stage 12.0 (TID 242) in 705 ms on 10.0.0.133 (executor driver) (153/208) +26/04/01 05:56:46 INFO Executor: Running task 156.0 in stage 12.0 (TID 246) +26/04/01 05:56:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:47 INFO Executor: Finished task 153.0 in stage 12.0 (TID 243). 2251 bytes result sent to driver +26/04/01 05:56:47 INFO TaskSetManager: Starting task 157.0 in stage 12.0 (TID 247) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:47 INFO TaskSetManager: Finished task 153.0 in stage 12.0 (TID 243) in 735 ms on 10.0.0.133 (executor driver) (154/208) +26/04/01 05:56:47 INFO Executor: Running task 157.0 in stage 12.0 (TID 247) +26/04/01 05:56:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:47 INFO Executor: Finished task 154.0 in stage 12.0 (TID 244). 2251 bytes result sent to driver +26/04/01 05:56:47 INFO TaskSetManager: Starting task 158.0 in stage 12.0 (TID 248) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:47 INFO TaskSetManager: Finished task 154.0 in stage 12.0 (TID 244) in 724 ms on 10.0.0.133 (executor driver) (155/208) +26/04/01 05:56:47 INFO Executor: Running task 158.0 in stage 12.0 (TID 248) +26/04/01 05:56:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:47 INFO Executor: Finished task 155.0 in stage 12.0 (TID 245). 2251 bytes result sent to driver +26/04/01 05:56:47 INFO TaskSetManager: Starting task 159.0 in stage 12.0 (TID 249) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:47 INFO TaskSetManager: Finished task 155.0 in stage 12.0 (TID 245) in 743 ms on 10.0.0.133 (executor driver) (156/208) +26/04/01 05:56:47 INFO Executor: Running task 159.0 in stage 12.0 (TID 249) +26/04/01 05:56:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:47 INFO Executor: Finished task 156.0 in stage 12.0 (TID 246). 2251 bytes result sent to driver +26/04/01 05:56:47 INFO TaskSetManager: Starting task 160.0 in stage 12.0 (TID 250) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:47 INFO Executor: Running task 160.0 in stage 12.0 (TID 250) +26/04/01 05:56:47 INFO TaskSetManager: Finished task 156.0 in stage 12.0 (TID 246) in 759 ms on 10.0.0.133 (executor driver) (157/208) +26/04/01 05:56:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:48 INFO Executor: Finished task 157.0 in stage 12.0 (TID 247). 2251 bytes result sent to driver +26/04/01 05:56:48 INFO TaskSetManager: Starting task 161.0 in stage 12.0 (TID 251) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:48 INFO TaskSetManager: Finished task 157.0 in stage 12.0 (TID 247) in 733 ms on 10.0.0.133 (executor driver) (158/208) +26/04/01 05:56:48 INFO Executor: Running task 161.0 in stage 12.0 (TID 251) +26/04/01 05:56:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:48 INFO Executor: Finished task 158.0 in stage 12.0 (TID 248). 2251 bytes result sent to driver +26/04/01 05:56:48 INFO TaskSetManager: Starting task 162.0 in stage 12.0 (TID 252) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:48 INFO TaskSetManager: Finished task 158.0 in stage 12.0 (TID 248) in 733 ms on 10.0.0.133 (executor driver) (159/208) +26/04/01 05:56:48 INFO Executor: Running task 162.0 in stage 12.0 (TID 252) +26/04/01 05:56:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:48 INFO Executor: Finished task 159.0 in stage 12.0 (TID 249). 2251 bytes result sent to driver +26/04/01 05:56:48 INFO TaskSetManager: Starting task 163.0 in stage 12.0 (TID 253) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:48 INFO TaskSetManager: Finished task 159.0 in stage 12.0 (TID 249) in 684 ms on 10.0.0.133 (executor driver) (160/208) +26/04/01 05:56:48 INFO Executor: Running task 163.0 in stage 12.0 (TID 253) +26/04/01 05:56:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:48 INFO Executor: Finished task 160.0 in stage 12.0 (TID 250). 2251 bytes result sent to driver +26/04/01 05:56:48 INFO TaskSetManager: Starting task 164.0 in stage 12.0 (TID 254) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:48 INFO TaskSetManager: Finished task 160.0 in stage 12.0 (TID 250) in 705 ms on 10.0.0.133 (executor driver) (161/208) +26/04/01 05:56:48 INFO Executor: Running task 164.0 in stage 12.0 (TID 254) +26/04/01 05:56:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:48 INFO Executor: Finished task 161.0 in stage 12.0 (TID 251). 2251 bytes result sent to driver +26/04/01 05:56:48 INFO TaskSetManager: Starting task 165.0 in stage 12.0 (TID 255) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:48 INFO Executor: Running task 165.0 in stage 12.0 (TID 255) +26/04/01 05:56:48 INFO TaskSetManager: Finished task 161.0 in stage 12.0 (TID 251) in 703 ms on 10.0.0.133 (executor driver) (162/208) +26/04/01 05:56:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:48 INFO Executor: Finished task 162.0 in stage 12.0 (TID 252). 2251 bytes result sent to driver +26/04/01 05:56:48 INFO TaskSetManager: Starting task 166.0 in stage 12.0 (TID 256) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:48 INFO Executor: Running task 166.0 in stage 12.0 (TID 256) +26/04/01 05:56:48 INFO TaskSetManager: Finished task 162.0 in stage 12.0 (TID 252) in 704 ms on 10.0.0.133 (executor driver) (163/208) +26/04/01 05:56:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:49 INFO Executor: Finished task 163.0 in stage 12.0 (TID 253). 2251 bytes result sent to driver +26/04/01 05:56:49 INFO TaskSetManager: Starting task 167.0 in stage 12.0 (TID 257) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:49 INFO TaskSetManager: Finished task 163.0 in stage 12.0 (TID 253) in 732 ms on 10.0.0.133 (executor driver) (164/208) +26/04/01 05:56:49 INFO Executor: Running task 167.0 in stage 12.0 (TID 257) +26/04/01 05:56:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:49 INFO Executor: Finished task 164.0 in stage 12.0 (TID 254). 2251 bytes result sent to driver +26/04/01 05:56:49 INFO TaskSetManager: Starting task 168.0 in stage 12.0 (TID 258) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:49 INFO Executor: Running task 168.0 in stage 12.0 (TID 258) +26/04/01 05:56:49 INFO TaskSetManager: Finished task 164.0 in stage 12.0 (TID 254) in 726 ms on 10.0.0.133 (executor driver) (165/208) +26/04/01 05:56:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:49 INFO Executor: Finished task 165.0 in stage 12.0 (TID 255). 2251 bytes result sent to driver +26/04/01 05:56:49 INFO TaskSetManager: Starting task 169.0 in stage 12.0 (TID 259) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:49 INFO Executor: Running task 169.0 in stage 12.0 (TID 259) +26/04/01 05:56:49 INFO TaskSetManager: Finished task 165.0 in stage 12.0 (TID 255) in 722 ms on 10.0.0.133 (executor driver) (166/208) +26/04/01 05:56:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:49 INFO Executor: Finished task 166.0 in stage 12.0 (TID 256). 2251 bytes result sent to driver +26/04/01 05:56:49 INFO TaskSetManager: Starting task 170.0 in stage 12.0 (TID 260) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:49 INFO TaskSetManager: Finished task 166.0 in stage 12.0 (TID 256) in 705 ms on 10.0.0.133 (executor driver) (167/208) +26/04/01 05:56:49 INFO Executor: Running task 170.0 in stage 12.0 (TID 260) +26/04/01 05:56:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:49 INFO Executor: Finished task 167.0 in stage 12.0 (TID 257). 2251 bytes result sent to driver +26/04/01 05:56:49 INFO TaskSetManager: Starting task 171.0 in stage 12.0 (TID 261) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:49 INFO Executor: Running task 171.0 in stage 12.0 (TID 261) +26/04/01 05:56:49 INFO TaskSetManager: Finished task 167.0 in stage 12.0 (TID 257) in 695 ms on 10.0.0.133 (executor driver) (168/208) +26/04/01 05:56:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:49 INFO Executor: Finished task 168.0 in stage 12.0 (TID 258). 2251 bytes result sent to driver +26/04/01 05:56:49 INFO TaskSetManager: Starting task 172.0 in stage 12.0 (TID 262) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:49 INFO TaskSetManager: Finished task 168.0 in stage 12.0 (TID 258) in 812 ms on 10.0.0.133 (executor driver) (169/208) +26/04/01 05:56:49 INFO Executor: Running task 172.0 in stage 12.0 (TID 262) +26/04/01 05:56:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:50 INFO Executor: Finished task 169.0 in stage 12.0 (TID 259). 2251 bytes result sent to driver +26/04/01 05:56:50 INFO TaskSetManager: Starting task 173.0 in stage 12.0 (TID 263) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:50 INFO TaskSetManager: Finished task 169.0 in stage 12.0 (TID 259) in 714 ms on 10.0.0.133 (executor driver) (170/208) +26/04/01 05:56:50 INFO Executor: Running task 173.0 in stage 12.0 (TID 263) +26/04/01 05:56:50 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:50 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:50 INFO Executor: Finished task 170.0 in stage 12.0 (TID 260). 2251 bytes result sent to driver +26/04/01 05:56:50 INFO TaskSetManager: Starting task 174.0 in stage 12.0 (TID 264) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:50 INFO TaskSetManager: Finished task 170.0 in stage 12.0 (TID 260) in 719 ms on 10.0.0.133 (executor driver) (171/208) +26/04/01 05:56:50 INFO Executor: Running task 174.0 in stage 12.0 (TID 264) +26/04/01 05:56:50 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:50 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:50 INFO Executor: Finished task 171.0 in stage 12.0 (TID 261). 2251 bytes result sent to driver +26/04/01 05:56:50 INFO TaskSetManager: Starting task 175.0 in stage 12.0 (TID 265) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:50 INFO Executor: Running task 175.0 in stage 12.0 (TID 265) +26/04/01 05:56:50 INFO TaskSetManager: Finished task 171.0 in stage 12.0 (TID 261) in 727 ms on 10.0.0.133 (executor driver) (172/208) +26/04/01 05:56:50 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:50 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:50 INFO Executor: Finished task 172.0 in stage 12.0 (TID 262). 2251 bytes result sent to driver +26/04/01 05:56:50 INFO TaskSetManager: Starting task 176.0 in stage 12.0 (TID 266) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:50 INFO TaskSetManager: Finished task 172.0 in stage 12.0 (TID 262) in 704 ms on 10.0.0.133 (executor driver) (173/208) +26/04/01 05:56:50 INFO Executor: Running task 176.0 in stage 12.0 (TID 266) +26/04/01 05:56:50 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:50 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:50 INFO Executor: Finished task 173.0 in stage 12.0 (TID 263). 2251 bytes result sent to driver +26/04/01 05:56:50 INFO TaskSetManager: Starting task 177.0 in stage 12.0 (TID 267) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:50 INFO Executor: Running task 177.0 in stage 12.0 (TID 267) +26/04/01 05:56:50 INFO TaskSetManager: Finished task 173.0 in stage 12.0 (TID 263) in 703 ms on 10.0.0.133 (executor driver) (174/208) +26/04/01 05:56:50 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:50 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:51 INFO Executor: Finished task 174.0 in stage 12.0 (TID 264). 2251 bytes result sent to driver +26/04/01 05:56:51 INFO TaskSetManager: Starting task 178.0 in stage 12.0 (TID 268) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:51 INFO Executor: Running task 178.0 in stage 12.0 (TID 268) +26/04/01 05:56:51 INFO TaskSetManager: Finished task 174.0 in stage 12.0 (TID 264) in 728 ms on 10.0.0.133 (executor driver) (175/208) +26/04/01 05:56:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:51 INFO Executor: Finished task 175.0 in stage 12.0 (TID 265). 2251 bytes result sent to driver +26/04/01 05:56:51 INFO TaskSetManager: Starting task 179.0 in stage 12.0 (TID 269) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9601 bytes) +26/04/01 05:56:51 INFO TaskSetManager: Finished task 175.0 in stage 12.0 (TID 265) in 736 ms on 10.0.0.133 (executor driver) (176/208) +26/04/01 05:56:51 INFO Executor: Running task 179.0 in stage 12.0 (TID 269) +26/04/01 05:56:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:51 INFO Executor: Finished task 176.0 in stage 12.0 (TID 266). 2251 bytes result sent to driver +26/04/01 05:56:51 INFO TaskSetManager: Starting task 180.0 in stage 12.0 (TID 270) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:51 INFO TaskSetManager: Finished task 176.0 in stage 12.0 (TID 266) in 754 ms on 10.0.0.133 (executor driver) (177/208) +26/04/01 05:56:51 INFO Executor: Running task 180.0 in stage 12.0 (TID 270) +26/04/01 05:56:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:51 INFO Executor: Finished task 177.0 in stage 12.0 (TID 267). 2251 bytes result sent to driver +26/04/01 05:56:51 INFO TaskSetManager: Starting task 181.0 in stage 12.0 (TID 271) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:51 INFO Executor: Running task 181.0 in stage 12.0 (TID 271) +26/04/01 05:56:51 INFO TaskSetManager: Finished task 177.0 in stage 12.0 (TID 267) in 726 ms on 10.0.0.133 (executor driver) (178/208) +26/04/01 05:56:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:51 INFO Executor: Finished task 178.0 in stage 12.0 (TID 268). 2251 bytes result sent to driver +26/04/01 05:56:51 INFO TaskSetManager: Starting task 182.0 in stage 12.0 (TID 272) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:51 INFO Executor: Running task 182.0 in stage 12.0 (TID 272) +26/04/01 05:56:51 INFO TaskSetManager: Finished task 178.0 in stage 12.0 (TID 268) in 731 ms on 10.0.0.133 (executor driver) (179/208) +26/04/01 05:56:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:51 INFO Executor: Finished task 179.0 in stage 12.0 (TID 269). 2251 bytes result sent to driver +26/04/01 05:56:51 INFO TaskSetManager: Starting task 183.0 in stage 12.0 (TID 273) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:51 INFO TaskSetManager: Finished task 179.0 in stage 12.0 (TID 269) in 725 ms on 10.0.0.133 (executor driver) (180/208) +26/04/01 05:56:51 INFO Executor: Running task 183.0 in stage 12.0 (TID 273) +26/04/01 05:56:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:52 INFO Executor: Finished task 180.0 in stage 12.0 (TID 270). 2251 bytes result sent to driver +26/04/01 05:56:52 INFO TaskSetManager: Starting task 184.0 in stage 12.0 (TID 274) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:52 INFO TaskSetManager: Finished task 180.0 in stage 12.0 (TID 270) in 707 ms on 10.0.0.133 (executor driver) (181/208) +26/04/01 05:56:52 INFO Executor: Running task 184.0 in stage 12.0 (TID 274) +26/04/01 05:56:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:52 INFO Executor: Finished task 181.0 in stage 12.0 (TID 271). 2251 bytes result sent to driver +26/04/01 05:56:52 INFO TaskSetManager: Starting task 185.0 in stage 12.0 (TID 275) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:52 INFO TaskSetManager: Finished task 181.0 in stage 12.0 (TID 271) in 741 ms on 10.0.0.133 (executor driver) (182/208) +26/04/01 05:56:52 INFO Executor: Running task 185.0 in stage 12.0 (TID 275) +26/04/01 05:56:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:52 INFO Executor: Finished task 182.0 in stage 12.0 (TID 272). 2251 bytes result sent to driver +26/04/01 05:56:52 INFO TaskSetManager: Starting task 186.0 in stage 12.0 (TID 276) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:52 INFO TaskSetManager: Finished task 182.0 in stage 12.0 (TID 272) in 717 ms on 10.0.0.133 (executor driver) (183/208) +26/04/01 05:56:52 INFO Executor: Running task 186.0 in stage 12.0 (TID 276) +26/04/01 05:56:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 05:56:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:52 INFO Executor: Finished task 183.0 in stage 12.0 (TID 273). 2251 bytes result sent to driver +26/04/01 05:56:52 INFO TaskSetManager: Starting task 187.0 in stage 12.0 (TID 277) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:52 INFO Executor: Running task 187.0 in stage 12.0 (TID 277) +26/04/01 05:56:52 INFO TaskSetManager: Finished task 183.0 in stage 12.0 (TID 273) in 754 ms on 10.0.0.133 (executor driver) (184/208) +26/04/01 05:56:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 05:56:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:52 INFO Executor: Finished task 184.0 in stage 12.0 (TID 274). 2251 bytes result sent to driver +26/04/01 05:56:52 INFO TaskSetManager: Starting task 188.0 in stage 12.0 (TID 278) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:52 INFO TaskSetManager: Finished task 184.0 in stage 12.0 (TID 274) in 738 ms on 10.0.0.133 (executor driver) (185/208) +26/04/01 05:56:52 INFO Executor: Running task 188.0 in stage 12.0 (TID 278) +26/04/01 05:56:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 05:56:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:53 INFO Executor: Finished task 185.0 in stage 12.0 (TID 275). 2251 bytes result sent to driver +26/04/01 05:56:53 INFO TaskSetManager: Starting task 189.0 in stage 12.0 (TID 279) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:53 INFO TaskSetManager: Finished task 185.0 in stage 12.0 (TID 275) in 724 ms on 10.0.0.133 (executor driver) (186/208) +26/04/01 05:56:53 INFO Executor: Running task 189.0 in stage 12.0 (TID 279) +26/04/01 05:56:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 05:56:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:53 INFO Executor: Finished task 186.0 in stage 12.0 (TID 276). 2251 bytes result sent to driver +26/04/01 05:56:53 INFO TaskSetManager: Starting task 190.0 in stage 12.0 (TID 280) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:53 INFO Executor: Running task 190.0 in stage 12.0 (TID 280) +26/04/01 05:56:53 INFO TaskSetManager: Finished task 186.0 in stage 12.0 (TID 276) in 725 ms on 10.0.0.133 (executor driver) (187/208) +26/04/01 05:56:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 05:56:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:53 INFO Executor: Finished task 187.0 in stage 12.0 (TID 277). 2251 bytes result sent to driver +26/04/01 05:56:53 INFO TaskSetManager: Starting task 191.0 in stage 12.0 (TID 281) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9602 bytes) +26/04/01 05:56:53 INFO TaskSetManager: Finished task 187.0 in stage 12.0 (TID 277) in 708 ms on 10.0.0.133 (executor driver) (188/208) +26/04/01 05:56:53 INFO Executor: Running task 191.0 in stage 12.0 (TID 281) +26/04/01 05:56:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 05:56:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:53 INFO Executor: Finished task 188.0 in stage 12.0 (TID 278). 2251 bytes result sent to driver +26/04/01 05:56:53 INFO TaskSetManager: Starting task 192.0 in stage 12.0 (TID 282) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:53 INFO TaskSetManager: Finished task 188.0 in stage 12.0 (TID 278) in 716 ms on 10.0.0.133 (executor driver) (189/208) +26/04/01 05:56:53 INFO Executor: Running task 192.0 in stage 12.0 (TID 282) +26/04/01 05:56:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 805306368-848649288, partition values: [empty row] +26/04/01 05:56:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 805306368-848594494, partition values: [empty row] +26/04/01 05:56:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:53 INFO Executor: Finished task 189.0 in stage 12.0 (TID 279). 2251 bytes result sent to driver +26/04/01 05:56:53 INFO TaskSetManager: Starting task 193.0 in stage 12.0 (TID 283) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:53 INFO TaskSetManager: Finished task 189.0 in stage 12.0 (TID 279) in 728 ms on 10.0.0.133 (executor driver) (190/208) +26/04/01 05:56:53 INFO Executor: Running task 193.0 in stage 12.0 (TID 283) +26/04/01 05:56:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 805306368-848563442, partition values: [empty row] +26/04/01 05:56:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO Executor: Finished task 190.0 in stage 12.0 (TID 280). 2251 bytes result sent to driver +26/04/01 05:56:54 INFO TaskSetManager: Starting task 194.0 in stage 12.0 (TID 284) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:54 INFO TaskSetManager: Finished task 190.0 in stage 12.0 (TID 280) in 715 ms on 10.0.0.133 (executor driver) (191/208) +26/04/01 05:56:54 INFO Executor: Running task 194.0 in stage 12.0 (TID 284) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 805306368-848496433, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO Executor: Finished task 192.0 in stage 12.0 (TID 282). 2251 bytes result sent to driver +26/04/01 05:56:54 INFO TaskSetManager: Starting task 195.0 in stage 12.0 (TID 285) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:54 INFO TaskSetManager: Finished task 192.0 in stage 12.0 (TID 282) in 467 ms on 10.0.0.133 (executor driver) (192/208) +26/04/01 05:56:54 INFO Executor: Running task 195.0 in stage 12.0 (TID 285) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 805306368-848463796, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO Executor: Finished task 191.0 in stage 12.0 (TID 281). 2251 bytes result sent to driver +26/04/01 05:56:54 INFO TaskSetManager: Starting task 196.0 in stage 12.0 (TID 286) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:54 INFO TaskSetManager: Finished task 191.0 in stage 12.0 (TID 281) in 732 ms on 10.0.0.133 (executor driver) (193/208) +26/04/01 05:56:54 INFO Executor: Running task 196.0 in stage 12.0 (TID 286) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 805306368-848432523, partition values: [empty row] +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 805306368-848521656, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 805306368-848489219, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 805306368-848442878, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 805306368-848423659, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO Executor: Finished task 193.0 in stage 12.0 (TID 283). 2251 bytes result sent to driver +26/04/01 05:56:54 INFO TaskSetManager: Starting task 197.0 in stage 12.0 (TID 287) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:54 INFO TaskSetManager: Finished task 193.0 in stage 12.0 (TID 283) in 469 ms on 10.0.0.133 (executor driver) (194/208) +26/04/01 05:56:54 INFO Executor: Running task 197.0 in stage 12.0 (TID 287) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 805306368-848422348, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO Executor: Finished task 194.0 in stage 12.0 (TID 284). 2251 bytes result sent to driver +26/04/01 05:56:54 INFO TaskSetManager: Starting task 198.0 in stage 12.0 (TID 288) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:54 INFO TaskSetManager: Finished task 194.0 in stage 12.0 (TID 284) in 484 ms on 10.0.0.133 (executor driver) (195/208) +26/04/01 05:56:54 INFO Executor: Running task 198.0 in stage 12.0 (TID 288) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 805306368-848399844, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO Executor: Finished task 195.0 in stage 12.0 (TID 285). 2251 bytes result sent to driver +26/04/01 05:56:54 INFO TaskSetManager: Starting task 199.0 in stage 12.0 (TID 289) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:54 INFO TaskSetManager: Finished task 195.0 in stage 12.0 (TID 285) in 481 ms on 10.0.0.133 (executor driver) (196/208) +26/04/01 05:56:54 INFO Executor: Running task 199.0 in stage 12.0 (TID 289) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 805306368-848378208, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 805306368-848411809, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO Executor: Finished task 196.0 in stage 12.0 (TID 286). 2251 bytes result sent to driver +26/04/01 05:56:54 INFO TaskSetManager: Starting task 200.0 in stage 12.0 (TID 290) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9708 bytes) +26/04/01 05:56:54 INFO TaskSetManager: Finished task 196.0 in stage 12.0 (TID 286) in 478 ms on 10.0.0.133 (executor driver) (197/208) +26/04/01 05:56:54 INFO Executor: Running task 200.0 in stage 12.0 (TID 290) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 805306368-848344114, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 805306368-848379799, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 805306368-848348606, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 805306368-848338051, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO Executor: Finished task 197.0 in stage 12.0 (TID 287). 2251 bytes result sent to driver +26/04/01 05:56:54 INFO TaskSetManager: Starting task 201.0 in stage 12.0 (TID 291) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:54 INFO TaskSetManager: Finished task 197.0 in stage 12.0 (TID 287) in 440 ms on 10.0.0.133 (executor driver) (198/208) +26/04/01 05:56:54 INFO Executor: Running task 201.0 in stage 12.0 (TID 291) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 805306368-848329366, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO Executor: Finished task 198.0 in stage 12.0 (TID 288). 2251 bytes result sent to driver +26/04/01 05:56:54 INFO TaskSetManager: Starting task 202.0 in stage 12.0 (TID 292) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:54 INFO TaskSetManager: Finished task 198.0 in stage 12.0 (TID 288) in 441 ms on 10.0.0.133 (executor driver) (199/208) +26/04/01 05:56:54 INFO Executor: Running task 202.0 in stage 12.0 (TID 292) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 805306368-848328570, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:54 INFO Executor: Finished task 199.0 in stage 12.0 (TID 289). 2251 bytes result sent to driver +26/04/01 05:56:54 INFO TaskSetManager: Starting task 203.0 in stage 12.0 (TID 293) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:54 INFO TaskSetManager: Finished task 199.0 in stage 12.0 (TID 289) in 446 ms on 10.0.0.133 (executor driver) (200/208) +26/04/01 05:56:54 INFO Executor: Running task 203.0 in stage 12.0 (TID 293) +26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 805306368-848301737, partition values: [empty row] +26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 805306368-848329327, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:55 INFO Executor: Finished task 200.0 in stage 12.0 (TID 290). 2251 bytes result sent to driver +26/04/01 05:56:55 INFO TaskSetManager: Starting task 204.0 in stage 12.0 (TID 294) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:55 INFO TaskSetManager: Finished task 200.0 in stage 12.0 (TID 290) in 441 ms on 10.0.0.133 (executor driver) (201/208) +26/04/01 05:56:55 INFO Executor: Running task 204.0 in stage 12.0 (TID 294) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 805306368-848263904, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 805306368-848306153, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 805306368-848278425, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 805306368-848247851, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:55 INFO Executor: Finished task 201.0 in stage 12.0 (TID 291). 2251 bytes result sent to driver +26/04/01 05:56:55 INFO TaskSetManager: Starting task 205.0 in stage 12.0 (TID 295) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9710 bytes) +26/04/01 05:56:55 INFO TaskSetManager: Finished task 201.0 in stage 12.0 (TID 291) in 439 ms on 10.0.0.133 (executor driver) (202/208) +26/04/01 05:56:55 INFO Executor: Running task 205.0 in stage 12.0 (TID 295) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 805306368-848235021, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:55 INFO Executor: Finished task 202.0 in stage 12.0 (TID 292). 2251 bytes result sent to driver +26/04/01 05:56:55 INFO TaskSetManager: Starting task 206.0 in stage 12.0 (TID 296) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:55 INFO TaskSetManager: Finished task 202.0 in stage 12.0 (TID 292) in 447 ms on 10.0.0.133 (executor driver) (203/208) +26/04/01 05:56:55 INFO Executor: Running task 206.0 in stage 12.0 (TID 296) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 805306368-848107384, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:55 INFO Executor: Finished task 203.0 in stage 12.0 (TID 293). 2251 bytes result sent to driver +26/04/01 05:56:55 INFO TaskSetManager: Starting task 207.0 in stage 12.0 (TID 297) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9709 bytes) +26/04/01 05:56:55 INFO TaskSetManager: Finished task 203.0 in stage 12.0 (TID 293) in 459 ms on 10.0.0.133 (executor driver) (204/208) +26/04/01 05:56:55 INFO Executor: Running task 207.0 in stage 12.0 (TID 297) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 805306368-848005421, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 805306368-848234776, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:55 INFO Executor: Finished task 204.0 in stage 12.0 (TID 294). 2251 bytes result sent to driver +26/04/01 05:56:55 INFO TaskSetManager: Starting task 0.0 in stage 13.0 (TID 298) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 10355 bytes) +26/04/01 05:56:55 INFO Executor: Running task 0.0 in stage 13.0 (TID 298) +26/04/01 05:56:55 INFO TaskSetManager: Finished task 204.0 in stage 12.0 (TID 294) in 468 ms on 10.0.0.133 (executor driver) (205/208) +26/04/01 05:56:55 INFO CodeGenerator: Code generated in 3.538958 ms +26/04/01 05:56:55 INFO CodeGenerator: Code generated in 2.368917 ms +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.9.parquet, range: 0-2801305, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.21.parquet, range: 0-2800748, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.13.parquet, range: 0-2800604, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.1.parquet, range: 0-2799999, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.31.parquet, range: 0-2799730, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.30.parquet, range: 0-2799509, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.25.parquet, range: 0-2798499, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.6.parquet, range: 0-2798414, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 805306368-848050794, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:55 INFO Executor: Finished task 0.0 in stage 13.0 (TID 298). 2208 bytes result sent to driver +26/04/01 05:56:55 INFO TaskSetManager: Starting task 1.0 in stage 13.0 (TID 299) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 10357 bytes) +26/04/01 05:56:55 INFO TaskSetManager: Finished task 0.0 in stage 13.0 (TID 298) in 88 ms on 10.0.0.133 (executor driver) (1/4) +26/04/01 05:56:55 INFO Executor: Running task 1.0 in stage 13.0 (TID 299) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.11.parquet, range: 0-2798396, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.18.parquet, range: 0-2797632, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.2.parquet, range: 0-2797502, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.12.parquet, range: 0-2797316, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.26.parquet, range: 0-2797291, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.14.parquet, range: 0-2797103, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.22.parquet, range: 0-2797014, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.17.parquet, range: 0-2795731, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 805306368-847699715, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) +26/04/01 05:56:55 INFO Executor: Finished task 1.0 in stage 13.0 (TID 299). 2208 bytes result sent to driver +26/04/01 05:56:55 INFO TaskSetManager: Starting task 2.0 in stage 13.0 (TID 300) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 10357 bytes) +26/04/01 05:56:55 INFO TaskSetManager: Finished task 1.0 in stage 13.0 (TID 299) in 89 ms on 10.0.0.133 (executor driver) (2/4) +26/04/01 05:56:55 INFO Executor: Running task 2.0 in stage 13.0 (TID 300) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.27.parquet, range: 0-2795177, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.10.parquet, range: 0-2795129, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO Executor: Finished task 205.0 in stage 12.0 (TID 295). 2251 bytes result sent to driver +26/04/01 05:56:55 INFO TaskSetManager: Starting task 3.0 in stage 13.0 (TID 301) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 10354 bytes) +26/04/01 05:56:55 INFO TaskSetManager: Finished task 205.0 in stage 12.0 (TID 295) in 456 ms on 10.0.0.133 (executor driver) (206/208) +26/04/01 05:56:55 INFO Executor: Running task 3.0 in stage 13.0 (TID 301) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.32.parquet, range: 0-2792572, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.19.parquet, range: 0-2794808, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.15.parquet, range: 0-2794714, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.3.parquet, range: 0-2794406, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.16.parquet, range: 0-2794298, partition values: [empty row] +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.5.parquet, range: 0-2792421, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.24.parquet, range: 0-2793929, partition values: [empty row] +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.7.parquet, range: 0-2791953, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.23.parquet, range: 0-2792937, partition values: [empty row] +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.29.parquet, range: 0-2791555, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.20.parquet, range: 0-2791117, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.28.parquet, range: 0-2790847, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.8.parquet, range: 0-2790608, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.4.parquet, range: 0-2789514, partition values: [empty row] +26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 05:56:55 INFO Executor: Finished task 2.0 in stage 13.0 (TID 300). 2251 bytes result sent to driver +26/04/01 05:56:55 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 302) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8999 bytes) +26/04/01 05:56:55 INFO Executor: Running task 0.0 in stage 15.0 (TID 302) +26/04/01 05:56:55 INFO TaskSetManager: Finished task 2.0 in stage 13.0 (TID 300) in 93 ms on 10.0.0.133 (executor driver) (3/4) +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 1 (5.2 KiB) non-empty blocks including 1 (5.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms +26/04/01 05:56:55 INFO Executor: Finished task 3.0 in stage 13.0 (TID 301). 2251 bytes result sent to driver +26/04/01 05:56:55 INFO TaskSetManager: Starting task 0.0 in stage 18.0 (TID 303) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9270 bytes) +26/04/01 05:56:55 INFO TaskSetManager: Finished task 3.0 in stage 13.0 (TID 301) in 87 ms on 10.0.0.133 (executor driver) (4/4) +26/04/01 05:56:55 INFO Executor: Running task 0.0 in stage 18.0 (TID 303) +26/04/01 05:56:55 INFO TaskSchedulerImpl: Removed TaskSet 13.0, whose tasks have all completed, from pool +26/04/01 05:56:55 INFO DAGScheduler: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 42.814 s +26/04/01 05:56:55 INFO DAGScheduler: looking for newly runnable stages +26/04/01 05:56:55 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 12, ShuffleMapStage 18) +26/04/01 05:56:55 INFO DAGScheduler: waiting: Set() +26/04/01 05:56:55 INFO DAGScheduler: failed: Set() +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 16 (20.5 MiB) non-empty blocks including 16 (20.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:55 INFO Executor: Finished task 0.0 in stage 15.0 (TID 302). 10188 bytes result sent to driver +26/04/01 05:56:55 INFO TaskSetManager: Starting task 1.0 in stage 18.0 (TID 304) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9270 bytes) +26/04/01 05:56:55 INFO Executor: Running task 1.0 in stage 18.0 (TID 304) +26/04/01 05:56:55 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 302) in 26 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:56:55 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool +26/04/01 05:56:55 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 41.361 s +26/04/01 05:56:55 INFO DAGScheduler: Job 14 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:56:55 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished +26/04/01 05:56:55 INFO DAGScheduler: Job 14 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 41.364141 s +26/04/01 05:56:55 INFO CodeGenerator: Code generated in 2.945959 ms +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 16 (20.5 MiB) non-empty blocks including 16 (20.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:55 INFO CodeGenerator: Code generated in 3.59775 ms +26/04/01 05:56:55 INFO CodeGenerator: Code generated in 3.850333 ms +26/04/01 05:56:55 INFO MemoryStore: Block broadcast_23 stored as values in memory (estimated size 201.9 KiB, free 8.6 GiB) +26/04/01 05:56:55 INFO CodeGenerator: Code generated in 1.692125 ms +26/04/01 05:56:55 INFO MemoryStore: Block broadcast_23_piece0 stored as bytes in memory (estimated size 35.2 KiB, free 8.6 GiB) +26/04/01 05:56:55 INFO BlockManagerInfo: Added broadcast_23_piece0 in memory on 10.0.0.133:58206 (size: 35.2 KiB, free: 8.6 GiB) +26/04/01 05:56:55 INFO SparkContext: Created broadcast 23 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 64 (42.8 MiB) non-empty blocks including 64 (42.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms +26/04/01 05:56:55 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 05:56:55 INFO CodeGenerator: Code generated in 2.804208 ms +26/04/01 05:56:55 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:56:55 INFO DAGScheduler: Got job 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 05:56:55 INFO DAGScheduler: Final stage: ResultStage 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:56:55 INFO DAGScheduler: Parents of final stage: List() +26/04/01 05:56:55 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:55 INFO DAGScheduler: Submitting ResultStage 19 (MapPartitionsRDD[54] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:56:55 INFO CodeGenerator: Code generated in 1.525541 ms +26/04/01 05:56:55 WARN DAGScheduler: Broadcasting large task binary with size 1045.3 KiB +26/04/01 05:56:55 INFO MemoryStore: Block broadcast_24 stored as values in memory (estimated size 1045.3 KiB, free 8.6 GiB) +26/04/01 05:56:55 INFO MemoryStore: Block broadcast_24_piece0 stored as bytes in memory (estimated size 12.7 KiB, free 8.6 GiB) +26/04/01 05:56:55 INFO BlockManagerInfo: Added broadcast_24_piece0 in memory on 10.0.0.133:58206 (size: 12.7 KiB, free: 8.6 GiB) +26/04/01 05:56:55 INFO SparkContext: Created broadcast 24 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:55 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 19 (MapPartitionsRDD[54] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:56:55 INFO TaskSchedulerImpl: Adding task set 19.0 with 1 tasks resource profile 0 +26/04/01 05:56:55 INFO CodeGenerator: Code generated in 1.738541 ms +26/04/01 05:56:55 INFO CodeGenerator: Code generated in 3.485708 ms +26/04/01 05:56:55 INFO Executor: Finished task 206.0 in stage 12.0 (TID 296). 2251 bytes result sent to driver +26/04/01 05:56:55 INFO TaskSetManager: Starting task 2.0 in stage 18.0 (TID 305) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9270 bytes) +26/04/01 05:56:55 INFO TaskSetManager: Finished task 206.0 in stage 12.0 (TID 296) in 476 ms on 10.0.0.133 (executor driver) (207/208) +26/04/01 05:56:55 INFO Executor: Running task 2.0 in stage 18.0 (TID 305) +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 16 (20.5 MiB) non-empty blocks including 16 (20.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:55 INFO Executor: Finished task 207.0 in stage 12.0 (TID 297). 2251 bytes result sent to driver +26/04/01 05:56:55 INFO TaskSetManager: Starting task 3.0 in stage 18.0 (TID 306) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9270 bytes) +26/04/01 05:56:55 INFO TaskSetManager: Finished task 207.0 in stage 12.0 (TID 297) in 463 ms on 10.0.0.133 (executor driver) (208/208) +26/04/01 05:56:55 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool +26/04/01 05:56:55 INFO Executor: Running task 3.0 in stage 18.0 (TID 306) +26/04/01 05:56:55 INFO DAGScheduler: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 42.935 s +26/04/01 05:56:55 INFO DAGScheduler: looking for newly runnable stages +26/04/01 05:56:55 INFO DAGScheduler: running: Set(ResultStage 19, ShuffleMapStage 18) +26/04/01 05:56:55 INFO DAGScheduler: waiting: Set() +26/04/01 05:56:55 INFO DAGScheduler: failed: Set() +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 16 (20.5 MiB) non-empty blocks including 16 (20.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:57 INFO Executor: Finished task 2.0 in stage 18.0 (TID 305). 7309 bytes result sent to driver +26/04/01 05:56:57 INFO TaskSetManager: Starting task 4.0 in stage 18.0 (TID 307) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9270 bytes) +26/04/01 05:56:57 INFO Executor: Running task 4.0 in stage 18.0 (TID 307) +26/04/01 05:56:57 INFO TaskSetManager: Finished task 2.0 in stage 18.0 (TID 305) in 1804 ms on 10.0.0.133 (executor driver) (1/7) +26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Getting 16 (20.5 MiB) non-empty blocks including 16 (20.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:57 INFO Executor: Finished task 3.0 in stage 18.0 (TID 306). 7309 bytes result sent to driver +26/04/01 05:56:57 INFO TaskSetManager: Starting task 5.0 in stage 18.0 (TID 308) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9270 bytes) +26/04/01 05:56:57 INFO TaskSetManager: Finished task 3.0 in stage 18.0 (TID 306) in 1746 ms on 10.0.0.133 (executor driver) (2/7) +26/04/01 05:56:57 INFO Executor: Running task 5.0 in stage 18.0 (TID 308) +26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Getting 16 (20.5 MiB) non-empty blocks including 16 (20.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:57 INFO Executor: Finished task 0.0 in stage 18.0 (TID 303). 7309 bytes result sent to driver +26/04/01 05:56:57 INFO TaskSetManager: Starting task 6.0 in stage 18.0 (TID 309) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9270 bytes) +26/04/01 05:56:57 INFO TaskSetManager: Finished task 0.0 in stage 18.0 (TID 303) in 1862 ms on 10.0.0.133 (executor driver) (3/7) +26/04/01 05:56:57 INFO Executor: Running task 6.0 in stage 18.0 (TID 309) +26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Getting 16 (9.2 MiB) non-empty blocks including 16 (9.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Getting 64 (19.3 MiB) non-empty blocks including 64 (19.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:57 INFO Executor: Finished task 1.0 in stage 18.0 (TID 304). 7266 bytes result sent to driver +26/04/01 05:56:57 INFO TaskSetManager: Starting task 0.0 in stage 19.0 (TID 310) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9608 bytes) +26/04/01 05:56:57 INFO TaskSetManager: Finished task 1.0 in stage 18.0 (TID 304) in 1860 ms on 10.0.0.133 (executor driver) (4/7) +26/04/01 05:56:57 INFO Executor: Running task 0.0 in stage 19.0 (TID 310) +26/04/01 05:56:57 INFO CodeGenerator: Code generated in 3.867541 ms +26/04/01 05:56:57 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/nation/nation.1.parquet, range: 0-2670, partition values: [empty row] +26/04/01 05:56:57 INFO FilterCompat: Filtering using predicate: and(noteq(n_nationkey, null), noteq(n_regionkey, null)) +26/04/01 05:56:57 INFO Executor: Finished task 0.0 in stage 19.0 (TID 310). 1921 bytes result sent to driver +26/04/01 05:56:57 INFO TaskSetManager: Finished task 0.0 in stage 19.0 (TID 310) in 9 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:56:57 INFO TaskSchedulerImpl: Removed TaskSet 19.0, whose tasks have all completed, from pool +26/04/01 05:56:57 INFO DAGScheduler: ResultStage 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 1.844 s +26/04/01 05:56:57 INFO DAGScheduler: Job 16 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:56:57 INFO TaskSchedulerImpl: Killing all running tasks in stage 19: Stage finished +26/04/01 05:56:57 INFO DAGScheduler: Job 16 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 1.844655 s +26/04/01 05:56:57 INFO CodeGenerator: Code generated in 1.84025 ms +26/04/01 05:56:57 INFO MemoryStore: Block broadcast_25 stored as values in memory (estimated size 1024.2 KiB, free 8.5 GiB) +26/04/01 05:56:57 INFO MemoryStore: Block broadcast_25_piece0 stored as bytes in memory (estimated size 326.0 B, free 8.5 GiB) +26/04/01 05:56:57 INFO BlockManagerInfo: Added broadcast_25_piece0 in memory on 10.0.0.133:58206 (size: 326.0 B, free: 8.6 GiB) +26/04/01 05:56:57 INFO SparkContext: Created broadcast 25 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:56:58 INFO BlockManagerInfo: Removed broadcast_24_piece0 on 10.0.0.133:58206 in memory (size: 12.7 KiB, free: 8.6 GiB) +26/04/01 05:56:58 INFO Executor: Finished task 6.0 in stage 18.0 (TID 309). 7266 bytes result sent to driver +26/04/01 05:56:58 INFO TaskSetManager: Finished task 6.0 in stage 18.0 (TID 309) in 687 ms on 10.0.0.133 (executor driver) (5/7) +26/04/01 05:56:59 INFO Executor: Finished task 4.0 in stage 18.0 (TID 307). 7309 bytes result sent to driver +26/04/01 05:56:59 INFO TaskSetManager: Finished task 4.0 in stage 18.0 (TID 307) in 1510 ms on 10.0.0.133 (executor driver) (6/7) +26/04/01 05:56:59 INFO Executor: Finished task 5.0 in stage 18.0 (TID 308). 7266 bytes result sent to driver +26/04/01 05:56:59 INFO TaskSetManager: Finished task 5.0 in stage 18.0 (TID 308) in 1509 ms on 10.0.0.133 (executor driver) (7/7) +26/04/01 05:56:59 INFO TaskSchedulerImpl: Removed TaskSet 18.0, whose tasks have all completed, from pool +26/04/01 05:56:59 INFO DAGScheduler: ShuffleMapStage 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 40.936 s +26/04/01 05:56:59 INFO DAGScheduler: looking for newly runnable stages +26/04/01 05:56:59 INFO DAGScheduler: running: Set() +26/04/01 05:56:59 INFO DAGScheduler: waiting: Set() +26/04/01 05:56:59 INFO DAGScheduler: failed: Set() +26/04/01 05:56:59 INFO ShufflePartitionsUtil: For shuffle(5, 3), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 05:56:59 INFO CodeGenerator: Code generated in 3.7285 ms +26/04/01 05:56:59 INFO CodeGenerator: Code generated in 1.978708 ms +26/04/01 05:56:59 INFO CodeGenerator: Code generated in 2.039917 ms +26/04/01 05:56:59 INFO DAGScheduler: Registering RDD 61 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 6 +26/04/01 05:56:59 INFO DAGScheduler: Got map stage job 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions +26/04/01 05:56:59 INFO DAGScheduler: Final stage: ShuffleMapStage 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:56:59 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 22, ShuffleMapStage 23) +26/04/01 05:56:59 INFO DAGScheduler: Missing parents: List() +26/04/01 05:56:59 INFO DAGScheduler: Submitting ShuffleMapStage 24 (MapPartitionsRDD[61] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:56:59 INFO MemoryStore: Block broadcast_26 stored as values in memory (estimated size 64.1 KiB, free 8.6 GiB) +26/04/01 05:56:59 INFO MemoryStore: Block broadcast_26_piece0 stored as bytes in memory (estimated size 29.5 KiB, free 8.6 GiB) +26/04/01 05:56:59 INFO BlockManagerInfo: Added broadcast_26_piece0 in memory on 10.0.0.133:58206 (size: 29.5 KiB, free: 8.6 GiB) +26/04/01 05:56:59 INFO SparkContext: Created broadcast 26 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:56:59 INFO DAGScheduler: Submitting 200 missing tasks from ShuffleMapStage 24 (MapPartitionsRDD[61] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 05:56:59 INFO TaskSchedulerImpl: Adding task set 24.0 with 200 tasks resource profile 0 +26/04/01 05:56:59 INFO TaskSetManager: Starting task 0.0 in stage 24.0 (TID 311) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9270 bytes) +26/04/01 05:56:59 INFO TaskSetManager: Starting task 1.0 in stage 24.0 (TID 312) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9270 bytes) +26/04/01 05:56:59 INFO TaskSetManager: Starting task 2.0 in stage 24.0 (TID 313) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9270 bytes) +26/04/01 05:56:59 INFO TaskSetManager: Starting task 3.0 in stage 24.0 (TID 314) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9270 bytes) +26/04/01 05:56:59 INFO Executor: Running task 0.0 in stage 24.0 (TID 311) +26/04/01 05:56:59 INFO Executor: Running task 2.0 in stage 24.0 (TID 313) +26/04/01 05:56:59 INFO Executor: Running task 1.0 in stage 24.0 (TID 312) +26/04/01 05:56:59 INFO Executor: Running task 3.0 in stage 24.0 (TID 314) +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:59 INFO CodeGenerator: Code generated in 2.124834 ms +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:59 INFO CodeGenerator: Code generated in 2.212 ms +26/04/01 05:56:59 INFO CodeGenerator: Code generated in 3.180792 ms +26/04/01 05:56:59 INFO CodeGenerator: Code generated in 2.126 ms +26/04/01 05:56:59 INFO Executor: Finished task 2.0 in stage 24.0 (TID 313). 10893 bytes result sent to driver +26/04/01 05:56:59 INFO TaskSetManager: Starting task 4.0 in stage 24.0 (TID 315) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9270 bytes) +26/04/01 05:56:59 INFO Executor: Running task 4.0 in stage 24.0 (TID 315) +26/04/01 05:56:59 INFO TaskSetManager: Finished task 2.0 in stage 24.0 (TID 313) in 569 ms on 10.0.0.133 (executor driver) (1/200) +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:59 INFO Executor: Finished task 3.0 in stage 24.0 (TID 314). 10893 bytes result sent to driver +26/04/01 05:56:59 INFO TaskSetManager: Starting task 5.0 in stage 24.0 (TID 316) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9270 bytes) +26/04/01 05:56:59 INFO TaskSetManager: Finished task 3.0 in stage 24.0 (TID 314) in 583 ms on 10.0.0.133 (executor driver) (2/200) +26/04/01 05:56:59 INFO Executor: Running task 5.0 in stage 24.0 (TID 316) +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:59 INFO Executor: Finished task 1.0 in stage 24.0 (TID 312). 10893 bytes result sent to driver +26/04/01 05:56:59 INFO TaskSetManager: Starting task 6.0 in stage 24.0 (TID 317) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9270 bytes) +26/04/01 05:56:59 INFO TaskSetManager: Finished task 1.0 in stage 24.0 (TID 312) in 599 ms on 10.0.0.133 (executor driver) (3/200) +26/04/01 05:56:59 INFO Executor: Running task 6.0 in stage 24.0 (TID 317) +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:59 INFO Executor: Finished task 0.0 in stage 24.0 (TID 311). 10893 bytes result sent to driver +26/04/01 05:56:59 INFO TaskSetManager: Starting task 7.0 in stage 24.0 (TID 318) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9270 bytes) +26/04/01 05:56:59 INFO TaskSetManager: Finished task 0.0 in stage 24.0 (TID 311) in 604 ms on 10.0.0.133 (executor driver) (4/200) +26/04/01 05:56:59 INFO Executor: Running task 7.0 in stage 24.0 (TID 318) +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO Executor: Finished task 4.0 in stage 24.0 (TID 315). 10893 bytes result sent to driver +26/04/01 05:57:00 INFO TaskSetManager: Starting task 8.0 in stage 24.0 (TID 319) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:00 INFO Executor: Running task 8.0 in stage 24.0 (TID 319) +26/04/01 05:57:00 INFO TaskSetManager: Finished task 4.0 in stage 24.0 (TID 315) in 522 ms on 10.0.0.133 (executor driver) (5/200) +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO Executor: Finished task 5.0 in stage 24.0 (TID 316). 10850 bytes result sent to driver +26/04/01 05:57:00 INFO TaskSetManager: Starting task 9.0 in stage 24.0 (TID 320) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:00 INFO TaskSetManager: Finished task 5.0 in stage 24.0 (TID 316) in 519 ms on 10.0.0.133 (executor driver) (6/200) +26/04/01 05:57:00 INFO Executor: Running task 9.0 in stage 24.0 (TID 320) +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO Executor: Finished task 6.0 in stage 24.0 (TID 317). 10850 bytes result sent to driver +26/04/01 05:57:00 INFO TaskSetManager: Starting task 10.0 in stage 24.0 (TID 321) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:00 INFO Executor: Running task 10.0 in stage 24.0 (TID 321) +26/04/01 05:57:00 INFO TaskSetManager: Finished task 6.0 in stage 24.0 (TID 317) in 519 ms on 10.0.0.133 (executor driver) (7/200) +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO Executor: Finished task 7.0 in stage 24.0 (TID 318). 10850 bytes result sent to driver +26/04/01 05:57:00 INFO TaskSetManager: Starting task 11.0 in stage 24.0 (TID 322) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:00 INFO Executor: Running task 11.0 in stage 24.0 (TID 322) +26/04/01 05:57:00 INFO TaskSetManager: Finished task 7.0 in stage 24.0 (TID 318) in 517 ms on 10.0.0.133 (executor driver) (8/200) +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO Executor: Finished task 8.0 in stage 24.0 (TID 319). 10850 bytes result sent to driver +26/04/01 05:57:00 INFO TaskSetManager: Starting task 12.0 in stage 24.0 (TID 323) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:00 INFO Executor: Running task 12.0 in stage 24.0 (TID 323) +26/04/01 05:57:00 INFO TaskSetManager: Finished task 8.0 in stage 24.0 (TID 319) in 511 ms on 10.0.0.133 (executor driver) (9/200) +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO Executor: Finished task 9.0 in stage 24.0 (TID 320). 10850 bytes result sent to driver +26/04/01 05:57:00 INFO TaskSetManager: Starting task 13.0 in stage 24.0 (TID 324) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:00 INFO Executor: Running task 13.0 in stage 24.0 (TID 324) +26/04/01 05:57:00 INFO TaskSetManager: Finished task 9.0 in stage 24.0 (TID 320) in 505 ms on 10.0.0.133 (executor driver) (10/200) +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO Executor: Finished task 10.0 in stage 24.0 (TID 321). 10807 bytes result sent to driver +26/04/01 05:57:00 INFO TaskSetManager: Starting task 14.0 in stage 24.0 (TID 325) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:00 INFO Executor: Running task 14.0 in stage 24.0 (TID 325) +26/04/01 05:57:00 INFO TaskSetManager: Finished task 10.0 in stage 24.0 (TID 321) in 501 ms on 10.0.0.133 (executor driver) (11/200) +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO Executor: Finished task 11.0 in stage 24.0 (TID 322). 10807 bytes result sent to driver +26/04/01 05:57:00 INFO TaskSetManager: Starting task 15.0 in stage 24.0 (TID 326) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:00 INFO TaskSetManager: Finished task 11.0 in stage 24.0 (TID 322) in 500 ms on 10.0.0.133 (executor driver) (12/200) +26/04/01 05:57:00 INFO Executor: Running task 15.0 in stage 24.0 (TID 326) +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO Executor: Finished task 12.0 in stage 24.0 (TID 323). 10893 bytes result sent to driver +26/04/01 05:57:01 INFO TaskSetManager: Starting task 16.0 in stage 24.0 (TID 327) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:01 INFO Executor: Running task 16.0 in stage 24.0 (TID 327) +26/04/01 05:57:01 INFO TaskSetManager: Finished task 12.0 in stage 24.0 (TID 323) in 508 ms on 10.0.0.133 (executor driver) (13/200) +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO Executor: Finished task 13.0 in stage 24.0 (TID 324). 10850 bytes result sent to driver +26/04/01 05:57:01 INFO TaskSetManager: Starting task 17.0 in stage 24.0 (TID 328) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:01 INFO TaskSetManager: Finished task 13.0 in stage 24.0 (TID 324) in 512 ms on 10.0.0.133 (executor driver) (14/200) +26/04/01 05:57:01 INFO Executor: Running task 17.0 in stage 24.0 (TID 328) +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO Executor: Finished task 14.0 in stage 24.0 (TID 325). 10893 bytes result sent to driver +26/04/01 05:57:01 INFO TaskSetManager: Starting task 18.0 in stage 24.0 (TID 329) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:01 INFO Executor: Running task 18.0 in stage 24.0 (TID 329) +26/04/01 05:57:01 INFO TaskSetManager: Finished task 14.0 in stage 24.0 (TID 325) in 509 ms on 10.0.0.133 (executor driver) (15/200) +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO Executor: Finished task 15.0 in stage 24.0 (TID 326). 10893 bytes result sent to driver +26/04/01 05:57:01 INFO TaskSetManager: Starting task 19.0 in stage 24.0 (TID 330) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:01 INFO TaskSetManager: Finished task 15.0 in stage 24.0 (TID 326) in 509 ms on 10.0.0.133 (executor driver) (16/200) +26/04/01 05:57:01 INFO Executor: Running task 19.0 in stage 24.0 (TID 330) +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO Executor: Finished task 16.0 in stage 24.0 (TID 327). 10893 bytes result sent to driver +26/04/01 05:57:01 INFO TaskSetManager: Starting task 20.0 in stage 24.0 (TID 331) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:01 INFO Executor: Running task 20.0 in stage 24.0 (TID 331) +26/04/01 05:57:01 INFO TaskSetManager: Finished task 16.0 in stage 24.0 (TID 327) in 498 ms on 10.0.0.133 (executor driver) (17/200) +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO Executor: Finished task 17.0 in stage 24.0 (TID 328). 10850 bytes result sent to driver +26/04/01 05:57:01 INFO TaskSetManager: Starting task 21.0 in stage 24.0 (TID 332) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:01 INFO TaskSetManager: Finished task 17.0 in stage 24.0 (TID 328) in 512 ms on 10.0.0.133 (executor driver) (18/200) +26/04/01 05:57:01 INFO Executor: Running task 21.0 in stage 24.0 (TID 332) +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1229.1 KiB) non-empty blocks including 7 (1229.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO Executor: Finished task 18.0 in stage 24.0 (TID 329). 10893 bytes result sent to driver +26/04/01 05:57:01 INFO TaskSetManager: Starting task 22.0 in stage 24.0 (TID 333) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:01 INFO TaskSetManager: Finished task 18.0 in stage 24.0 (TID 329) in 518 ms on 10.0.0.133 (executor driver) (19/200) +26/04/01 05:57:01 INFO Executor: Running task 22.0 in stage 24.0 (TID 333) +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO Executor: Finished task 19.0 in stage 24.0 (TID 330). 10850 bytes result sent to driver +26/04/01 05:57:01 INFO TaskSetManager: Starting task 23.0 in stage 24.0 (TID 334) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:01 INFO TaskSetManager: Finished task 19.0 in stage 24.0 (TID 330) in 519 ms on 10.0.0.133 (executor driver) (20/200) +26/04/01 05:57:01 INFO Executor: Running task 23.0 in stage 24.0 (TID 334) +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO Executor: Finished task 20.0 in stage 24.0 (TID 331). 10850 bytes result sent to driver +26/04/01 05:57:02 INFO TaskSetManager: Starting task 24.0 in stage 24.0 (TID 335) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:02 INFO Executor: Running task 24.0 in stage 24.0 (TID 335) +26/04/01 05:57:02 INFO TaskSetManager: Finished task 20.0 in stage 24.0 (TID 331) in 505 ms on 10.0.0.133 (executor driver) (21/200) +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO Executor: Finished task 21.0 in stage 24.0 (TID 332). 10850 bytes result sent to driver +26/04/01 05:57:02 INFO TaskSetManager: Starting task 25.0 in stage 24.0 (TID 336) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:02 INFO TaskSetManager: Finished task 21.0 in stage 24.0 (TID 332) in 512 ms on 10.0.0.133 (executor driver) (22/200) +26/04/01 05:57:02 INFO Executor: Running task 25.0 in stage 24.0 (TID 336) +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO Executor: Finished task 22.0 in stage 24.0 (TID 333). 10807 bytes result sent to driver +26/04/01 05:57:02 INFO TaskSetManager: Starting task 26.0 in stage 24.0 (TID 337) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:02 INFO TaskSetManager: Finished task 22.0 in stage 24.0 (TID 333) in 506 ms on 10.0.0.133 (executor driver) (23/200) +26/04/01 05:57:02 INFO Executor: Running task 26.0 in stage 24.0 (TID 337) +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO Executor: Finished task 23.0 in stage 24.0 (TID 334). 10850 bytes result sent to driver +26/04/01 05:57:02 INFO TaskSetManager: Starting task 27.0 in stage 24.0 (TID 338) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:02 INFO Executor: Running task 27.0 in stage 24.0 (TID 338) +26/04/01 05:57:02 INFO TaskSetManager: Finished task 23.0 in stage 24.0 (TID 334) in 506 ms on 10.0.0.133 (executor driver) (24/200) +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO Executor: Finished task 24.0 in stage 24.0 (TID 335). 10893 bytes result sent to driver +26/04/01 05:57:02 INFO TaskSetManager: Starting task 28.0 in stage 24.0 (TID 339) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:02 INFO Executor: Running task 28.0 in stage 24.0 (TID 339) +26/04/01 05:57:02 INFO TaskSetManager: Finished task 24.0 in stage 24.0 (TID 335) in 497 ms on 10.0.0.133 (executor driver) (25/200) +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO Executor: Finished task 25.0 in stage 24.0 (TID 336). 10893 bytes result sent to driver +26/04/01 05:57:02 INFO TaskSetManager: Starting task 29.0 in stage 24.0 (TID 340) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:02 INFO Executor: Running task 29.0 in stage 24.0 (TID 340) +26/04/01 05:57:02 INFO TaskSetManager: Finished task 25.0 in stage 24.0 (TID 336) in 517 ms on 10.0.0.133 (executor driver) (26/200) +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO Executor: Finished task 27.0 in stage 24.0 (TID 338). 10893 bytes result sent to driver +26/04/01 05:57:02 INFO TaskSetManager: Starting task 30.0 in stage 24.0 (TID 341) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:02 INFO Executor: Running task 30.0 in stage 24.0 (TID 341) +26/04/01 05:57:02 INFO TaskSetManager: Finished task 27.0 in stage 24.0 (TID 338) in 513 ms on 10.0.0.133 (executor driver) (27/200) +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO Executor: Finished task 26.0 in stage 24.0 (TID 337). 10850 bytes result sent to driver +26/04/01 05:57:02 INFO TaskSetManager: Starting task 31.0 in stage 24.0 (TID 342) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:02 INFO Executor: Running task 31.0 in stage 24.0 (TID 342) +26/04/01 05:57:02 INFO TaskSetManager: Finished task 26.0 in stage 24.0 (TID 337) in 523 ms on 10.0.0.133 (executor driver) (28/200) +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO Executor: Finished task 28.0 in stage 24.0 (TID 339). 10893 bytes result sent to driver +26/04/01 05:57:03 INFO TaskSetManager: Starting task 32.0 in stage 24.0 (TID 343) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:03 INFO Executor: Running task 32.0 in stage 24.0 (TID 343) +26/04/01 05:57:03 INFO TaskSetManager: Finished task 28.0 in stage 24.0 (TID 339) in 490 ms on 10.0.0.133 (executor driver) (29/200) +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO Executor: Finished task 29.0 in stage 24.0 (TID 340). 10893 bytes result sent to driver +26/04/01 05:57:03 INFO TaskSetManager: Starting task 33.0 in stage 24.0 (TID 344) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:03 INFO TaskSetManager: Finished task 29.0 in stage 24.0 (TID 340) in 507 ms on 10.0.0.133 (executor driver) (30/200) +26/04/01 05:57:03 INFO Executor: Running task 33.0 in stage 24.0 (TID 344) +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO Executor: Finished task 30.0 in stage 24.0 (TID 341). 10850 bytes result sent to driver +26/04/01 05:57:03 INFO TaskSetManager: Starting task 34.0 in stage 24.0 (TID 345) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:03 INFO Executor: Running task 34.0 in stage 24.0 (TID 345) +26/04/01 05:57:03 INFO TaskSetManager: Finished task 30.0 in stage 24.0 (TID 341) in 506 ms on 10.0.0.133 (executor driver) (31/200) +26/04/01 05:57:03 INFO Executor: Finished task 31.0 in stage 24.0 (TID 342). 10850 bytes result sent to driver +26/04/01 05:57:03 INFO TaskSetManager: Starting task 35.0 in stage 24.0 (TID 346) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:03 INFO Executor: Running task 35.0 in stage 24.0 (TID 346) +26/04/01 05:57:03 INFO TaskSetManager: Finished task 31.0 in stage 24.0 (TID 342) in 498 ms on 10.0.0.133 (executor driver) (32/200) +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO Executor: Finished task 32.0 in stage 24.0 (TID 343). 10850 bytes result sent to driver +26/04/01 05:57:03 INFO TaskSetManager: Starting task 36.0 in stage 24.0 (TID 347) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:03 INFO TaskSetManager: Finished task 32.0 in stage 24.0 (TID 343) in 484 ms on 10.0.0.133 (executor driver) (33/200) +26/04/01 05:57:03 INFO Executor: Running task 36.0 in stage 24.0 (TID 347) +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO Executor: Finished task 33.0 in stage 24.0 (TID 344). 10850 bytes result sent to driver +26/04/01 05:57:03 INFO Executor: Finished task 34.0 in stage 24.0 (TID 345). 10850 bytes result sent to driver +26/04/01 05:57:03 INFO TaskSetManager: Starting task 37.0 in stage 24.0 (TID 348) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:03 INFO Executor: Running task 37.0 in stage 24.0 (TID 348) +26/04/01 05:57:03 INFO TaskSetManager: Starting task 38.0 in stage 24.0 (TID 349) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:03 INFO Executor: Running task 38.0 in stage 24.0 (TID 349) +26/04/01 05:57:03 INFO TaskSetManager: Finished task 33.0 in stage 24.0 (TID 344) in 507 ms on 10.0.0.133 (executor driver) (34/200) +26/04/01 05:57:03 INFO TaskSetManager: Finished task 34.0 in stage 24.0 (TID 345) in 503 ms on 10.0.0.133 (executor driver) (35/200) +26/04/01 05:57:03 INFO Executor: Finished task 35.0 in stage 24.0 (TID 346). 10850 bytes result sent to driver +26/04/01 05:57:03 INFO TaskSetManager: Starting task 39.0 in stage 24.0 (TID 350) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:03 INFO TaskSetManager: Finished task 35.0 in stage 24.0 (TID 346) in 503 ms on 10.0.0.133 (executor driver) (36/200) +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO Executor: Running task 39.0 in stage 24.0 (TID 350) +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO Executor: Finished task 36.0 in stage 24.0 (TID 347). 10893 bytes result sent to driver +26/04/01 05:57:04 INFO TaskSetManager: Starting task 40.0 in stage 24.0 (TID 351) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:04 INFO Executor: Running task 40.0 in stage 24.0 (TID 351) +26/04/01 05:57:04 INFO TaskSetManager: Finished task 36.0 in stage 24.0 (TID 347) in 519 ms on 10.0.0.133 (executor driver) (37/200) +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO Executor: Finished task 39.0 in stage 24.0 (TID 350). 10850 bytes result sent to driver +26/04/01 05:57:04 INFO TaskSetManager: Starting task 41.0 in stage 24.0 (TID 352) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:04 INFO Executor: Running task 41.0 in stage 24.0 (TID 352) +26/04/01 05:57:04 INFO TaskSetManager: Finished task 39.0 in stage 24.0 (TID 350) in 509 ms on 10.0.0.133 (executor driver) (38/200) +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO Executor: Finished task 37.0 in stage 24.0 (TID 348). 10893 bytes result sent to driver +26/04/01 05:57:04 INFO TaskSetManager: Starting task 42.0 in stage 24.0 (TID 353) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:04 INFO Executor: Running task 42.0 in stage 24.0 (TID 353) +26/04/01 05:57:04 INFO TaskSetManager: Finished task 37.0 in stage 24.0 (TID 348) in 513 ms on 10.0.0.133 (executor driver) (39/200) +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO Executor: Finished task 38.0 in stage 24.0 (TID 349). 10893 bytes result sent to driver +26/04/01 05:57:04 INFO TaskSetManager: Starting task 43.0 in stage 24.0 (TID 354) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:04 INFO TaskSetManager: Finished task 38.0 in stage 24.0 (TID 349) in 515 ms on 10.0.0.133 (executor driver) (40/200) +26/04/01 05:57:04 INFO Executor: Running task 43.0 in stage 24.0 (TID 354) +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO Executor: Finished task 40.0 in stage 24.0 (TID 351). 10893 bytes result sent to driver +26/04/01 05:57:04 INFO TaskSetManager: Starting task 44.0 in stage 24.0 (TID 355) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:04 INFO TaskSetManager: Finished task 40.0 in stage 24.0 (TID 351) in 482 ms on 10.0.0.133 (executor driver) (41/200) +26/04/01 05:57:04 INFO Executor: Running task 44.0 in stage 24.0 (TID 355) +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO Executor: Finished task 43.0 in stage 24.0 (TID 354). 10850 bytes result sent to driver +26/04/01 05:57:04 INFO TaskSetManager: Starting task 45.0 in stage 24.0 (TID 356) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:04 INFO Executor: Running task 45.0 in stage 24.0 (TID 356) +26/04/01 05:57:04 INFO TaskSetManager: Finished task 43.0 in stage 24.0 (TID 354) in 504 ms on 10.0.0.133 (executor driver) (42/200) +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO Executor: Finished task 41.0 in stage 24.0 (TID 352). 10893 bytes result sent to driver +26/04/01 05:57:04 INFO TaskSetManager: Starting task 46.0 in stage 24.0 (TID 357) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:04 INFO Executor: Running task 46.0 in stage 24.0 (TID 357) +26/04/01 05:57:04 INFO TaskSetManager: Finished task 41.0 in stage 24.0 (TID 352) in 519 ms on 10.0.0.133 (executor driver) (43/200) +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO Executor: Finished task 42.0 in stage 24.0 (TID 353). 10850 bytes result sent to driver +26/04/01 05:57:04 INFO TaskSetManager: Starting task 47.0 in stage 24.0 (TID 358) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:04 INFO TaskSetManager: Finished task 42.0 in stage 24.0 (TID 353) in 518 ms on 10.0.0.133 (executor driver) (44/200) +26/04/01 05:57:04 INFO Executor: Running task 47.0 in stage 24.0 (TID 358) +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO Executor: Finished task 44.0 in stage 24.0 (TID 355). 10850 bytes result sent to driver +26/04/01 05:57:05 INFO TaskSetManager: Starting task 48.0 in stage 24.0 (TID 359) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:05 INFO TaskSetManager: Finished task 44.0 in stage 24.0 (TID 355) in 480 ms on 10.0.0.133 (executor driver) (45/200) +26/04/01 05:57:05 INFO Executor: Running task 48.0 in stage 24.0 (TID 359) +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO Executor: Finished task 45.0 in stage 24.0 (TID 356). 10850 bytes result sent to driver +26/04/01 05:57:05 INFO TaskSetManager: Starting task 49.0 in stage 24.0 (TID 360) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:05 INFO TaskSetManager: Finished task 45.0 in stage 24.0 (TID 356) in 493 ms on 10.0.0.133 (executor driver) (46/200) +26/04/01 05:57:05 INFO Executor: Running task 49.0 in stage 24.0 (TID 360) +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO Executor: Finished task 46.0 in stage 24.0 (TID 357). 10807 bytes result sent to driver +26/04/01 05:57:05 INFO TaskSetManager: Starting task 50.0 in stage 24.0 (TID 361) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:05 INFO Executor: Running task 50.0 in stage 24.0 (TID 361) +26/04/01 05:57:05 INFO TaskSetManager: Finished task 46.0 in stage 24.0 (TID 357) in 497 ms on 10.0.0.133 (executor driver) (47/200) +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO Executor: Finished task 47.0 in stage 24.0 (TID 358). 10807 bytes result sent to driver +26/04/01 05:57:05 INFO TaskSetManager: Starting task 51.0 in stage 24.0 (TID 362) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:05 INFO Executor: Running task 51.0 in stage 24.0 (TID 362) +26/04/01 05:57:05 INFO TaskSetManager: Finished task 47.0 in stage 24.0 (TID 358) in 507 ms on 10.0.0.133 (executor driver) (48/200) +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO Executor: Finished task 48.0 in stage 24.0 (TID 359). 10893 bytes result sent to driver +26/04/01 05:57:05 INFO TaskSetManager: Starting task 52.0 in stage 24.0 (TID 363) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:05 INFO TaskSetManager: Finished task 48.0 in stage 24.0 (TID 359) in 482 ms on 10.0.0.133 (executor driver) (49/200) +26/04/01 05:57:05 INFO Executor: Running task 52.0 in stage 24.0 (TID 363) +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO Executor: Finished task 49.0 in stage 24.0 (TID 360). 10893 bytes result sent to driver +26/04/01 05:57:05 INFO TaskSetManager: Starting task 53.0 in stage 24.0 (TID 364) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:05 INFO Executor: Running task 53.0 in stage 24.0 (TID 364) +26/04/01 05:57:05 INFO TaskSetManager: Finished task 49.0 in stage 24.0 (TID 360) in 493 ms on 10.0.0.133 (executor driver) (50/200) +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO Executor: Finished task 50.0 in stage 24.0 (TID 361). 10850 bytes result sent to driver +26/04/01 05:57:05 INFO TaskSetManager: Starting task 54.0 in stage 24.0 (TID 365) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:05 INFO Executor: Running task 54.0 in stage 24.0 (TID 365) +26/04/01 05:57:05 INFO TaskSetManager: Finished task 50.0 in stage 24.0 (TID 361) in 496 ms on 10.0.0.133 (executor driver) (51/200) +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO Executor: Finished task 51.0 in stage 24.0 (TID 362). 10850 bytes result sent to driver +26/04/01 05:57:05 INFO TaskSetManager: Starting task 55.0 in stage 24.0 (TID 366) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:05 INFO TaskSetManager: Finished task 51.0 in stage 24.0 (TID 362) in 494 ms on 10.0.0.133 (executor driver) (52/200) +26/04/01 05:57:05 INFO Executor: Running task 55.0 in stage 24.0 (TID 366) +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO Executor: Finished task 52.0 in stage 24.0 (TID 363). 10850 bytes result sent to driver +26/04/01 05:57:06 INFO TaskSetManager: Starting task 56.0 in stage 24.0 (TID 367) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:06 INFO TaskSetManager: Finished task 52.0 in stage 24.0 (TID 363) in 478 ms on 10.0.0.133 (executor driver) (53/200) +26/04/01 05:57:06 INFO Executor: Running task 56.0 in stage 24.0 (TID 367) +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO Executor: Finished task 53.0 in stage 24.0 (TID 364). 10893 bytes result sent to driver +26/04/01 05:57:06 INFO TaskSetManager: Starting task 57.0 in stage 24.0 (TID 368) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:06 INFO Executor: Running task 57.0 in stage 24.0 (TID 368) +26/04/01 05:57:06 INFO TaskSetManager: Finished task 53.0 in stage 24.0 (TID 364) in 497 ms on 10.0.0.133 (executor driver) (54/200) +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO Executor: Finished task 54.0 in stage 24.0 (TID 365). 10850 bytes result sent to driver +26/04/01 05:57:06 INFO TaskSetManager: Starting task 58.0 in stage 24.0 (TID 369) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:06 INFO Executor: Running task 58.0 in stage 24.0 (TID 369) +26/04/01 05:57:06 INFO TaskSetManager: Finished task 54.0 in stage 24.0 (TID 365) in 496 ms on 10.0.0.133 (executor driver) (55/200) +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO Executor: Finished task 55.0 in stage 24.0 (TID 366). 10850 bytes result sent to driver +26/04/01 05:57:06 INFO TaskSetManager: Starting task 59.0 in stage 24.0 (TID 370) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:06 INFO TaskSetManager: Finished task 55.0 in stage 24.0 (TID 366) in 496 ms on 10.0.0.133 (executor driver) (56/200) +26/04/01 05:57:06 INFO Executor: Running task 59.0 in stage 24.0 (TID 370) +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO Executor: Finished task 56.0 in stage 24.0 (TID 367). 10850 bytes result sent to driver +26/04/01 05:57:06 INFO TaskSetManager: Starting task 60.0 in stage 24.0 (TID 371) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:06 INFO TaskSetManager: Finished task 56.0 in stage 24.0 (TID 367) in 486 ms on 10.0.0.133 (executor driver) (57/200) +26/04/01 05:57:06 INFO Executor: Running task 60.0 in stage 24.0 (TID 371) +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO Executor: Finished task 57.0 in stage 24.0 (TID 368). 10850 bytes result sent to driver +26/04/01 05:57:06 INFO TaskSetManager: Starting task 61.0 in stage 24.0 (TID 372) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:06 INFO TaskSetManager: Finished task 57.0 in stage 24.0 (TID 368) in 496 ms on 10.0.0.133 (executor driver) (58/200) +26/04/01 05:57:06 INFO Executor: Running task 61.0 in stage 24.0 (TID 372) +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO Executor: Finished task 58.0 in stage 24.0 (TID 369). 10807 bytes result sent to driver +26/04/01 05:57:06 INFO TaskSetManager: Starting task 62.0 in stage 24.0 (TID 373) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:06 INFO TaskSetManager: Finished task 58.0 in stage 24.0 (TID 369) in 496 ms on 10.0.0.133 (executor driver) (59/200) +26/04/01 05:57:06 INFO Executor: Running task 62.0 in stage 24.0 (TID 373) +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO Executor: Finished task 59.0 in stage 24.0 (TID 370). 10807 bytes result sent to driver +26/04/01 05:57:06 INFO TaskSetManager: Starting task 63.0 in stage 24.0 (TID 374) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:06 INFO Executor: Running task 63.0 in stage 24.0 (TID 374) +26/04/01 05:57:06 INFO TaskSetManager: Finished task 59.0 in stage 24.0 (TID 370) in 495 ms on 10.0.0.133 (executor driver) (60/200) +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO Executor: Finished task 60.0 in stage 24.0 (TID 371). 10893 bytes result sent to driver +26/04/01 05:57:07 INFO TaskSetManager: Starting task 64.0 in stage 24.0 (TID 375) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:07 INFO Executor: Running task 64.0 in stage 24.0 (TID 375) +26/04/01 05:57:07 INFO TaskSetManager: Finished task 60.0 in stage 24.0 (TID 371) in 497 ms on 10.0.0.133 (executor driver) (61/200) +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO Executor: Finished task 61.0 in stage 24.0 (TID 372). 10893 bytes result sent to driver +26/04/01 05:57:07 INFO TaskSetManager: Starting task 65.0 in stage 24.0 (TID 376) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:07 INFO Executor: Running task 65.0 in stage 24.0 (TID 376) +26/04/01 05:57:07 INFO TaskSetManager: Finished task 61.0 in stage 24.0 (TID 372) in 504 ms on 10.0.0.133 (executor driver) (62/200) +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO Executor: Finished task 63.0 in stage 24.0 (TID 374). 10850 bytes result sent to driver +26/04/01 05:57:07 INFO TaskSetManager: Starting task 66.0 in stage 24.0 (TID 377) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:07 INFO Executor: Running task 66.0 in stage 24.0 (TID 377) +26/04/01 05:57:07 INFO Executor: Finished task 62.0 in stage 24.0 (TID 373). 10893 bytes result sent to driver +26/04/01 05:57:07 INFO TaskSetManager: Finished task 63.0 in stage 24.0 (TID 374) in 498 ms on 10.0.0.133 (executor driver) (63/200) +26/04/01 05:57:07 INFO TaskSetManager: Starting task 67.0 in stage 24.0 (TID 378) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:07 INFO Executor: Running task 67.0 in stage 24.0 (TID 378) +26/04/01 05:57:07 INFO TaskSetManager: Finished task 62.0 in stage 24.0 (TID 373) in 506 ms on 10.0.0.133 (executor driver) (64/200) +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO Executor: Finished task 64.0 in stage 24.0 (TID 375). 10850 bytes result sent to driver +26/04/01 05:57:07 INFO TaskSetManager: Starting task 68.0 in stage 24.0 (TID 379) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:07 INFO TaskSetManager: Finished task 64.0 in stage 24.0 (TID 375) in 488 ms on 10.0.0.133 (executor driver) (65/200) +26/04/01 05:57:07 INFO Executor: Running task 68.0 in stage 24.0 (TID 379) +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO Executor: Finished task 65.0 in stage 24.0 (TID 376). 10893 bytes result sent to driver +26/04/01 05:57:07 INFO TaskSetManager: Starting task 69.0 in stage 24.0 (TID 380) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:07 INFO TaskSetManager: Finished task 65.0 in stage 24.0 (TID 376) in 495 ms on 10.0.0.133 (executor driver) (66/200) +26/04/01 05:57:07 INFO Executor: Running task 69.0 in stage 24.0 (TID 380) +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO Executor: Finished task 67.0 in stage 24.0 (TID 378). 10850 bytes result sent to driver +26/04/01 05:57:07 INFO Executor: Finished task 66.0 in stage 24.0 (TID 377). 10850 bytes result sent to driver +26/04/01 05:57:07 INFO TaskSetManager: Starting task 70.0 in stage 24.0 (TID 381) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:07 INFO Executor: Running task 70.0 in stage 24.0 (TID 381) +26/04/01 05:57:07 INFO TaskSetManager: Starting task 71.0 in stage 24.0 (TID 382) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:07 INFO Executor: Running task 71.0 in stage 24.0 (TID 382) +26/04/01 05:57:07 INFO TaskSetManager: Finished task 67.0 in stage 24.0 (TID 378) in 505 ms on 10.0.0.133 (executor driver) (67/200) +26/04/01 05:57:07 INFO TaskSetManager: Finished task 66.0 in stage 24.0 (TID 377) in 505 ms on 10.0.0.133 (executor driver) (68/200) +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1229.1 KiB) non-empty blocks including 7 (1229.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO Executor: Finished task 68.0 in stage 24.0 (TID 379). 10893 bytes result sent to driver +26/04/01 05:57:08 INFO TaskSetManager: Starting task 72.0 in stage 24.0 (TID 383) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:08 INFO TaskSetManager: Finished task 68.0 in stage 24.0 (TID 379) in 488 ms on 10.0.0.133 (executor driver) (69/200) +26/04/01 05:57:08 INFO Executor: Running task 72.0 in stage 24.0 (TID 383) +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO Executor: Finished task 69.0 in stage 24.0 (TID 380). 10850 bytes result sent to driver +26/04/01 05:57:08 INFO TaskSetManager: Starting task 73.0 in stage 24.0 (TID 384) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:08 INFO TaskSetManager: Finished task 69.0 in stage 24.0 (TID 380) in 480 ms on 10.0.0.133 (executor driver) (70/200) +26/04/01 05:57:08 INFO Executor: Running task 73.0 in stage 24.0 (TID 384) +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO Executor: Finished task 70.0 in stage 24.0 (TID 381). 10850 bytes result sent to driver +26/04/01 05:57:08 INFO TaskSetManager: Starting task 74.0 in stage 24.0 (TID 385) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:08 INFO TaskSetManager: Finished task 70.0 in stage 24.0 (TID 381) in 492 ms on 10.0.0.133 (executor driver) (71/200) +26/04/01 05:57:08 INFO Executor: Running task 74.0 in stage 24.0 (TID 385) +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO Executor: Finished task 71.0 in stage 24.0 (TID 382). 10807 bytes result sent to driver +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO TaskSetManager: Starting task 75.0 in stage 24.0 (TID 386) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:08 INFO Executor: Running task 75.0 in stage 24.0 (TID 386) +26/04/01 05:57:08 INFO TaskSetManager: Finished task 71.0 in stage 24.0 (TID 382) in 495 ms on 10.0.0.133 (executor driver) (72/200) +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO Executor: Finished task 72.0 in stage 24.0 (TID 383). 10850 bytes result sent to driver +26/04/01 05:57:08 INFO TaskSetManager: Starting task 76.0 in stage 24.0 (TID 387) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:08 INFO Executor: Running task 76.0 in stage 24.0 (TID 387) +26/04/01 05:57:08 INFO TaskSetManager: Finished task 72.0 in stage 24.0 (TID 383) in 492 ms on 10.0.0.133 (executor driver) (73/200) +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO Executor: Finished task 73.0 in stage 24.0 (TID 384). 10893 bytes result sent to driver +26/04/01 05:57:08 INFO TaskSetManager: Starting task 77.0 in stage 24.0 (TID 388) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:08 INFO Executor: Running task 77.0 in stage 24.0 (TID 388) +26/04/01 05:57:08 INFO TaskSetManager: Finished task 73.0 in stage 24.0 (TID 384) in 492 ms on 10.0.0.133 (executor driver) (74/200) +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO Executor: Finished task 74.0 in stage 24.0 (TID 385). 10893 bytes result sent to driver +26/04/01 05:57:08 INFO TaskSetManager: Starting task 78.0 in stage 24.0 (TID 389) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:08 INFO TaskSetManager: Finished task 74.0 in stage 24.0 (TID 385) in 501 ms on 10.0.0.133 (executor driver) (75/200) +26/04/01 05:57:08 INFO Executor: Running task 78.0 in stage 24.0 (TID 389) +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO Executor: Finished task 75.0 in stage 24.0 (TID 386). 10850 bytes result sent to driver +26/04/01 05:57:08 INFO TaskSetManager: Starting task 79.0 in stage 24.0 (TID 390) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:08 INFO Executor: Running task 79.0 in stage 24.0 (TID 390) +26/04/01 05:57:08 INFO TaskSetManager: Finished task 75.0 in stage 24.0 (TID 386) in 505 ms on 10.0.0.133 (executor driver) (76/200) +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1229.1 KiB) non-empty blocks including 7 (1229.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO Executor: Finished task 76.0 in stage 24.0 (TID 387). 10893 bytes result sent to driver +26/04/01 05:57:09 INFO TaskSetManager: Starting task 80.0 in stage 24.0 (TID 391) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:09 INFO Executor: Running task 80.0 in stage 24.0 (TID 391) +26/04/01 05:57:09 INFO TaskSetManager: Finished task 76.0 in stage 24.0 (TID 387) in 491 ms on 10.0.0.133 (executor driver) (77/200) +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO Executor: Finished task 77.0 in stage 24.0 (TID 388). 10893 bytes result sent to driver +26/04/01 05:57:09 INFO TaskSetManager: Starting task 81.0 in stage 24.0 (TID 392) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:09 INFO Executor: Running task 81.0 in stage 24.0 (TID 392) +26/04/01 05:57:09 INFO TaskSetManager: Finished task 77.0 in stage 24.0 (TID 388) in 501 ms on 10.0.0.133 (executor driver) (78/200) +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO Executor: Finished task 78.0 in stage 24.0 (TID 389). 10893 bytes result sent to driver +26/04/01 05:57:09 INFO TaskSetManager: Starting task 82.0 in stage 24.0 (TID 393) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:09 INFO Executor: Running task 82.0 in stage 24.0 (TID 393) +26/04/01 05:57:09 INFO TaskSetManager: Finished task 78.0 in stage 24.0 (TID 389) in 504 ms on 10.0.0.133 (executor driver) (79/200) +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO Executor: Finished task 79.0 in stage 24.0 (TID 390). 10850 bytes result sent to driver +26/04/01 05:57:09 INFO TaskSetManager: Starting task 83.0 in stage 24.0 (TID 394) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:09 INFO Executor: Running task 83.0 in stage 24.0 (TID 394) +26/04/01 05:57:09 INFO TaskSetManager: Finished task 79.0 in stage 24.0 (TID 390) in 502 ms on 10.0.0.133 (executor driver) (80/200) +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO Executor: Finished task 80.0 in stage 24.0 (TID 391). 10893 bytes result sent to driver +26/04/01 05:57:09 INFO TaskSetManager: Starting task 84.0 in stage 24.0 (TID 395) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:09 INFO Executor: Running task 84.0 in stage 24.0 (TID 395) +26/04/01 05:57:09 INFO TaskSetManager: Finished task 80.0 in stage 24.0 (TID 391) in 484 ms on 10.0.0.133 (executor driver) (81/200) +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO Executor: Finished task 81.0 in stage 24.0 (TID 392). 10850 bytes result sent to driver +26/04/01 05:57:09 INFO TaskSetManager: Starting task 85.0 in stage 24.0 (TID 396) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:09 INFO Executor: Running task 85.0 in stage 24.0 (TID 396) +26/04/01 05:57:09 INFO TaskSetManager: Finished task 81.0 in stage 24.0 (TID 392) in 492 ms on 10.0.0.133 (executor driver) (82/200) +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO Executor: Finished task 82.0 in stage 24.0 (TID 393). 10850 bytes result sent to driver +26/04/01 05:57:09 INFO TaskSetManager: Starting task 86.0 in stage 24.0 (TID 397) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:09 INFO TaskSetManager: Finished task 82.0 in stage 24.0 (TID 393) in 491 ms on 10.0.0.133 (executor driver) (83/200) +26/04/01 05:57:09 INFO Executor: Running task 86.0 in stage 24.0 (TID 397) +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO Executor: Finished task 83.0 in stage 24.0 (TID 394). 10807 bytes result sent to driver +26/04/01 05:57:09 INFO TaskSetManager: Starting task 87.0 in stage 24.0 (TID 398) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:09 INFO Executor: Running task 87.0 in stage 24.0 (TID 398) +26/04/01 05:57:09 INFO TaskSetManager: Finished task 83.0 in stage 24.0 (TID 394) in 492 ms on 10.0.0.133 (executor driver) (84/200) +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO Executor: Finished task 84.0 in stage 24.0 (TID 395). 10850 bytes result sent to driver +26/04/01 05:57:10 INFO TaskSetManager: Starting task 88.0 in stage 24.0 (TID 399) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:10 INFO Executor: Running task 88.0 in stage 24.0 (TID 399) +26/04/01 05:57:10 INFO TaskSetManager: Finished task 84.0 in stage 24.0 (TID 395) in 479 ms on 10.0.0.133 (executor driver) (85/200) +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO Executor: Finished task 85.0 in stage 24.0 (TID 396). 10893 bytes result sent to driver +26/04/01 05:57:10 INFO TaskSetManager: Starting task 89.0 in stage 24.0 (TID 400) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:10 INFO Executor: Running task 89.0 in stage 24.0 (TID 400) +26/04/01 05:57:10 INFO TaskSetManager: Finished task 85.0 in stage 24.0 (TID 396) in 490 ms on 10.0.0.133 (executor driver) (86/200) +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO Executor: Finished task 86.0 in stage 24.0 (TID 397). 10893 bytes result sent to driver +26/04/01 05:57:10 INFO TaskSetManager: Starting task 90.0 in stage 24.0 (TID 401) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:10 INFO TaskSetManager: Finished task 86.0 in stage 24.0 (TID 397) in 498 ms on 10.0.0.133 (executor driver) (87/200) +26/04/01 05:57:10 INFO Executor: Running task 90.0 in stage 24.0 (TID 401) +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO Executor: Finished task 87.0 in stage 24.0 (TID 398). 10850 bytes result sent to driver +26/04/01 05:57:10 INFO TaskSetManager: Starting task 91.0 in stage 24.0 (TID 402) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:10 INFO Executor: Running task 91.0 in stage 24.0 (TID 402) +26/04/01 05:57:10 INFO TaskSetManager: Finished task 87.0 in stage 24.0 (TID 398) in 494 ms on 10.0.0.133 (executor driver) (88/200) +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO Executor: Finished task 88.0 in stage 24.0 (TID 399). 10850 bytes result sent to driver +26/04/01 05:57:10 INFO TaskSetManager: Starting task 92.0 in stage 24.0 (TID 403) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:10 INFO Executor: Running task 92.0 in stage 24.0 (TID 403) +26/04/01 05:57:10 INFO TaskSetManager: Finished task 88.0 in stage 24.0 (TID 399) in 486 ms on 10.0.0.133 (executor driver) (89/200) +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO Executor: Finished task 89.0 in stage 24.0 (TID 400). 10850 bytes result sent to driver +26/04/01 05:57:10 INFO TaskSetManager: Starting task 93.0 in stage 24.0 (TID 404) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:10 INFO Executor: Running task 93.0 in stage 24.0 (TID 404) +26/04/01 05:57:10 INFO TaskSetManager: Finished task 89.0 in stage 24.0 (TID 400) in 486 ms on 10.0.0.133 (executor driver) (90/200) +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO Executor: Finished task 90.0 in stage 24.0 (TID 401). 10850 bytes result sent to driver +26/04/01 05:57:10 INFO TaskSetManager: Starting task 94.0 in stage 24.0 (TID 405) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:10 INFO Executor: Running task 94.0 in stage 24.0 (TID 405) +26/04/01 05:57:10 INFO TaskSetManager: Finished task 90.0 in stage 24.0 (TID 401) in 494 ms on 10.0.0.133 (executor driver) (91/200) +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO Executor: Finished task 91.0 in stage 24.0 (TID 402). 10807 bytes result sent to driver +26/04/01 05:57:10 INFO TaskSetManager: Starting task 95.0 in stage 24.0 (TID 406) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:10 INFO TaskSetManager: Finished task 91.0 in stage 24.0 (TID 402) in 497 ms on 10.0.0.133 (executor driver) (92/200) +26/04/01 05:57:10 INFO Executor: Running task 95.0 in stage 24.0 (TID 406) +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO Executor: Finished task 92.0 in stage 24.0 (TID 403). 10893 bytes result sent to driver +26/04/01 05:57:11 INFO TaskSetManager: Starting task 96.0 in stage 24.0 (TID 407) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:11 INFO Executor: Running task 96.0 in stage 24.0 (TID 407) +26/04/01 05:57:11 INFO TaskSetManager: Finished task 92.0 in stage 24.0 (TID 403) in 486 ms on 10.0.0.133 (executor driver) (93/200) +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO Executor: Finished task 93.0 in stage 24.0 (TID 404). 10893 bytes result sent to driver +26/04/01 05:57:11 INFO TaskSetManager: Starting task 97.0 in stage 24.0 (TID 408) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:11 INFO Executor: Running task 97.0 in stage 24.0 (TID 408) +26/04/01 05:57:11 INFO TaskSetManager: Finished task 93.0 in stage 24.0 (TID 404) in 520 ms on 10.0.0.133 (executor driver) (94/200) +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO Executor: Finished task 94.0 in stage 24.0 (TID 405). 10893 bytes result sent to driver +26/04/01 05:57:11 INFO TaskSetManager: Starting task 98.0 in stage 24.0 (TID 409) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:11 INFO Executor: Running task 98.0 in stage 24.0 (TID 409) +26/04/01 05:57:11 INFO TaskSetManager: Finished task 94.0 in stage 24.0 (TID 405) in 500 ms on 10.0.0.133 (executor driver) (95/200) +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO Executor: Finished task 95.0 in stage 24.0 (TID 406). 10893 bytes result sent to driver +26/04/01 05:57:11 INFO TaskSetManager: Starting task 99.0 in stage 24.0 (TID 410) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:11 INFO Executor: Running task 99.0 in stage 24.0 (TID 410) +26/04/01 05:57:11 INFO TaskSetManager: Finished task 95.0 in stage 24.0 (TID 406) in 494 ms on 10.0.0.133 (executor driver) (96/200) +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO Executor: Finished task 96.0 in stage 24.0 (TID 407). 10850 bytes result sent to driver +26/04/01 05:57:11 INFO TaskSetManager: Starting task 100.0 in stage 24.0 (TID 411) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:11 INFO TaskSetManager: Finished task 96.0 in stage 24.0 (TID 407) in 484 ms on 10.0.0.133 (executor driver) (97/200) +26/04/01 05:57:11 INFO Executor: Running task 100.0 in stage 24.0 (TID 411) +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO Executor: Finished task 97.0 in stage 24.0 (TID 408). 10893 bytes result sent to driver +26/04/01 05:57:11 INFO TaskSetManager: Starting task 101.0 in stage 24.0 (TID 412) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:11 INFO TaskSetManager: Finished task 97.0 in stage 24.0 (TID 408) in 490 ms on 10.0.0.133 (executor driver) (98/200) +26/04/01 05:57:11 INFO Executor: Running task 101.0 in stage 24.0 (TID 412) +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO Executor: Finished task 98.0 in stage 24.0 (TID 409). 10893 bytes result sent to driver +26/04/01 05:57:11 INFO TaskSetManager: Starting task 102.0 in stage 24.0 (TID 413) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:11 INFO Executor: Running task 102.0 in stage 24.0 (TID 413) +26/04/01 05:57:11 INFO TaskSetManager: Finished task 98.0 in stage 24.0 (TID 409) in 500 ms on 10.0.0.133 (executor driver) (99/200) +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO Executor: Finished task 99.0 in stage 24.0 (TID 410). 10850 bytes result sent to driver +26/04/01 05:57:11 INFO TaskSetManager: Starting task 103.0 in stage 24.0 (TID 414) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:11 INFO Executor: Running task 103.0 in stage 24.0 (TID 414) +26/04/01 05:57:11 INFO TaskSetManager: Finished task 99.0 in stage 24.0 (TID 410) in 502 ms on 10.0.0.133 (executor driver) (100/200) +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO Executor: Finished task 100.0 in stage 24.0 (TID 411). 10893 bytes result sent to driver +26/04/01 05:57:12 INFO TaskSetManager: Starting task 104.0 in stage 24.0 (TID 415) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:12 INFO Executor: Running task 104.0 in stage 24.0 (TID 415) +26/04/01 05:57:12 INFO TaskSetManager: Finished task 100.0 in stage 24.0 (TID 411) in 488 ms on 10.0.0.133 (executor driver) (101/200) +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO Executor: Finished task 101.0 in stage 24.0 (TID 412). 10850 bytes result sent to driver +26/04/01 05:57:12 INFO TaskSetManager: Starting task 105.0 in stage 24.0 (TID 416) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:12 INFO TaskSetManager: Finished task 101.0 in stage 24.0 (TID 412) in 484 ms on 10.0.0.133 (executor driver) (102/200) +26/04/01 05:57:12 INFO Executor: Running task 105.0 in stage 24.0 (TID 416) +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO Executor: Finished task 103.0 in stage 24.0 (TID 414). 10807 bytes result sent to driver +26/04/01 05:57:12 INFO TaskSetManager: Starting task 106.0 in stage 24.0 (TID 417) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:12 INFO Executor: Running task 106.0 in stage 24.0 (TID 417) +26/04/01 05:57:12 INFO TaskSetManager: Finished task 103.0 in stage 24.0 (TID 414) in 494 ms on 10.0.0.133 (executor driver) (103/200) +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO Executor: Finished task 102.0 in stage 24.0 (TID 413). 10850 bytes result sent to driver +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO TaskSetManager: Starting task 107.0 in stage 24.0 (TID 418) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:12 INFO TaskSetManager: Finished task 102.0 in stage 24.0 (TID 413) in 499 ms on 10.0.0.133 (executor driver) (104/200) +26/04/01 05:57:12 INFO Executor: Running task 107.0 in stage 24.0 (TID 418) +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO Executor: Finished task 104.0 in stage 24.0 (TID 415). 10893 bytes result sent to driver +26/04/01 05:57:12 INFO TaskSetManager: Starting task 108.0 in stage 24.0 (TID 419) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:12 INFO Executor: Running task 108.0 in stage 24.0 (TID 419) +26/04/01 05:57:12 INFO TaskSetManager: Finished task 104.0 in stage 24.0 (TID 415) in 486 ms on 10.0.0.133 (executor driver) (105/200) +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO Executor: Finished task 105.0 in stage 24.0 (TID 416). 10893 bytes result sent to driver +26/04/01 05:57:12 INFO TaskSetManager: Starting task 109.0 in stage 24.0 (TID 420) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:12 INFO TaskSetManager: Finished task 105.0 in stage 24.0 (TID 416) in 486 ms on 10.0.0.133 (executor driver) (106/200) +26/04/01 05:57:12 INFO Executor: Running task 109.0 in stage 24.0 (TID 420) +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO Executor: Finished task 107.0 in stage 24.0 (TID 418). 10850 bytes result sent to driver +26/04/01 05:57:12 INFO TaskSetManager: Starting task 110.0 in stage 24.0 (TID 421) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:12 INFO Executor: Running task 110.0 in stage 24.0 (TID 421) +26/04/01 05:57:12 INFO TaskSetManager: Finished task 107.0 in stage 24.0 (TID 418) in 497 ms on 10.0.0.133 (executor driver) (107/200) +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO Executor: Finished task 106.0 in stage 24.0 (TID 417). 10850 bytes result sent to driver +26/04/01 05:57:12 INFO TaskSetManager: Starting task 111.0 in stage 24.0 (TID 422) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:12 INFO TaskSetManager: Finished task 106.0 in stage 24.0 (TID 417) in 506 ms on 10.0.0.133 (executor driver) (108/200) +26/04/01 05:57:12 INFO Executor: Running task 111.0 in stage 24.0 (TID 422) +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO Executor: Finished task 108.0 in stage 24.0 (TID 419). 10850 bytes result sent to driver +26/04/01 05:57:13 INFO TaskSetManager: Starting task 112.0 in stage 24.0 (TID 423) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:13 INFO TaskSetManager: Finished task 108.0 in stage 24.0 (TID 419) in 484 ms on 10.0.0.133 (executor driver) (109/200) +26/04/01 05:57:13 INFO Executor: Running task 112.0 in stage 24.0 (TID 423) +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO Executor: Finished task 109.0 in stage 24.0 (TID 420). 10893 bytes result sent to driver +26/04/01 05:57:13 INFO TaskSetManager: Starting task 113.0 in stage 24.0 (TID 424) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:13 INFO TaskSetManager: Finished task 109.0 in stage 24.0 (TID 420) in 488 ms on 10.0.0.133 (executor driver) (110/200) +26/04/01 05:57:13 INFO Executor: Running task 113.0 in stage 24.0 (TID 424) +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO Executor: Finished task 110.0 in stage 24.0 (TID 421). 10893 bytes result sent to driver +26/04/01 05:57:13 INFO TaskSetManager: Starting task 114.0 in stage 24.0 (TID 425) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:13 INFO TaskSetManager: Finished task 110.0 in stage 24.0 (TID 421) in 496 ms on 10.0.0.133 (executor driver) (111/200) +26/04/01 05:57:13 INFO Executor: Running task 114.0 in stage 24.0 (TID 425) +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO Executor: Finished task 111.0 in stage 24.0 (TID 422). 10850 bytes result sent to driver +26/04/01 05:57:13 INFO TaskSetManager: Starting task 115.0 in stage 24.0 (TID 426) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:13 INFO TaskSetManager: Finished task 111.0 in stage 24.0 (TID 422) in 494 ms on 10.0.0.133 (executor driver) (112/200) +26/04/01 05:57:13 INFO Executor: Running task 115.0 in stage 24.0 (TID 426) +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO Executor: Finished task 112.0 in stage 24.0 (TID 423). 10893 bytes result sent to driver +26/04/01 05:57:13 INFO TaskSetManager: Starting task 116.0 in stage 24.0 (TID 427) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:13 INFO Executor: Running task 116.0 in stage 24.0 (TID 427) +26/04/01 05:57:13 INFO TaskSetManager: Finished task 112.0 in stage 24.0 (TID 423) in 484 ms on 10.0.0.133 (executor driver) (113/200) +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO Executor: Finished task 113.0 in stage 24.0 (TID 424). 10850 bytes result sent to driver +26/04/01 05:57:13 INFO TaskSetManager: Starting task 117.0 in stage 24.0 (TID 428) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:13 INFO Executor: Running task 117.0 in stage 24.0 (TID 428) +26/04/01 05:57:13 INFO TaskSetManager: Finished task 113.0 in stage 24.0 (TID 424) in 483 ms on 10.0.0.133 (executor driver) (114/200) +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1229.1 KiB) non-empty blocks including 7 (1229.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO Executor: Finished task 114.0 in stage 24.0 (TID 425). 10850 bytes result sent to driver +26/04/01 05:57:13 INFO TaskSetManager: Starting task 118.0 in stage 24.0 (TID 429) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:13 INFO TaskSetManager: Finished task 114.0 in stage 24.0 (TID 425) in 490 ms on 10.0.0.133 (executor driver) (115/200) +26/04/01 05:57:13 INFO Executor: Running task 118.0 in stage 24.0 (TID 429) +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO Executor: Finished task 115.0 in stage 24.0 (TID 426). 10807 bytes result sent to driver +26/04/01 05:57:13 INFO TaskSetManager: Starting task 119.0 in stage 24.0 (TID 430) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:13 INFO TaskSetManager: Finished task 115.0 in stage 24.0 (TID 426) in 492 ms on 10.0.0.133 (executor driver) (116/200) +26/04/01 05:57:13 INFO Executor: Running task 119.0 in stage 24.0 (TID 430) +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:13 INFO Executor: Finished task 116.0 in stage 24.0 (TID 427). 10807 bytes result sent to driver +26/04/01 05:57:13 INFO TaskSetManager: Starting task 120.0 in stage 24.0 (TID 431) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:13 INFO TaskSetManager: Finished task 116.0 in stage 24.0 (TID 427) in 484 ms on 10.0.0.133 (executor driver) (117/200) +26/04/01 05:57:13 INFO Executor: Running task 120.0 in stage 24.0 (TID 431) +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO Executor: Finished task 117.0 in stage 24.0 (TID 428). 10893 bytes result sent to driver +26/04/01 05:57:14 INFO TaskSetManager: Starting task 121.0 in stage 24.0 (TID 432) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:14 INFO Executor: Running task 121.0 in stage 24.0 (TID 432) +26/04/01 05:57:14 INFO TaskSetManager: Finished task 117.0 in stage 24.0 (TID 428) in 487 ms on 10.0.0.133 (executor driver) (118/200) +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO Executor: Finished task 118.0 in stage 24.0 (TID 429). 10893 bytes result sent to driver +26/04/01 05:57:14 INFO TaskSetManager: Starting task 122.0 in stage 24.0 (TID 433) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:14 INFO TaskSetManager: Finished task 118.0 in stage 24.0 (TID 429) in 497 ms on 10.0.0.133 (executor driver) (119/200) +26/04/01 05:57:14 INFO Executor: Running task 122.0 in stage 24.0 (TID 433) +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO Executor: Finished task 119.0 in stage 24.0 (TID 430). 10850 bytes result sent to driver +26/04/01 05:57:14 INFO TaskSetManager: Starting task 123.0 in stage 24.0 (TID 434) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:14 INFO Executor: Running task 123.0 in stage 24.0 (TID 434) +26/04/01 05:57:14 INFO TaskSetManager: Finished task 119.0 in stage 24.0 (TID 430) in 499 ms on 10.0.0.133 (executor driver) (120/200) +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO Executor: Finished task 120.0 in stage 24.0 (TID 431). 10893 bytes result sent to driver +26/04/01 05:57:14 INFO TaskSetManager: Starting task 124.0 in stage 24.0 (TID 435) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:14 INFO Executor: Running task 124.0 in stage 24.0 (TID 435) +26/04/01 05:57:14 INFO TaskSetManager: Finished task 120.0 in stage 24.0 (TID 431) in 486 ms on 10.0.0.133 (executor driver) (121/200) +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO Executor: Finished task 121.0 in stage 24.0 (TID 432). 10850 bytes result sent to driver +26/04/01 05:57:14 INFO TaskSetManager: Starting task 125.0 in stage 24.0 (TID 436) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:14 INFO Executor: Running task 125.0 in stage 24.0 (TID 436) +26/04/01 05:57:14 INFO TaskSetManager: Finished task 121.0 in stage 24.0 (TID 432) in 483 ms on 10.0.0.133 (executor driver) (122/200) +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO Executor: Finished task 122.0 in stage 24.0 (TID 433). 10893 bytes result sent to driver +26/04/01 05:57:14 INFO TaskSetManager: Starting task 126.0 in stage 24.0 (TID 437) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:14 INFO Executor: Finished task 123.0 in stage 24.0 (TID 434). 10850 bytes result sent to driver +26/04/01 05:57:14 INFO Executor: Running task 126.0 in stage 24.0 (TID 437) +26/04/01 05:57:14 INFO TaskSetManager: Starting task 127.0 in stage 24.0 (TID 438) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:14 INFO Executor: Running task 127.0 in stage 24.0 (TID 438) +26/04/01 05:57:14 INFO TaskSetManager: Finished task 122.0 in stage 24.0 (TID 433) in 523 ms on 10.0.0.133 (executor driver) (123/200) +26/04/01 05:57:14 INFO TaskSetManager: Finished task 123.0 in stage 24.0 (TID 434) in 513 ms on 10.0.0.133 (executor driver) (124/200) +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO Executor: Finished task 124.0 in stage 24.0 (TID 435). 10893 bytes result sent to driver +26/04/01 05:57:14 INFO TaskSetManager: Starting task 128.0 in stage 24.0 (TID 439) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:14 INFO TaskSetManager: Finished task 124.0 in stage 24.0 (TID 435) in 482 ms on 10.0.0.133 (executor driver) (125/200) +26/04/01 05:57:14 INFO Executor: Running task 128.0 in stage 24.0 (TID 439) +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO Executor: Finished task 125.0 in stage 24.0 (TID 436). 10893 bytes result sent to driver +26/04/01 05:57:15 INFO TaskSetManager: Starting task 129.0 in stage 24.0 (TID 440) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:15 INFO Executor: Running task 129.0 in stage 24.0 (TID 440) +26/04/01 05:57:15 INFO TaskSetManager: Finished task 125.0 in stage 24.0 (TID 436) in 483 ms on 10.0.0.133 (executor driver) (126/200) +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO Executor: Finished task 126.0 in stage 24.0 (TID 437). 10850 bytes result sent to driver +26/04/01 05:57:15 INFO TaskSetManager: Starting task 130.0 in stage 24.0 (TID 441) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:15 INFO TaskSetManager: Finished task 126.0 in stage 24.0 (TID 437) in 495 ms on 10.0.0.133 (executor driver) (127/200) +26/04/01 05:57:15 INFO Executor: Running task 130.0 in stage 24.0 (TID 441) +26/04/01 05:57:15 INFO Executor: Finished task 127.0 in stage 24.0 (TID 438). 10807 bytes result sent to driver +26/04/01 05:57:15 INFO TaskSetManager: Starting task 131.0 in stage 24.0 (TID 442) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:15 INFO Executor: Running task 131.0 in stage 24.0 (TID 442) +26/04/01 05:57:15 INFO TaskSetManager: Finished task 127.0 in stage 24.0 (TID 438) in 496 ms on 10.0.0.133 (executor driver) (128/200) +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO Executor: Finished task 128.0 in stage 24.0 (TID 439). 10850 bytes result sent to driver +26/04/01 05:57:15 INFO TaskSetManager: Starting task 132.0 in stage 24.0 (TID 443) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:15 INFO Executor: Running task 132.0 in stage 24.0 (TID 443) +26/04/01 05:57:15 INFO TaskSetManager: Finished task 128.0 in stage 24.0 (TID 439) in 483 ms on 10.0.0.133 (executor driver) (129/200) +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO Executor: Finished task 129.0 in stage 24.0 (TID 440). 10893 bytes result sent to driver +26/04/01 05:57:15 INFO TaskSetManager: Starting task 133.0 in stage 24.0 (TID 444) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:15 INFO Executor: Running task 133.0 in stage 24.0 (TID 444) +26/04/01 05:57:15 INFO TaskSetManager: Finished task 129.0 in stage 24.0 (TID 440) in 484 ms on 10.0.0.133 (executor driver) (130/200) +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO Executor: Finished task 131.0 in stage 24.0 (TID 442). 10850 bytes result sent to driver +26/04/01 05:57:15 INFO TaskSetManager: Starting task 134.0 in stage 24.0 (TID 445) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:15 INFO Executor: Running task 134.0 in stage 24.0 (TID 445) +26/04/01 05:57:15 INFO TaskSetManager: Finished task 131.0 in stage 24.0 (TID 442) in 494 ms on 10.0.0.133 (executor driver) (131/200) +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO Executor: Finished task 130.0 in stage 24.0 (TID 441). 10893 bytes result sent to driver +26/04/01 05:57:15 INFO TaskSetManager: Starting task 135.0 in stage 24.0 (TID 446) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:15 INFO Executor: Running task 135.0 in stage 24.0 (TID 446) +26/04/01 05:57:15 INFO TaskSetManager: Finished task 130.0 in stage 24.0 (TID 441) in 501 ms on 10.0.0.133 (executor driver) (132/200) +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO Executor: Finished task 132.0 in stage 24.0 (TID 443). 10893 bytes result sent to driver +26/04/01 05:57:15 INFO TaskSetManager: Starting task 136.0 in stage 24.0 (TID 447) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:15 INFO Executor: Running task 136.0 in stage 24.0 (TID 447) +26/04/01 05:57:15 INFO TaskSetManager: Finished task 132.0 in stage 24.0 (TID 443) in 486 ms on 10.0.0.133 (executor driver) (133/200) +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO Executor: Finished task 133.0 in stage 24.0 (TID 444). 10850 bytes result sent to driver +26/04/01 05:57:16 INFO TaskSetManager: Starting task 137.0 in stage 24.0 (TID 448) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:16 INFO Executor: Running task 137.0 in stage 24.0 (TID 448) +26/04/01 05:57:16 INFO TaskSetManager: Finished task 133.0 in stage 24.0 (TID 444) in 489 ms on 10.0.0.133 (executor driver) (134/200) +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO Executor: Finished task 134.0 in stage 24.0 (TID 445). 10850 bytes result sent to driver +26/04/01 05:57:16 INFO TaskSetManager: Starting task 138.0 in stage 24.0 (TID 449) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:16 INFO Executor: Running task 138.0 in stage 24.0 (TID 449) +26/04/01 05:57:16 INFO TaskSetManager: Finished task 134.0 in stage 24.0 (TID 445) in 496 ms on 10.0.0.133 (executor driver) (135/200) +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO Executor: Finished task 135.0 in stage 24.0 (TID 446). 10850 bytes result sent to driver +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO TaskSetManager: Starting task 139.0 in stage 24.0 (TID 450) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:16 INFO Executor: Running task 139.0 in stage 24.0 (TID 450) +26/04/01 05:57:16 INFO TaskSetManager: Finished task 135.0 in stage 24.0 (TID 446) in 495 ms on 10.0.0.133 (executor driver) (136/200) +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO Executor: Finished task 136.0 in stage 24.0 (TID 447). 10850 bytes result sent to driver +26/04/01 05:57:16 INFO TaskSetManager: Starting task 140.0 in stage 24.0 (TID 451) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:16 INFO Executor: Running task 140.0 in stage 24.0 (TID 451) +26/04/01 05:57:16 INFO TaskSetManager: Finished task 136.0 in stage 24.0 (TID 447) in 483 ms on 10.0.0.133 (executor driver) (137/200) +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO Executor: Finished task 137.0 in stage 24.0 (TID 448). 10893 bytes result sent to driver +26/04/01 05:57:16 INFO TaskSetManager: Starting task 141.0 in stage 24.0 (TID 452) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:16 INFO Executor: Running task 141.0 in stage 24.0 (TID 452) +26/04/01 05:57:16 INFO TaskSetManager: Finished task 137.0 in stage 24.0 (TID 448) in 490 ms on 10.0.0.133 (executor driver) (138/200) +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO Executor: Finished task 139.0 in stage 24.0 (TID 450). 10807 bytes result sent to driver +26/04/01 05:57:16 INFO Executor: Finished task 138.0 in stage 24.0 (TID 449). 10850 bytes result sent to driver +26/04/01 05:57:16 INFO TaskSetManager: Starting task 142.0 in stage 24.0 (TID 453) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:16 INFO TaskSetManager: Starting task 143.0 in stage 24.0 (TID 454) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:16 INFO Executor: Running task 142.0 in stage 24.0 (TID 453) +26/04/01 05:57:16 INFO TaskSetManager: Finished task 139.0 in stage 24.0 (TID 450) in 529 ms on 10.0.0.133 (executor driver) (139/200) +26/04/01 05:57:16 INFO Executor: Running task 143.0 in stage 24.0 (TID 454) +26/04/01 05:57:16 INFO TaskSetManager: Finished task 138.0 in stage 24.0 (TID 449) in 533 ms on 10.0.0.133 (executor driver) (140/200) +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO Executor: Finished task 140.0 in stage 24.0 (TID 451). 10807 bytes result sent to driver +26/04/01 05:57:16 INFO TaskSetManager: Starting task 144.0 in stage 24.0 (TID 455) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:16 INFO TaskSetManager: Finished task 140.0 in stage 24.0 (TID 451) in 501 ms on 10.0.0.133 (executor driver) (141/200) +26/04/01 05:57:16 INFO Executor: Running task 144.0 in stage 24.0 (TID 455) +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:17 INFO Executor: Finished task 141.0 in stage 24.0 (TID 452). 10850 bytes result sent to driver +26/04/01 05:57:17 INFO TaskSetManager: Starting task 145.0 in stage 24.0 (TID 456) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:17 INFO Executor: Running task 145.0 in stage 24.0 (TID 456) +26/04/01 05:57:17 INFO TaskSetManager: Finished task 141.0 in stage 24.0 (TID 452) in 488 ms on 10.0.0.133 (executor driver) (142/200) +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:17 INFO Executor: Finished task 143.0 in stage 24.0 (TID 454). 10893 bytes result sent to driver +26/04/01 05:57:17 INFO TaskSetManager: Starting task 146.0 in stage 24.0 (TID 457) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:17 INFO TaskSetManager: Finished task 143.0 in stage 24.0 (TID 454) in 521 ms on 10.0.0.133 (executor driver) (143/200) +26/04/01 05:57:17 INFO Executor: Running task 146.0 in stage 24.0 (TID 457) +26/04/01 05:57:17 INFO Executor: Finished task 142.0 in stage 24.0 (TID 453). 10893 bytes result sent to driver +26/04/01 05:57:17 INFO TaskSetManager: Starting task 147.0 in stage 24.0 (TID 458) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:17 INFO Executor: Running task 147.0 in stage 24.0 (TID 458) +26/04/01 05:57:17 INFO TaskSetManager: Finished task 142.0 in stage 24.0 (TID 453) in 522 ms on 10.0.0.133 (executor driver) (144/200) +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 8 ms +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 8 ms +26/04/01 05:57:17 INFO Executor: Finished task 144.0 in stage 24.0 (TID 455). 10893 bytes result sent to driver +26/04/01 05:57:17 INFO TaskSetManager: Starting task 148.0 in stage 24.0 (TID 459) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:17 INFO Executor: Running task 148.0 in stage 24.0 (TID 459) +26/04/01 05:57:17 INFO TaskSetManager: Finished task 144.0 in stage 24.0 (TID 455) in 542 ms on 10.0.0.133 (executor driver) (145/200) +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:17 INFO Executor: Finished task 145.0 in stage 24.0 (TID 456). 10893 bytes result sent to driver +26/04/01 05:57:17 INFO TaskSetManager: Starting task 149.0 in stage 24.0 (TID 460) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:17 INFO Executor: Running task 149.0 in stage 24.0 (TID 460) +26/04/01 05:57:17 INFO TaskSetManager: Finished task 145.0 in stage 24.0 (TID 456) in 495 ms on 10.0.0.133 (executor driver) (146/200) +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:17 INFO Executor: Finished task 147.0 in stage 24.0 (TID 458). 10807 bytes result sent to driver +26/04/01 05:57:17 INFO TaskSetManager: Starting task 150.0 in stage 24.0 (TID 461) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:17 INFO Executor: Running task 150.0 in stage 24.0 (TID 461) +26/04/01 05:57:17 INFO TaskSetManager: Finished task 147.0 in stage 24.0 (TID 458) in 504 ms on 10.0.0.133 (executor driver) (147/200) +26/04/01 05:57:17 INFO Executor: Finished task 146.0 in stage 24.0 (TID 457). 10850 bytes result sent to driver +26/04/01 05:57:17 INFO TaskSetManager: Starting task 151.0 in stage 24.0 (TID 462) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:17 INFO Executor: Running task 151.0 in stage 24.0 (TID 462) +26/04/01 05:57:17 INFO TaskSetManager: Finished task 146.0 in stage 24.0 (TID 457) in 509 ms on 10.0.0.133 (executor driver) (148/200) +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:17 INFO Executor: Finished task 148.0 in stage 24.0 (TID 459). 10807 bytes result sent to driver +26/04/01 05:57:17 INFO TaskSetManager: Starting task 152.0 in stage 24.0 (TID 463) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:17 INFO Executor: Running task 152.0 in stage 24.0 (TID 463) +26/04/01 05:57:17 INFO TaskSetManager: Finished task 148.0 in stage 24.0 (TID 459) in 494 ms on 10.0.0.133 (executor driver) (149/200) +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO Executor: Finished task 149.0 in stage 24.0 (TID 460). 10850 bytes result sent to driver +26/04/01 05:57:18 INFO TaskSetManager: Starting task 153.0 in stage 24.0 (TID 464) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:18 INFO Executor: Running task 153.0 in stage 24.0 (TID 464) +26/04/01 05:57:18 INFO TaskSetManager: Finished task 149.0 in stage 24.0 (TID 460) in 483 ms on 10.0.0.133 (executor driver) (150/200) +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO Executor: Finished task 150.0 in stage 24.0 (TID 461). 10893 bytes result sent to driver +26/04/01 05:57:18 INFO TaskSetManager: Starting task 154.0 in stage 24.0 (TID 465) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:18 INFO Executor: Running task 154.0 in stage 24.0 (TID 465) +26/04/01 05:57:18 INFO TaskSetManager: Finished task 150.0 in stage 24.0 (TID 461) in 503 ms on 10.0.0.133 (executor driver) (151/200) +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO Executor: Finished task 151.0 in stage 24.0 (TID 462). 10850 bytes result sent to driver +26/04/01 05:57:18 INFO TaskSetManager: Starting task 155.0 in stage 24.0 (TID 466) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:18 INFO Executor: Running task 155.0 in stage 24.0 (TID 466) +26/04/01 05:57:18 INFO TaskSetManager: Finished task 151.0 in stage 24.0 (TID 462) in 506 ms on 10.0.0.133 (executor driver) (152/200) +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO Executor: Finished task 152.0 in stage 24.0 (TID 463). 10850 bytes result sent to driver +26/04/01 05:57:18 INFO TaskSetManager: Starting task 156.0 in stage 24.0 (TID 467) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:18 INFO Executor: Running task 156.0 in stage 24.0 (TID 467) +26/04/01 05:57:18 INFO TaskSetManager: Finished task 152.0 in stage 24.0 (TID 463) in 491 ms on 10.0.0.133 (executor driver) (153/200) +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO Executor: Finished task 153.0 in stage 24.0 (TID 464). 10850 bytes result sent to driver +26/04/01 05:57:18 INFO TaskSetManager: Starting task 157.0 in stage 24.0 (TID 468) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:18 INFO Executor: Running task 157.0 in stage 24.0 (TID 468) +26/04/01 05:57:18 INFO TaskSetManager: Finished task 153.0 in stage 24.0 (TID 464) in 485 ms on 10.0.0.133 (executor driver) (154/200) +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO Executor: Finished task 155.0 in stage 24.0 (TID 466). 10850 bytes result sent to driver +26/04/01 05:57:18 INFO TaskSetManager: Starting task 158.0 in stage 24.0 (TID 469) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:18 INFO Executor: Running task 158.0 in stage 24.0 (TID 469) +26/04/01 05:57:18 INFO TaskSetManager: Finished task 155.0 in stage 24.0 (TID 466) in 510 ms on 10.0.0.133 (executor driver) (155/200) +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO Executor: Finished task 154.0 in stage 24.0 (TID 465). 10893 bytes result sent to driver +26/04/01 05:57:18 INFO TaskSetManager: Starting task 159.0 in stage 24.0 (TID 470) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:18 INFO Executor: Running task 159.0 in stage 24.0 (TID 470) +26/04/01 05:57:18 INFO TaskSetManager: Finished task 154.0 in stage 24.0 (TID 465) in 528 ms on 10.0.0.133 (executor driver) (156/200) +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO Executor: Finished task 156.0 in stage 24.0 (TID 467). 10850 bytes result sent to driver +26/04/01 05:57:18 INFO TaskSetManager: Starting task 160.0 in stage 24.0 (TID 471) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:18 INFO Executor: Running task 160.0 in stage 24.0 (TID 471) +26/04/01 05:57:18 INFO TaskSetManager: Finished task 156.0 in stage 24.0 (TID 467) in 505 ms on 10.0.0.133 (executor driver) (157/200) +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO Executor: Finished task 157.0 in stage 24.0 (TID 468). 10893 bytes result sent to driver +26/04/01 05:57:19 INFO TaskSetManager: Starting task 161.0 in stage 24.0 (TID 472) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:19 INFO Executor: Running task 161.0 in stage 24.0 (TID 472) +26/04/01 05:57:19 INFO TaskSetManager: Finished task 157.0 in stage 24.0 (TID 468) in 490 ms on 10.0.0.133 (executor driver) (158/200) +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO Executor: Finished task 158.0 in stage 24.0 (TID 469). 10850 bytes result sent to driver +26/04/01 05:57:19 INFO TaskSetManager: Starting task 162.0 in stage 24.0 (TID 473) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:19 INFO Executor: Running task 162.0 in stage 24.0 (TID 473) +26/04/01 05:57:19 INFO TaskSetManager: Finished task 158.0 in stage 24.0 (TID 469) in 501 ms on 10.0.0.133 (executor driver) (159/200) +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO Executor: Finished task 160.0 in stage 24.0 (TID 471). 10807 bytes result sent to driver +26/04/01 05:57:19 INFO TaskSetManager: Starting task 163.0 in stage 24.0 (TID 474) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:19 INFO Executor: Running task 163.0 in stage 24.0 (TID 474) +26/04/01 05:57:19 INFO TaskSetManager: Finished task 160.0 in stage 24.0 (TID 471) in 498 ms on 10.0.0.133 (executor driver) (160/200) +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO Executor: Finished task 159.0 in stage 24.0 (TID 470). 10807 bytes result sent to driver +26/04/01 05:57:19 INFO TaskSetManager: Starting task 164.0 in stage 24.0 (TID 475) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:19 INFO TaskSetManager: Finished task 159.0 in stage 24.0 (TID 470) in 504 ms on 10.0.0.133 (executor driver) (161/200) +26/04/01 05:57:19 INFO Executor: Running task 164.0 in stage 24.0 (TID 475) +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO Executor: Finished task 161.0 in stage 24.0 (TID 472). 10807 bytes result sent to driver +26/04/01 05:57:19 INFO TaskSetManager: Starting task 165.0 in stage 24.0 (TID 476) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:19 INFO Executor: Running task 165.0 in stage 24.0 (TID 476) +26/04/01 05:57:19 INFO TaskSetManager: Finished task 161.0 in stage 24.0 (TID 472) in 481 ms on 10.0.0.133 (executor driver) (162/200) +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO Executor: Finished task 162.0 in stage 24.0 (TID 473). 10850 bytes result sent to driver +26/04/01 05:57:19 INFO TaskSetManager: Starting task 166.0 in stage 24.0 (TID 477) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:19 INFO Executor: Running task 166.0 in stage 24.0 (TID 477) +26/04/01 05:57:19 INFO TaskSetManager: Finished task 162.0 in stage 24.0 (TID 473) in 498 ms on 10.0.0.133 (executor driver) (163/200) +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO Executor: Finished task 163.0 in stage 24.0 (TID 474). 10850 bytes result sent to driver +26/04/01 05:57:19 INFO TaskSetManager: Starting task 167.0 in stage 24.0 (TID 478) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:19 INFO Executor: Running task 167.0 in stage 24.0 (TID 478) +26/04/01 05:57:19 INFO TaskSetManager: Finished task 163.0 in stage 24.0 (TID 474) in 497 ms on 10.0.0.133 (executor driver) (164/200) +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO Executor: Finished task 164.0 in stage 24.0 (TID 475). 10850 bytes result sent to driver +26/04/01 05:57:19 INFO TaskSetManager: Starting task 168.0 in stage 24.0 (TID 479) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:19 INFO Executor: Running task 168.0 in stage 24.0 (TID 479) +26/04/01 05:57:19 INFO TaskSetManager: Finished task 164.0 in stage 24.0 (TID 475) in 498 ms on 10.0.0.133 (executor driver) (165/200) +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO Executor: Finished task 165.0 in stage 24.0 (TID 476). 10850 bytes result sent to driver +26/04/01 05:57:20 INFO TaskSetManager: Starting task 169.0 in stage 24.0 (TID 480) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:20 INFO Executor: Running task 169.0 in stage 24.0 (TID 480) +26/04/01 05:57:20 INFO TaskSetManager: Finished task 165.0 in stage 24.0 (TID 476) in 488 ms on 10.0.0.133 (executor driver) (166/200) +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO Executor: Finished task 166.0 in stage 24.0 (TID 477). 10893 bytes result sent to driver +26/04/01 05:57:20 INFO TaskSetManager: Starting task 170.0 in stage 24.0 (TID 481) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:20 INFO TaskSetManager: Finished task 166.0 in stage 24.0 (TID 477) in 502 ms on 10.0.0.133 (executor driver) (167/200) +26/04/01 05:57:20 INFO Executor: Running task 170.0 in stage 24.0 (TID 481) +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO Executor: Finished task 167.0 in stage 24.0 (TID 478). 10850 bytes result sent to driver +26/04/01 05:57:20 INFO TaskSetManager: Starting task 171.0 in stage 24.0 (TID 482) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:20 INFO Executor: Finished task 168.0 in stage 24.0 (TID 479). 10850 bytes result sent to driver +26/04/01 05:57:20 INFO Executor: Running task 171.0 in stage 24.0 (TID 482) +26/04/01 05:57:20 INFO TaskSetManager: Finished task 167.0 in stage 24.0 (TID 478) in 510 ms on 10.0.0.133 (executor driver) (168/200) +26/04/01 05:57:20 INFO TaskSetManager: Starting task 172.0 in stage 24.0 (TID 483) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:20 INFO Executor: Running task 172.0 in stage 24.0 (TID 483) +26/04/01 05:57:20 INFO TaskSetManager: Finished task 168.0 in stage 24.0 (TID 479) in 505 ms on 10.0.0.133 (executor driver) (169/200) +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO Executor: Finished task 169.0 in stage 24.0 (TID 480). 10850 bytes result sent to driver +26/04/01 05:57:20 INFO TaskSetManager: Starting task 173.0 in stage 24.0 (TID 484) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:20 INFO Executor: Running task 173.0 in stage 24.0 (TID 484) +26/04/01 05:57:20 INFO TaskSetManager: Finished task 169.0 in stage 24.0 (TID 480) in 481 ms on 10.0.0.133 (executor driver) (170/200) +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO Executor: Finished task 170.0 in stage 24.0 (TID 481). 10850 bytes result sent to driver +26/04/01 05:57:20 INFO TaskSetManager: Starting task 174.0 in stage 24.0 (TID 485) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:20 INFO Executor: Running task 174.0 in stage 24.0 (TID 485) +26/04/01 05:57:20 INFO TaskSetManager: Finished task 170.0 in stage 24.0 (TID 481) in 497 ms on 10.0.0.133 (executor driver) (171/200) +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO Executor: Finished task 172.0 in stage 24.0 (TID 483). 10807 bytes result sent to driver +26/04/01 05:57:20 INFO TaskSetManager: Starting task 175.0 in stage 24.0 (TID 486) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:20 INFO TaskSetManager: Finished task 172.0 in stage 24.0 (TID 483) in 500 ms on 10.0.0.133 (executor driver) (172/200) +26/04/01 05:57:20 INFO Executor: Running task 175.0 in stage 24.0 (TID 486) +26/04/01 05:57:20 INFO Executor: Finished task 171.0 in stage 24.0 (TID 482). 10807 bytes result sent to driver +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO TaskSetManager: Starting task 176.0 in stage 24.0 (TID 487) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO TaskSetManager: Finished task 171.0 in stage 24.0 (TID 482) in 501 ms on 10.0.0.133 (executor driver) (173/200) +26/04/01 05:57:20 INFO Executor: Running task 176.0 in stage 24.0 (TID 487) +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO Executor: Finished task 173.0 in stage 24.0 (TID 484). 10807 bytes result sent to driver +26/04/01 05:57:21 INFO TaskSetManager: Starting task 177.0 in stage 24.0 (TID 488) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:21 INFO TaskSetManager: Finished task 173.0 in stage 24.0 (TID 484) in 480 ms on 10.0.0.133 (executor driver) (174/200) +26/04/01 05:57:21 INFO Executor: Running task 177.0 in stage 24.0 (TID 488) +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO Executor: Finished task 174.0 in stage 24.0 (TID 485). 10893 bytes result sent to driver +26/04/01 05:57:21 INFO TaskSetManager: Starting task 178.0 in stage 24.0 (TID 489) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:21 INFO Executor: Running task 178.0 in stage 24.0 (TID 489) +26/04/01 05:57:21 INFO TaskSetManager: Finished task 174.0 in stage 24.0 (TID 485) in 496 ms on 10.0.0.133 (executor driver) (175/200) +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO Executor: Finished task 175.0 in stage 24.0 (TID 486). 10850 bytes result sent to driver +26/04/01 05:57:21 INFO TaskSetManager: Starting task 179.0 in stage 24.0 (TID 490) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:21 INFO TaskSetManager: Finished task 175.0 in stage 24.0 (TID 486) in 508 ms on 10.0.0.133 (executor driver) (176/200) +26/04/01 05:57:21 INFO Executor: Running task 179.0 in stage 24.0 (TID 490) +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO Executor: Finished task 176.0 in stage 24.0 (TID 487). 10850 bytes result sent to driver +26/04/01 05:57:21 INFO TaskSetManager: Starting task 180.0 in stage 24.0 (TID 491) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:21 INFO Executor: Running task 180.0 in stage 24.0 (TID 491) +26/04/01 05:57:21 INFO TaskSetManager: Finished task 176.0 in stage 24.0 (TID 487) in 512 ms on 10.0.0.133 (executor driver) (177/200) +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO Executor: Finished task 177.0 in stage 24.0 (TID 488). 10893 bytes result sent to driver +26/04/01 05:57:21 INFO TaskSetManager: Starting task 181.0 in stage 24.0 (TID 492) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:21 INFO Executor: Running task 181.0 in stage 24.0 (TID 492) +26/04/01 05:57:21 INFO TaskSetManager: Finished task 177.0 in stage 24.0 (TID 488) in 485 ms on 10.0.0.133 (executor driver) (178/200) +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO Executor: Finished task 178.0 in stage 24.0 (TID 489). 10893 bytes result sent to driver +26/04/01 05:57:21 INFO TaskSetManager: Starting task 182.0 in stage 24.0 (TID 493) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:21 INFO Executor: Running task 182.0 in stage 24.0 (TID 493) +26/04/01 05:57:21 INFO TaskSetManager: Finished task 178.0 in stage 24.0 (TID 489) in 494 ms on 10.0.0.133 (executor driver) (179/200) +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO Executor: Finished task 180.0 in stage 24.0 (TID 491). 10850 bytes result sent to driver +26/04/01 05:57:21 INFO TaskSetManager: Starting task 183.0 in stage 24.0 (TID 494) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:21 INFO TaskSetManager: Finished task 180.0 in stage 24.0 (TID 491) in 491 ms on 10.0.0.133 (executor driver) (180/200) +26/04/01 05:57:21 INFO Executor: Running task 183.0 in stage 24.0 (TID 494) +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO Executor: Finished task 179.0 in stage 24.0 (TID 490). 10850 bytes result sent to driver +26/04/01 05:57:21 INFO TaskSetManager: Starting task 184.0 in stage 24.0 (TID 495) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:21 INFO Executor: Running task 184.0 in stage 24.0 (TID 495) +26/04/01 05:57:21 INFO TaskSetManager: Finished task 179.0 in stage 24.0 (TID 490) in 499 ms on 10.0.0.133 (executor driver) (181/200) +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO Executor: Finished task 181.0 in stage 24.0 (TID 492). 10850 bytes result sent to driver +26/04/01 05:57:22 INFO TaskSetManager: Starting task 185.0 in stage 24.0 (TID 496) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:22 INFO Executor: Running task 185.0 in stage 24.0 (TID 496) +26/04/01 05:57:22 INFO TaskSetManager: Finished task 181.0 in stage 24.0 (TID 492) in 484 ms on 10.0.0.133 (executor driver) (182/200) +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO Executor: Finished task 182.0 in stage 24.0 (TID 493). 10850 bytes result sent to driver +26/04/01 05:57:22 INFO TaskSetManager: Starting task 186.0 in stage 24.0 (TID 497) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:22 INFO Executor: Running task 186.0 in stage 24.0 (TID 497) +26/04/01 05:57:22 INFO TaskSetManager: Finished task 182.0 in stage 24.0 (TID 493) in 491 ms on 10.0.0.133 (executor driver) (183/200) +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO Executor: Finished task 183.0 in stage 24.0 (TID 494). 10807 bytes result sent to driver +26/04/01 05:57:22 INFO TaskSetManager: Starting task 187.0 in stage 24.0 (TID 498) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:22 INFO TaskSetManager: Finished task 183.0 in stage 24.0 (TID 494) in 514 ms on 10.0.0.133 (executor driver) (184/200) +26/04/01 05:57:22 INFO Executor: Running task 187.0 in stage 24.0 (TID 498) +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO Executor: Finished task 184.0 in stage 24.0 (TID 495). 10807 bytes result sent to driver +26/04/01 05:57:22 INFO TaskSetManager: Starting task 188.0 in stage 24.0 (TID 499) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:22 INFO Executor: Running task 188.0 in stage 24.0 (TID 499) +26/04/01 05:57:22 INFO TaskSetManager: Finished task 184.0 in stage 24.0 (TID 495) in 515 ms on 10.0.0.133 (executor driver) (185/200) +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO Executor: Finished task 185.0 in stage 24.0 (TID 496). 10807 bytes result sent to driver +26/04/01 05:57:22 INFO TaskSetManager: Starting task 189.0 in stage 24.0 (TID 500) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:22 INFO Executor: Running task 189.0 in stage 24.0 (TID 500) +26/04/01 05:57:22 INFO TaskSetManager: Finished task 185.0 in stage 24.0 (TID 496) in 498 ms on 10.0.0.133 (executor driver) (186/200) +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO Executor: Finished task 186.0 in stage 24.0 (TID 497). 10893 bytes result sent to driver +26/04/01 05:57:22 INFO TaskSetManager: Starting task 190.0 in stage 24.0 (TID 501) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:22 INFO Executor: Running task 190.0 in stage 24.0 (TID 501) +26/04/01 05:57:22 INFO TaskSetManager: Finished task 186.0 in stage 24.0 (TID 497) in 489 ms on 10.0.0.133 (executor driver) (187/200) +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO Executor: Finished task 187.0 in stage 24.0 (TID 498). 10850 bytes result sent to driver +26/04/01 05:57:22 INFO TaskSetManager: Starting task 191.0 in stage 24.0 (TID 502) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:22 INFO Executor: Running task 191.0 in stage 24.0 (TID 502) +26/04/01 05:57:22 INFO TaskSetManager: Finished task 187.0 in stage 24.0 (TID 498) in 495 ms on 10.0.0.133 (executor driver) (188/200) +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO Executor: Finished task 188.0 in stage 24.0 (TID 499). 10850 bytes result sent to driver +26/04/01 05:57:22 INFO TaskSetManager: Starting task 192.0 in stage 24.0 (TID 503) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:22 INFO Executor: Running task 192.0 in stage 24.0 (TID 503) +26/04/01 05:57:22 INFO TaskSetManager: Finished task 188.0 in stage 24.0 (TID 499) in 498 ms on 10.0.0.133 (executor driver) (189/200) +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO Executor: Finished task 189.0 in stage 24.0 (TID 500). 10850 bytes result sent to driver +26/04/01 05:57:22 INFO TaskSetManager: Starting task 193.0 in stage 24.0 (TID 504) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:22 INFO TaskSetManager: Finished task 189.0 in stage 24.0 (TID 500) in 486 ms on 10.0.0.133 (executor driver) (190/200) +26/04/01 05:57:22 INFO Executor: Running task 193.0 in stage 24.0 (TID 504) +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:23 INFO Executor: Finished task 190.0 in stage 24.0 (TID 501). 10893 bytes result sent to driver +26/04/01 05:57:23 INFO TaskSetManager: Starting task 194.0 in stage 24.0 (TID 505) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:23 INFO Executor: Running task 194.0 in stage 24.0 (TID 505) +26/04/01 05:57:23 INFO TaskSetManager: Finished task 190.0 in stage 24.0 (TID 501) in 483 ms on 10.0.0.133 (executor driver) (191/200) +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:23 INFO Executor: Finished task 191.0 in stage 24.0 (TID 502). 10850 bytes result sent to driver +26/04/01 05:57:23 INFO TaskSetManager: Starting task 195.0 in stage 24.0 (TID 506) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:23 INFO TaskSetManager: Finished task 191.0 in stage 24.0 (TID 502) in 496 ms on 10.0.0.133 (executor driver) (192/200) +26/04/01 05:57:23 INFO Executor: Running task 195.0 in stage 24.0 (TID 506) +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:23 INFO Executor: Finished task 192.0 in stage 24.0 (TID 503). 10850 bytes result sent to driver +26/04/01 05:57:23 INFO TaskSetManager: Starting task 196.0 in stage 24.0 (TID 507) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:23 INFO Executor: Running task 196.0 in stage 24.0 (TID 507) +26/04/01 05:57:23 INFO TaskSetManager: Finished task 192.0 in stage 24.0 (TID 503) in 501 ms on 10.0.0.133 (executor driver) (193/200) +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:23 INFO Executor: Finished task 193.0 in stage 24.0 (TID 504). 10850 bytes result sent to driver +26/04/01 05:57:23 INFO TaskSetManager: Starting task 197.0 in stage 24.0 (TID 508) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:23 INFO TaskSetManager: Finished task 193.0 in stage 24.0 (TID 504) in 491 ms on 10.0.0.133 (executor driver) (194/200) +26/04/01 05:57:23 INFO Executor: Running task 197.0 in stage 24.0 (TID 508) +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:23 INFO Executor: Finished task 194.0 in stage 24.0 (TID 505). 10850 bytes result sent to driver +26/04/01 05:57:23 INFO TaskSetManager: Starting task 198.0 in stage 24.0 (TID 509) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:23 INFO Executor: Running task 198.0 in stage 24.0 (TID 509) +26/04/01 05:57:23 INFO TaskSetManager: Finished task 194.0 in stage 24.0 (TID 505) in 482 ms on 10.0.0.133 (executor driver) (195/200) +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:23 INFO Executor: Finished task 195.0 in stage 24.0 (TID 506). 10850 bytes result sent to driver +26/04/01 05:57:23 INFO TaskSetManager: Starting task 199.0 in stage 24.0 (TID 510) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:23 INFO Executor: Running task 199.0 in stage 24.0 (TID 510) +26/04/01 05:57:23 INFO TaskSetManager: Finished task 195.0 in stage 24.0 (TID 506) in 496 ms on 10.0.0.133 (executor driver) (196/200) +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:23 INFO Executor: Finished task 196.0 in stage 24.0 (TID 507). 10807 bytes result sent to driver +26/04/01 05:57:23 INFO TaskSetManager: Finished task 196.0 in stage 24.0 (TID 507) in 496 ms on 10.0.0.133 (executor driver) (197/200) +26/04/01 05:57:23 INFO Executor: Finished task 197.0 in stage 24.0 (TID 508). 10807 bytes result sent to driver +26/04/01 05:57:23 INFO TaskSetManager: Finished task 197.0 in stage 24.0 (TID 508) in 494 ms on 10.0.0.133 (executor driver) (198/200) +26/04/01 05:57:24 INFO Executor: Finished task 198.0 in stage 24.0 (TID 509). 10893 bytes result sent to driver +26/04/01 05:57:24 INFO TaskSetManager: Finished task 198.0 in stage 24.0 (TID 509) in 455 ms on 10.0.0.133 (executor driver) (199/200) +26/04/01 05:57:24 INFO Executor: Finished task 199.0 in stage 24.0 (TID 510). 10850 bytes result sent to driver +26/04/01 05:57:24 INFO TaskSetManager: Finished task 199.0 in stage 24.0 (TID 510) in 453 ms on 10.0.0.133 (executor driver) (200/200) +26/04/01 05:57:24 INFO TaskSchedulerImpl: Removed TaskSet 24.0, whose tasks have all completed, from pool +26/04/01 05:57:24 INFO DAGScheduler: ShuffleMapStage 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 25.219 s +26/04/01 05:57:24 INFO DAGScheduler: looking for newly runnable stages +26/04/01 05:57:24 INFO DAGScheduler: running: Set() +26/04/01 05:57:24 INFO DAGScheduler: waiting: Set() +26/04/01 05:57:24 INFO DAGScheduler: failed: Set() +26/04/01 05:57:24 INFO ShufflePartitionsUtil: For shuffle(6, 4), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 05:57:24 INFO HashAggregateExec: spark.sql.codegen.aggregate.map.twolevel.enabled is set to true, but current version of codegened fast hashmap does not support this aggregate. +26/04/01 05:57:24 INFO CodeGenerator: Code generated in 9.013417 ms +26/04/01 05:57:24 INFO CodeGenerator: Code generated in 2.152792 ms +26/04/01 05:57:24 INFO CodeGenerator: Code generated in 2.298209 ms +26/04/01 05:57:24 INFO DAGScheduler: Registering RDD 68 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 7 +26/04/01 05:57:24 INFO DAGScheduler: Got map stage job 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 29 output partitions +26/04/01 05:57:24 INFO DAGScheduler: Final stage: ShuffleMapStage 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:57:24 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 30, ShuffleMapStage 29) +26/04/01 05:57:24 INFO DAGScheduler: Missing parents: List() +26/04/01 05:57:24 INFO DAGScheduler: Submitting ShuffleMapStage 31 (MapPartitionsRDD[68] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:57:24 WARN DAGScheduler: Broadcasting large task binary with size 1137.7 KiB +26/04/01 05:57:24 INFO MemoryStore: Block broadcast_27 stored as values in memory (estimated size 1137.7 KiB, free 8.6 GiB) +26/04/01 05:57:24 INFO MemoryStore: Block broadcast_27_piece0 stored as bytes in memory (estimated size 55.3 KiB, free 8.6 GiB) +26/04/01 05:57:24 INFO BlockManagerInfo: Added broadcast_27_piece0 in memory on 10.0.0.133:58206 (size: 55.3 KiB, free: 8.6 GiB) +26/04/01 05:57:24 INFO SparkContext: Created broadcast 27 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:57:24 INFO DAGScheduler: Submitting 29 missing tasks from ShuffleMapStage 31 (MapPartitionsRDD[68] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 05:57:24 INFO TaskSchedulerImpl: Adding task set 31.0 with 29 tasks resource profile 0 +26/04/01 05:57:24 INFO TaskSetManager: Starting task 0.0 in stage 31.0 (TID 511) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:24 INFO TaskSetManager: Starting task 1.0 in stage 31.0 (TID 512) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:24 INFO TaskSetManager: Starting task 2.0 in stage 31.0 (TID 513) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:24 INFO TaskSetManager: Starting task 3.0 in stage 31.0 (TID 514) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:24 INFO Executor: Running task 1.0 in stage 31.0 (TID 512) +26/04/01 05:57:24 INFO Executor: Running task 2.0 in stage 31.0 (TID 513) +26/04/01 05:57:24 INFO Executor: Running task 3.0 in stage 31.0 (TID 514) +26/04/01 05:57:24 INFO Executor: Running task 0.0 in stage 31.0 (TID 511) +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 05:57:24 INFO CodeGenerator: Code generated in 3.116625 ms +26/04/01 05:57:24 INFO CodeGenerator: Code generated in 1.721291 ms +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 4 (323.6 KiB) non-empty blocks including 4 (323.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 4 (327.0 KiB) non-empty blocks including 4 (327.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 4 (325.9 KiB) non-empty blocks including 4 (325.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 4 (323.6 KiB) non-empty blocks including 4 (323.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:24 INFO CodeGenerator: Code generated in 2.260084 ms +26/04/01 05:57:24 INFO CodeGenerator: Code generated in 1.543 ms +26/04/01 05:57:24 INFO CodeGenerator: Code generated in 5.617625 ms +26/04/01 05:57:24 INFO CodeGenerator: Code generated in 1.698208 ms +26/04/01 05:57:24 INFO CodeGenerator: Code generated in 0.855083 ms +26/04/01 05:57:24 INFO CodeGenerator: Code generated in 1.547417 ms +26/04/01 05:57:24 INFO CodeGenerator: Code generated in 1.028583 ms +26/04/01 05:57:26 INFO Executor: Finished task 1.0 in stage 31.0 (TID 512). 19177 bytes result sent to driver +26/04/01 05:57:26 INFO Executor: Finished task 0.0 in stage 31.0 (TID 511). 19177 bytes result sent to driver +26/04/01 05:57:26 INFO TaskSetManager: Starting task 4.0 in stage 31.0 (TID 515) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:26 INFO TaskSetManager: Finished task 1.0 in stage 31.0 (TID 512) in 1818 ms on 10.0.0.133 (executor driver) (1/29) +26/04/01 05:57:26 INFO TaskSetManager: Starting task 5.0 in stage 31.0 (TID 516) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:26 INFO Executor: Running task 4.0 in stage 31.0 (TID 515) +26/04/01 05:57:26 INFO Executor: Running task 5.0 in stage 31.0 (TID 516) +26/04/01 05:57:26 INFO TaskSetManager: Finished task 0.0 in stage 31.0 (TID 511) in 1819 ms on 10.0.0.133 (executor driver) (2/29) +26/04/01 05:57:26 INFO Executor: Finished task 3.0 in stage 31.0 (TID 514). 19177 bytes result sent to driver +26/04/01 05:57:26 INFO TaskSetManager: Starting task 6.0 in stage 31.0 (TID 517) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:26 INFO Executor: Running task 6.0 in stage 31.0 (TID 517) +26/04/01 05:57:26 INFO Executor: Finished task 2.0 in stage 31.0 (TID 513). 19177 bytes result sent to driver +26/04/01 05:57:26 INFO TaskSetManager: Finished task 3.0 in stage 31.0 (TID 514) in 1822 ms on 10.0.0.133 (executor driver) (3/29) +26/04/01 05:57:26 INFO TaskSetManager: Starting task 7.0 in stage 31.0 (TID 518) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:26 INFO Executor: Running task 7.0 in stage 31.0 (TID 518) +26/04/01 05:57:26 INFO TaskSetManager: Finished task 2.0 in stage 31.0 (TID 513) in 1822 ms on 10.0.0.133 (executor driver) (4/29) +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 200 (63.1 MiB) non-empty blocks including 200 (63.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 200 (63.3 MiB) non-empty blocks including 200 (63.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 4 (323.6 KiB) non-empty blocks including 4 (323.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 4 (329.2 KiB) non-empty blocks including 4 (329.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 4 (325.9 KiB) non-empty blocks including 4 (325.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 4 (323.6 KiB) non-empty blocks including 4 (323.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:27 INFO Executor: Finished task 5.0 in stage 31.0 (TID 516). 19177 bytes result sent to driver +26/04/01 05:57:27 INFO TaskSetManager: Starting task 8.0 in stage 31.0 (TID 519) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:27 INFO Executor: Running task 8.0 in stage 31.0 (TID 519) +26/04/01 05:57:27 INFO TaskSetManager: Finished task 5.0 in stage 31.0 (TID 516) in 1661 ms on 10.0.0.133 (executor driver) (5/29) +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 4 (324.7 KiB) non-empty blocks including 4 (324.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:27 INFO Executor: Finished task 7.0 in stage 31.0 (TID 518). 19177 bytes result sent to driver +26/04/01 05:57:27 INFO TaskSetManager: Starting task 9.0 in stage 31.0 (TID 520) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:27 INFO Executor: Running task 9.0 in stage 31.0 (TID 520) +26/04/01 05:57:27 INFO TaskSetManager: Finished task 7.0 in stage 31.0 (TID 518) in 1670 ms on 10.0.0.133 (executor driver) (6/29) +26/04/01 05:57:27 INFO Executor: Finished task 6.0 in stage 31.0 (TID 517). 19177 bytes result sent to driver +26/04/01 05:57:27 INFO TaskSetManager: Starting task 10.0 in stage 31.0 (TID 521) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:27 INFO Executor: Running task 10.0 in stage 31.0 (TID 521) +26/04/01 05:57:27 INFO TaskSetManager: Finished task 6.0 in stage 31.0 (TID 517) in 1670 ms on 10.0.0.133 (executor driver) (7/29) +26/04/01 05:57:27 INFO Executor: Finished task 4.0 in stage 31.0 (TID 515). 19177 bytes result sent to driver +26/04/01 05:57:27 INFO TaskSetManager: Starting task 11.0 in stage 31.0 (TID 522) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:27 INFO Executor: Running task 11.0 in stage 31.0 (TID 522) +26/04/01 05:57:27 INFO TaskSetManager: Finished task 4.0 in stage 31.0 (TID 515) in 1676 ms on 10.0.0.133 (executor driver) (8/29) +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 200 (63.1 MiB) non-empty blocks including 200 (63.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 4 (328.1 KiB) non-empty blocks including 4 (328.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 4 (323.6 KiB) non-empty blocks including 4 (323.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 4 (323.6 KiB) non-empty blocks including 4 (323.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:29 INFO Executor: Finished task 8.0 in stage 31.0 (TID 519). 19220 bytes result sent to driver +26/04/01 05:57:29 INFO TaskSetManager: Starting task 12.0 in stage 31.0 (TID 523) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:29 INFO TaskSetManager: Finished task 8.0 in stage 31.0 (TID 519) in 1623 ms on 10.0.0.133 (executor driver) (9/29) +26/04/01 05:57:29 INFO Executor: Running task 12.0 in stage 31.0 (TID 523) +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 4 (324.7 KiB) non-empty blocks including 4 (324.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:29 INFO Executor: Finished task 9.0 in stage 31.0 (TID 520). 19220 bytes result sent to driver +26/04/01 05:57:29 INFO TaskSetManager: Starting task 13.0 in stage 31.0 (TID 524) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:29 INFO TaskSetManager: Finished task 9.0 in stage 31.0 (TID 520) in 1621 ms on 10.0.0.133 (executor driver) (10/29) +26/04/01 05:57:29 INFO Executor: Running task 13.0 in stage 31.0 (TID 524) +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:29 INFO Executor: Finished task 11.0 in stage 31.0 (TID 522). 19220 bytes result sent to driver +26/04/01 05:57:29 INFO TaskSetManager: Starting task 14.0 in stage 31.0 (TID 525) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:29 INFO Executor: Running task 14.0 in stage 31.0 (TID 525) +26/04/01 05:57:29 INFO TaskSetManager: Finished task 11.0 in stage 31.0 (TID 522) in 1623 ms on 10.0.0.133 (executor driver) (11/29) +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 200 (63.3 MiB) non-empty blocks including 200 (63.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 4 (325.9 KiB) non-empty blocks including 4 (325.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 4 (325.9 KiB) non-empty blocks including 4 (325.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:29 INFO Executor: Finished task 10.0 in stage 31.0 (TID 521). 19220 bytes result sent to driver +26/04/01 05:57:29 INFO TaskSetManager: Starting task 15.0 in stage 31.0 (TID 526) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:29 INFO TaskSetManager: Finished task 10.0 in stage 31.0 (TID 521) in 1651 ms on 10.0.0.133 (executor driver) (12/29) +26/04/01 05:57:29 INFO Executor: Running task 15.0 in stage 31.0 (TID 526) +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 200 (63.1 MiB) non-empty blocks including 200 (63.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 4 (329.2 KiB) non-empty blocks including 4 (329.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:31 INFO Executor: Finished task 12.0 in stage 31.0 (TID 523). 19177 bytes result sent to driver +26/04/01 05:57:31 INFO TaskSetManager: Starting task 16.0 in stage 31.0 (TID 527) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:31 INFO Executor: Running task 16.0 in stage 31.0 (TID 527) +26/04/01 05:57:31 INFO TaskSetManager: Finished task 12.0 in stage 31.0 (TID 523) in 1618 ms on 10.0.0.133 (executor driver) (13/29) +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 200 (63.3 MiB) non-empty blocks including 200 (63.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 4 (323.6 KiB) non-empty blocks including 4 (323.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:31 INFO Executor: Finished task 13.0 in stage 31.0 (TID 524). 19177 bytes result sent to driver +26/04/01 05:57:31 INFO TaskSetManager: Starting task 17.0 in stage 31.0 (TID 528) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:31 INFO Executor: Running task 17.0 in stage 31.0 (TID 528) +26/04/01 05:57:31 INFO TaskSetManager: Finished task 13.0 in stage 31.0 (TID 524) in 1635 ms on 10.0.0.133 (executor driver) (14/29) +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 4 (330.3 KiB) non-empty blocks including 4 (330.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:31 INFO Executor: Finished task 14.0 in stage 31.0 (TID 525). 19177 bytes result sent to driver +26/04/01 05:57:31 INFO TaskSetManager: Starting task 18.0 in stage 31.0 (TID 529) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:31 INFO Executor: Running task 18.0 in stage 31.0 (TID 529) +26/04/01 05:57:31 INFO TaskSetManager: Finished task 14.0 in stage 31.0 (TID 525) in 1651 ms on 10.0.0.133 (executor driver) (15/29) +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 4 (327.0 KiB) non-empty blocks including 4 (327.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:31 INFO Executor: Finished task 15.0 in stage 31.0 (TID 526). 19177 bytes result sent to driver +26/04/01 05:57:31 INFO TaskSetManager: Starting task 19.0 in stage 31.0 (TID 530) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:31 INFO TaskSetManager: Finished task 15.0 in stage 31.0 (TID 526) in 1636 ms on 10.0.0.133 (executor driver) (16/29) +26/04/01 05:57:31 INFO Executor: Running task 19.0 in stage 31.0 (TID 530) +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 200 (63.1 MiB) non-empty blocks including 200 (63.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 4 (322.5 KiB) non-empty blocks including 4 (322.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:32 INFO Executor: Finished task 16.0 in stage 31.0 (TID 527). 19220 bytes result sent to driver +26/04/01 05:57:32 INFO TaskSetManager: Starting task 20.0 in stage 31.0 (TID 531) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:32 INFO Executor: Running task 20.0 in stage 31.0 (TID 531) +26/04/01 05:57:32 INFO TaskSetManager: Finished task 16.0 in stage 31.0 (TID 527) in 1670 ms on 10.0.0.133 (executor driver) (17/29) +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:32 INFO Executor: Finished task 17.0 in stage 31.0 (TID 528). 19220 bytes result sent to driver +26/04/01 05:57:32 INFO TaskSetManager: Starting task 21.0 in stage 31.0 (TID 532) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:32 INFO Executor: Running task 21.0 in stage 31.0 (TID 532) +26/04/01 05:57:32 INFO TaskSetManager: Finished task 17.0 in stage 31.0 (TID 528) in 1647 ms on 10.0.0.133 (executor driver) (18/29) +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 4 (321.4 KiB) non-empty blocks including 4 (321.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 4 (320.3 KiB) non-empty blocks including 4 (320.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:32 INFO Executor: Finished task 19.0 in stage 31.0 (TID 530). 19220 bytes result sent to driver +26/04/01 05:57:32 INFO TaskSetManager: Starting task 22.0 in stage 31.0 (TID 533) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:32 INFO Executor: Running task 22.0 in stage 31.0 (TID 533) +26/04/01 05:57:32 INFO TaskSetManager: Finished task 19.0 in stage 31.0 (TID 530) in 1645 ms on 10.0.0.133 (executor driver) (19/29) +26/04/01 05:57:32 INFO Executor: Finished task 18.0 in stage 31.0 (TID 529). 19220 bytes result sent to driver +26/04/01 05:57:32 INFO TaskSetManager: Starting task 23.0 in stage 31.0 (TID 534) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:32 INFO Executor: Running task 23.0 in stage 31.0 (TID 534) +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:32 INFO TaskSetManager: Finished task 18.0 in stage 31.0 (TID 529) in 1658 ms on 10.0.0.133 (executor driver) (20/29) +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 4 (328.1 KiB) non-empty blocks including 4 (328.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 4 (325.9 KiB) non-empty blocks including 4 (325.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:34 INFO Executor: Finished task 21.0 in stage 31.0 (TID 532). 19177 bytes result sent to driver +26/04/01 05:57:34 INFO TaskSetManager: Starting task 24.0 in stage 31.0 (TID 535) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:34 INFO Executor: Running task 24.0 in stage 31.0 (TID 535) +26/04/01 05:57:34 INFO TaskSetManager: Finished task 21.0 in stage 31.0 (TID 532) in 1538 ms on 10.0.0.133 (executor driver) (21/29) +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 200 (63.1 MiB) non-empty blocks including 200 (63.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:34 INFO Executor: Finished task 20.0 in stage 31.0 (TID 531). 19177 bytes result sent to driver +26/04/01 05:57:34 INFO TaskSetManager: Starting task 25.0 in stage 31.0 (TID 536) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:34 INFO Executor: Running task 25.0 in stage 31.0 (TID 536) +26/04/01 05:57:34 INFO TaskSetManager: Finished task 20.0 in stage 31.0 (TID 531) in 1545 ms on 10.0.0.133 (executor driver) (22/29) +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 4 (321.4 KiB) non-empty blocks including 4 (321.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 4 (327.0 KiB) non-empty blocks including 4 (327.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:34 INFO Executor: Finished task 22.0 in stage 31.0 (TID 533). 19177 bytes result sent to driver +26/04/01 05:57:34 INFO TaskSetManager: Starting task 26.0 in stage 31.0 (TID 537) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:34 INFO TaskSetManager: Finished task 22.0 in stage 31.0 (TID 533) in 1531 ms on 10.0.0.133 (executor driver) (23/29) +26/04/01 05:57:34 INFO Executor: Running task 26.0 in stage 31.0 (TID 537) +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 4 (327.0 KiB) non-empty blocks including 4 (327.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:34 INFO Executor: Finished task 23.0 in stage 31.0 (TID 534). 19177 bytes result sent to driver +26/04/01 05:57:34 INFO TaskSetManager: Starting task 27.0 in stage 31.0 (TID 538) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:34 INFO TaskSetManager: Finished task 23.0 in stage 31.0 (TID 534) in 1538 ms on 10.0.0.133 (executor driver) (24/29) +26/04/01 05:57:34 INFO Executor: Running task 27.0 in stage 31.0 (TID 538) +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 200 (63.3 MiB) non-empty blocks including 200 (63.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 4 (330.3 KiB) non-empty blocks including 4 (330.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:35 INFO Executor: Finished task 24.0 in stage 31.0 (TID 535). 19177 bytes result sent to driver +26/04/01 05:57:35 INFO TaskSetManager: Starting task 28.0 in stage 31.0 (TID 539) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9270 bytes) +26/04/01 05:57:35 INFO Executor: Running task 28.0 in stage 31.0 (TID 539) +26/04/01 05:57:35 INFO TaskSetManager: Finished task 24.0 in stage 31.0 (TID 535) in 1493 ms on 10.0.0.133 (executor driver) (25/29) +26/04/01 05:57:35 INFO ShuffleBlockFetcherIterator: Getting 200 (36.1 MiB) non-empty blocks including 200 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:35 INFO ShuffleBlockFetcherIterator: Getting 4 (187.9 KiB) non-empty blocks including 4 (187.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:35 INFO Executor: Finished task 25.0 in stage 31.0 (TID 536). 19177 bytes result sent to driver +26/04/01 05:57:35 INFO TaskSetManager: Finished task 25.0 in stage 31.0 (TID 536) in 1514 ms on 10.0.0.133 (executor driver) (26/29) +26/04/01 05:57:35 INFO Executor: Finished task 26.0 in stage 31.0 (TID 537). 19177 bytes result sent to driver +26/04/01 05:57:35 INFO TaskSetManager: Finished task 26.0 in stage 31.0 (TID 537) in 1512 ms on 10.0.0.133 (executor driver) (27/29) +26/04/01 05:57:35 INFO Executor: Finished task 27.0 in stage 31.0 (TID 538). 19177 bytes result sent to driver +26/04/01 05:57:35 INFO TaskSetManager: Finished task 27.0 in stage 31.0 (TID 538) in 1509 ms on 10.0.0.133 (executor driver) (28/29) +26/04/01 05:57:36 INFO Executor: Finished task 28.0 in stage 31.0 (TID 539). 19177 bytes result sent to driver +26/04/01 05:57:36 INFO TaskSetManager: Finished task 28.0 in stage 31.0 (TID 539) in 638 ms on 10.0.0.133 (executor driver) (29/29) +26/04/01 05:57:36 INFO TaskSchedulerImpl: Removed TaskSet 31.0, whose tasks have all completed, from pool +26/04/01 05:57:36 INFO DAGScheduler: ShuffleMapStage 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 12.067 s +26/04/01 05:57:36 INFO DAGScheduler: looking for newly runnable stages +26/04/01 05:57:36 INFO DAGScheduler: running: Set() +26/04/01 05:57:36 INFO DAGScheduler: waiting: Set() +26/04/01 05:57:36 INFO DAGScheduler: failed: Set() +26/04/01 05:57:36 INFO ShufflePartitionsUtil: For shuffle(7), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 05:57:36 INFO HashAggregateExec: spark.sql.codegen.aggregate.map.twolevel.enabled is set to true, but current version of codegened fast hashmap does not support this aggregate. +26/04/01 05:57:36 INFO CodeGenerator: Code generated in 3.514166 ms +26/04/01 05:57:36 INFO CodeGenerator: Code generated in 1.259 ms +26/04/01 05:57:36 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:57:36 INFO DAGScheduler: Got job 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 05:57:36 INFO DAGScheduler: Final stage: ResultStage 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:57:36 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 38) +26/04/01 05:57:36 INFO DAGScheduler: Missing parents: List() +26/04/01 05:57:36 INFO DAGScheduler: Submitting ResultStage 39 (MapPartitionsRDD[73] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:57:36 WARN DAGScheduler: Broadcasting large task binary with size 1133.8 KiB +26/04/01 05:57:36 INFO MemoryStore: Block broadcast_28 stored as values in memory (estimated size 1133.8 KiB, free 8.6 GiB) +26/04/01 05:57:36 INFO MemoryStore: Block broadcast_28_piece0 stored as bytes in memory (estimated size 51.1 KiB, free 8.6 GiB) +26/04/01 05:57:36 INFO BlockManagerInfo: Added broadcast_28_piece0 in memory on 10.0.0.133:58206 (size: 51.1 KiB, free: 8.6 GiB) +26/04/01 05:57:36 INFO SparkContext: Created broadcast 28 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:57:36 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 39 (MapPartitionsRDD[73] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:57:36 INFO TaskSchedulerImpl: Adding task set 39.0 with 1 tasks resource profile 0 +26/04/01 05:57:36 INFO TaskSetManager: Starting task 0.0 in stage 39.0 (TID 540) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8999 bytes) +26/04/01 05:57:36 INFO Executor: Running task 0.0 in stage 39.0 (TID 540) +26/04/01 05:57:36 INFO ShuffleBlockFetcherIterator: Getting 29 (14.0 KiB) non-empty blocks including 29 (14.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:36 INFO CodeGenerator: Code generated in 3.10925 ms +26/04/01 05:57:36 INFO CodeGenerator: Code generated in 1.063625 ms +26/04/01 05:57:36 INFO Executor: Finished task 0.0 in stage 39.0 (TID 540). 20812 bytes result sent to driver +26/04/01 05:57:36 INFO TaskSetManager: Finished task 0.0 in stage 39.0 (TID 540) in 12 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:57:36 INFO TaskSchedulerImpl: Removed TaskSet 39.0, whose tasks have all completed, from pool +26/04/01 05:57:36 INFO DAGScheduler: ResultStage 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.015 s +26/04/01 05:57:36 INFO DAGScheduler: Job 19 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:57:36 INFO TaskSchedulerImpl: Killing all running tasks in stage 39: Stage finished +26/04/01 05:57:36 INFO DAGScheduler: Job 19 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.016231 s +26/04/01 05:57:36 INFO DAGScheduler: Registering RDD 74 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 8 +26/04/01 05:57:36 INFO DAGScheduler: Got map stage job 20 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 05:57:36 INFO DAGScheduler: Final stage: ShuffleMapStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:57:36 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 46) +26/04/01 05:57:36 INFO DAGScheduler: Missing parents: List() +26/04/01 05:57:36 INFO DAGScheduler: Submitting ShuffleMapStage 47 (MapPartitionsRDD[74] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:57:36 WARN DAGScheduler: Broadcasting large task binary with size 1134.5 KiB +26/04/01 05:57:36 INFO MemoryStore: Block broadcast_29 stored as values in memory (estimated size 1134.5 KiB, free 8.6 GiB) +26/04/01 05:57:36 INFO MemoryStore: Block broadcast_29_piece0 stored as bytes in memory (estimated size 51.7 KiB, free 8.6 GiB) +26/04/01 05:57:36 INFO BlockManagerInfo: Added broadcast_29_piece0 in memory on 10.0.0.133:58206 (size: 51.7 KiB, free: 8.6 GiB) +26/04/01 05:57:36 INFO SparkContext: Created broadcast 29 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:57:36 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 47 (MapPartitionsRDD[74] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:57:36 INFO TaskSchedulerImpl: Adding task set 47.0 with 1 tasks resource profile 0 +26/04/01 05:57:36 INFO TaskSetManager: Starting task 0.0 in stage 47.0 (TID 541) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8988 bytes) +26/04/01 05:57:36 INFO Executor: Running task 0.0 in stage 47.0 (TID 541) +26/04/01 05:57:36 INFO CodeGenerator: Code generated in 2.003417 ms +26/04/01 05:57:36 INFO ShuffleBlockFetcherIterator: Getting 29 (14.0 KiB) non-empty blocks including 29 (14.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:36 INFO Executor: Finished task 0.0 in stage 47.0 (TID 541). 20603 bytes result sent to driver +26/04/01 05:57:36 INFO TaskSetManager: Finished task 0.0 in stage 47.0 (TID 541) in 9 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:57:36 INFO TaskSchedulerImpl: Removed TaskSet 47.0, whose tasks have all completed, from pool +26/04/01 05:57:36 INFO DAGScheduler: ShuffleMapStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.013 s +26/04/01 05:57:36 INFO DAGScheduler: looking for newly runnable stages +26/04/01 05:57:36 INFO DAGScheduler: running: Set() +26/04/01 05:57:36 INFO DAGScheduler: waiting: Set() +26/04/01 05:57:36 INFO DAGScheduler: failed: Set() +26/04/01 05:57:36 INFO ShufflePartitionsUtil: For shuffle(8), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 05:57:36 INFO CodeGenerator: Code generated in 1.812209 ms +26/04/01 05:57:36 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 05:57:36 INFO DAGScheduler: Got job 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 05:57:36 INFO DAGScheduler: Final stage: ResultStage 56 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 05:57:36 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 55) +26/04/01 05:57:36 INFO DAGScheduler: Missing parents: List() +26/04/01 05:57:36 INFO DAGScheduler: Submitting ResultStage 56 (MapPartitionsRDD[77] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 05:57:36 WARN DAGScheduler: Broadcasting large task binary with size 1129.3 KiB +26/04/01 05:57:36 INFO MemoryStore: Block broadcast_30 stored as values in memory (estimated size 1129.3 KiB, free 8.6 GiB) +26/04/01 05:57:36 INFO MemoryStore: Block broadcast_30_piece0 stored as bytes in memory (estimated size 49.8 KiB, free 8.6 GiB) +26/04/01 05:57:36 INFO BlockManagerInfo: Added broadcast_30_piece0 in memory on 10.0.0.133:58206 (size: 49.8 KiB, free: 8.6 GiB) +26/04/01 05:57:36 INFO SparkContext: Created broadcast 30 from broadcast at DAGScheduler.scala:1611 +26/04/01 05:57:36 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 56 (MapPartitionsRDD[77] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 05:57:36 INFO TaskSchedulerImpl: Adding task set 56.0 with 1 tasks resource profile 0 +26/04/01 05:57:36 INFO TaskSetManager: Starting task 0.0 in stage 56.0 (TID 542) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8999 bytes) +26/04/01 05:57:36 INFO Executor: Running task 0.0 in stage 56.0 (TID 542) +26/04/01 05:57:36 INFO ShuffleBlockFetcherIterator: Getting 1 (494.0 B) non-empty blocks including 1 (494.0 B) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 05:57:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 05:57:36 INFO CodeGenerator: Code generated in 1.718833 ms +26/04/01 05:57:36 INFO CodeGenerator: Code generated in 1.094666 ms +26/04/01 05:57:36 INFO CodeGenerator: Code generated in 1.436 ms +26/04/01 05:57:36 INFO Executor: Finished task 0.0 in stage 56.0 (TID 542). 22060 bytes result sent to driver +26/04/01 05:57:36 INFO TaskSetManager: Finished task 0.0 in stage 56.0 (TID 542) in 8 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 05:57:36 INFO TaskSchedulerImpl: Removed TaskSet 56.0, whose tasks have all completed, from pool +26/04/01 05:57:36 INFO DAGScheduler: ResultStage 56 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.011 s +26/04/01 05:57:36 INFO DAGScheduler: Job 21 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 05:57:36 INFO TaskSchedulerImpl: Killing all running tasks in stage 56: Stage finished +26/04/01 05:57:36 INFO DAGScheduler: Job 21 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.012059 s +26/04/01 05:57:36 INFO SparkContext: SparkContext is stopping with exitCode 0. +26/04/01 05:57:36 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! +26/04/01 05:57:37 INFO MemoryStore: MemoryStore cleared +26/04/01 05:57:37 INFO BlockManager: BlockManager stopped +26/04/01 05:57:37 INFO BlockManagerMaster: BlockManagerMaster stopped +26/04/01 05:57:37 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! +26/04/01 05:57:37 INFO SparkContext: Successfully stopped SparkContext +26/04/01 05:57:37 INFO ShutdownHookManager: Shutdown hook called +26/04/01 05:57:37 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-88f9396d-516c-4d9d-9b8e-7e76a0cb2fd6 +26/04/01 05:57:37 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-dc11a204-3745-4760-9e95-7bbc7e262fc3 +26/04/01 05:57:37 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-dc11a204-3745-4760-9e95-7bbc7e262fc3/pyspark-1df01b8d-5da9-49b5-bce4-fc6a2c0407d4 + 87.48 real 294.71 user 39.61 sys + 5418713088 maximum resident set size + 0 average shared memory size + 0 average unshared data size + 0 average unshared stack size + 1246378 page reclaims + 37 page faults + 0 swaps + 0 block input operations + 0 block output operations + 1014 messages sent + 1031 messages received + 137 signals received + 25107 voluntary context switches + 312151 involuntary context switches + 4230771037201 instructions retired + 1213690734800 cycles elapsed + 5398664160 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.log b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.log new file mode 100644 index 0000000000..c3d9e20f8c --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.log @@ -0,0 +1,290 @@ +Registering table customer from /opt/tpch/sf100/customer +Registering table lineitem from /opt/tpch/sf100/lineitem +Registering table nation from /opt/tpch/sf100/nation +Registering table orders from /opt/tpch/sf100/orders +Registering table part from /opt/tpch/sf100/part +Registering table partsupp from /opt/tpch/sf100/partsupp +Registering table region from /opt/tpch/sf100/region +Registering table supplier from /opt/tpch/sf100/supplier + +============================================================ +Starting iteration 1 of 1 +============================================================ + +Running query 9 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q9.sql +Executing: -- CometBench-H query 9 derived from TPC-H query 9 under the terms of the TPC Fair Use Policy. +-- TP... +== Physical Plan == +AdaptiveSparkPlan (46) ++- Sort (45) + +- Exchange (44) + +- HashAggregate (43) + +- Exchange (42) + +- HashAggregate (41) + +- Project (40) + +- BroadcastHashJoin Inner BuildRight (39) + :- Project (35) + : +- SortMergeJoin Inner (34) + : :- Sort (29) + : : +- Exchange (28) + : : +- Project (27) + : : +- SortMergeJoin Inner (26) + : : :- Sort (21) + : : : +- Exchange (20) + : : : +- Project (19) + : : : +- SortMergeJoin Inner (18) + : : : :- Sort (13) + : : : : +- Exchange (12) + : : : : +- Project (11) + : : : : +- SortMergeJoin Inner (10) + : : : : :- Sort (5) + : : : : : +- Exchange (4) + : : : : : +- Project (3) + : : : : : +- Filter (2) + : : : : : +- Scan parquet (1) + : : : : +- Sort (9) + : : : : +- Exchange (8) + : : : : +- Filter (7) + : : : : +- Scan parquet (6) + : : : +- Sort (17) + : : : +- Exchange (16) + : : : +- Filter (15) + : : : +- Scan parquet (14) + : : +- Sort (25) + : : +- Exchange (24) + : : +- Filter (23) + : : +- Scan parquet (22) + : +- Sort (33) + : +- Exchange (32) + : +- Filter (31) + : +- Scan parquet (30) + +- BroadcastExchange (38) + +- Filter (37) + +- Scan parquet (36) + + +(1) Scan parquet +Output [2]: [p_partkey#74L, p_name#75] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/part] +PushedFilters: [IsNotNull(p_name), StringContains(p_name,moccasin), IsNotNull(p_partkey)] +ReadSchema: struct + +(2) Filter +Input [2]: [p_partkey#74L, p_name#75] +Condition : ((isnotnull(p_name#75) AND Contains(p_name#75, moccasin)) AND isnotnull(p_partkey#74L)) + +(3) Project +Output [1]: [p_partkey#74L] +Input [2]: [p_partkey#74L, p_name#75] + +(4) Exchange +Input [1]: [p_partkey#74L] +Arguments: hashpartitioning(p_partkey#74L, 200), ENSURE_REQUIREMENTS, [plan_id=137] + +(5) Sort +Input [1]: [p_partkey#74L] +Arguments: [p_partkey#74L ASC NULLS FIRST], false, 0 + +(6) Scan parquet +Output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] +PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] +ReadSchema: struct + +(7) Filter +Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Condition : ((isnotnull(l_partkey#17L) AND isnotnull(l_suppkey#18L)) AND isnotnull(l_orderkey#16L)) + +(8) Exchange +Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: hashpartitioning(l_partkey#17L, 200), ENSURE_REQUIREMENTS, [plan_id=138] + +(9) Sort +Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: [l_partkey#17L ASC NULLS FIRST], false, 0 + +(10) SortMergeJoin +Left keys [1]: [p_partkey#74L] +Right keys [1]: [l_partkey#17L] +Join type: Inner +Join condition: None + +(11) Project +Output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Input [7]: [p_partkey#74L, l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] + +(12) Exchange +Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: hashpartitioning(l_suppkey#18L, 200), ENSURE_REQUIREMENTS, [plan_id=145] + +(13) Sort +Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] +Arguments: [l_suppkey#18L ASC NULLS FIRST], false, 0 + +(14) Scan parquet +Output [2]: [s_suppkey#108L, s_nationkey#111L] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/supplier] +PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] +ReadSchema: struct + +(15) Filter +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Condition : (isnotnull(s_suppkey#108L) AND isnotnull(s_nationkey#111L)) + +(16) Exchange +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: hashpartitioning(s_suppkey#108L, 200), ENSURE_REQUIREMENTS, [plan_id=146] + +(17) Sort +Input [2]: [s_suppkey#108L, s_nationkey#111L] +Arguments: [s_suppkey#108L ASC NULLS FIRST], false, 0 + +(18) SortMergeJoin +Left keys [1]: [l_suppkey#18L] +Right keys [1]: [s_suppkey#108L] +Join type: Inner +Join condition: None + +(19) Project +Output [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] +Input [8]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_suppkey#108L, s_nationkey#111L] + +(20) Exchange +Input [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] +Arguments: hashpartitioning(l_suppkey#18L, l_partkey#17L, 200), ENSURE_REQUIREMENTS, [plan_id=154] + +(21) Sort +Input [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] +Arguments: [l_suppkey#18L ASC NULLS FIRST, l_partkey#17L ASC NULLS FIRST], false, 0 + +(22) Scan parquet +Output [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/partsupp] +PushedFilters: [IsNotNull(ps_suppkey), IsNotNull(ps_partkey)] +ReadSchema: struct + +(23) Filter +Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Condition : (isnotnull(ps_suppkey#93L) AND isnotnull(ps_partkey#92L)) + +(24) Exchange +Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Arguments: hashpartitioning(ps_suppkey#93L, ps_partkey#92L, 200), ENSURE_REQUIREMENTS, [plan_id=153] + +(25) Sort +Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] +Arguments: [ps_suppkey#93L ASC NULLS FIRST, ps_partkey#92L ASC NULLS FIRST], false, 0 + +(26) SortMergeJoin +Left keys [2]: [l_suppkey#18L, l_partkey#17L] +Right keys [2]: [ps_suppkey#93L, ps_partkey#92L] +Join type: Inner +Join condition: None + +(27) Project +Output [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] +Input [10]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] + +(28) Exchange +Input [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] +Arguments: hashpartitioning(l_orderkey#16L, 200), ENSURE_REQUIREMENTS, [plan_id=161] + +(29) Sort +Input [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] +Arguments: [l_orderkey#16L ASC NULLS FIRST], false, 0 + +(30) Scan parquet +Output [2]: [o_orderkey#56L, o_orderdate#60] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/orders] +PushedFilters: [IsNotNull(o_orderkey)] +ReadSchema: struct + +(31) Filter +Input [2]: [o_orderkey#56L, o_orderdate#60] +Condition : isnotnull(o_orderkey#56L) + +(32) Exchange +Input [2]: [o_orderkey#56L, o_orderdate#60] +Arguments: hashpartitioning(o_orderkey#56L, 200), ENSURE_REQUIREMENTS, [plan_id=162] + +(33) Sort +Input [2]: [o_orderkey#56L, o_orderdate#60] +Arguments: [o_orderkey#56L ASC NULLS FIRST], false, 0 + +(34) SortMergeJoin +Left keys [1]: [l_orderkey#16L] +Right keys [1]: [o_orderkey#56L] +Join type: Inner +Join condition: None + +(35) Project +Output [6]: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60] +Input [8]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderkey#56L, o_orderdate#60] + +(36) Scan parquet +Output [2]: [n_nationkey#48L, n_name#49] +Batched: true +Location: InMemoryFileIndex [file:/opt/tpch/sf100/nation] +PushedFilters: [IsNotNull(n_nationkey)] +ReadSchema: struct + +(37) Filter +Input [2]: [n_nationkey#48L, n_name#49] +Condition : isnotnull(n_nationkey#48L) + +(38) BroadcastExchange +Input [2]: [n_nationkey#48L, n_name#49] +Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [plan_id=168] + +(39) BroadcastHashJoin +Left keys [1]: [s_nationkey#111L] +Right keys [1]: [n_nationkey#48L] +Join type: Inner +Join condition: None + +(40) Project +Output [3]: [n_name#49 AS nation#122, year(o_orderdate#60) AS o_year#123, ((l_extendedprice#21 * (1 - l_discount#22)) - (ps_supplycost#95 * l_quantity#20)) AS amount#124] +Input [8]: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60, n_nationkey#48L, n_name#49] + +(41) HashAggregate +Input [3]: [nation#122, o_year#123, amount#124] +Keys [2]: [nation#122, o_year#123] +Functions [1]: [partial_sum(amount#124)] +Aggregate Attributes [2]: [sum#130, isEmpty#131] +Results [4]: [nation#122, o_year#123, sum#132, isEmpty#133] + +(42) Exchange +Input [4]: [nation#122, o_year#123, sum#132, isEmpty#133] +Arguments: hashpartitioning(nation#122, o_year#123, 200), ENSURE_REQUIREMENTS, [plan_id=173] + +(43) HashAggregate +Input [4]: [nation#122, o_year#123, sum#132, isEmpty#133] +Keys [2]: [nation#122, o_year#123] +Functions [1]: [sum(amount#124)] +Aggregate Attributes [1]: [sum(amount#124)#126] +Results [3]: [nation#122, o_year#123, sum(amount#124)#126 AS sum_profit#125] + +(44) Exchange +Input [3]: [nation#122, o_year#123, sum_profit#125] +Arguments: rangepartitioning(nation#122 ASC NULLS FIRST, o_year#123 DESC NULLS LAST, 200), ENSURE_REQUIREMENTS, [plan_id=176] + +(45) Sort +Input [3]: [nation#122, o_year#123, sum_profit#125] +Arguments: [nation#122 ASC NULLS FIRST, o_year#123 DESC NULLS LAST], true, 0 + +(46) AdaptiveSparkPlan +Output [3]: [nation#122, o_year#123, sum_profit#125] +Arguments: isFinalPlan=false + + +Query 9 returned 175 rows, hash=fff4ba3023e74505f304fa3243cbeeb6 +Query 9 took 79.93 seconds + +Iteration 1 took 79.93 seconds + +Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/spark-offheap4g-q9-tpch-1775054366701.json diff --git a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.time b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.time new file mode 100644 index 0000000000..85ad524fcd --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.time @@ -0,0 +1,5067 @@ +26/04/01 08:38:04 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) +26/04/01 08:38:04 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address +26/04/01 08:38:04 INFO SparkContext: Running Spark version 3.5.8 +26/04/01 08:38:04 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 08:38:04 INFO SparkContext: Java version 17.0.17 +26/04/01 08:38:04 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable +26/04/01 08:38:04 INFO ResourceUtils: ============================================================== +26/04/01 08:38:04 INFO ResourceUtils: No custom resources configured for spark.driver. +26/04/01 08:38:04 INFO ResourceUtils: ============================================================== +26/04/01 08:38:04 INFO SparkContext: Submitted application: spark-offheap4g-q9 benchmark derived from tpch +26/04/01 08:38:04 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 4096, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) +26/04/01 08:38:04 INFO ResourceProfile: Limiting resource is cpu +26/04/01 08:38:04 INFO ResourceProfileManager: Added ResourceProfile id: 0 +26/04/01 08:38:04 INFO SecurityManager: Changing view acls to: andy +26/04/01 08:38:04 INFO SecurityManager: Changing modify acls to: andy +26/04/01 08:38:04 INFO SecurityManager: Changing view acls groups to: +26/04/01 08:38:04 INFO SecurityManager: Changing modify acls groups to: +26/04/01 08:38:04 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY +26/04/01 08:38:05 INFO Utils: Successfully started service 'sparkDriver' on port 59287. +26/04/01 08:38:05 INFO SparkEnv: Registering MapOutputTracker +26/04/01 08:38:05 INFO SparkEnv: Registering BlockManagerMaster +26/04/01 08:38:05 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information +26/04/01 08:38:05 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up +26/04/01 08:38:05 INFO SparkEnv: Registering BlockManagerMasterHeartbeat +26/04/01 08:38:05 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-f0da0939-da02-4fb4-99f2-68ea3c8dd138 +26/04/01 08:38:05 INFO MemoryStore: MemoryStore started with capacity 8.6 GiB +26/04/01 08:38:05 INFO SparkEnv: Registering OutputCommitCoordinator +26/04/01 08:38:05 INFO Executor: Starting executor ID driver on host 10.0.0.133 +26/04/01 08:38:05 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 +26/04/01 08:38:05 INFO Executor: Java version 17.0.17 +26/04/01 08:38:05 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' +26/04/01 08:38:05 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@73312b9e for default. +26/04/01 08:38:05 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 59288. +26/04/01 08:38:05 INFO NettyBlockTransferService: Server created on 10.0.0.133:59288 +26/04/01 08:38:05 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy +26/04/01 08:38:05 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 59288, None) +26/04/01 08:38:05 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:59288 with 8.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 59288, None) +26/04/01 08:38:05 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 59288, None) +26/04/01 08:38:05 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 59288, None) +26/04/01 08:38:05 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. +26/04/01 08:38:05 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. +26/04/01 08:38:05 INFO InMemoryFileIndex: It took 21 ms to list leaf files for 1 paths. +26/04/01 08:38:05 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:38:05 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:38:05 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:38:05 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:38:05 INFO DAGScheduler: Missing parents: List() +26/04/01 08:38:05 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:38:05 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 08:38:05 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 08:38:05 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:05 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:38:05 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:38:05 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 +26/04/01 08:38:05 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) +26/04/01 08:38:05 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) +26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 2041 bytes result sent to driver +26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 125 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool +26/04/01 08:38:06 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.412 s +26/04/01 08:38:06 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished +26/04/01 08:38:06 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.424579 s +26/04/01 08:38:06 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:06 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. +26/04/01 08:38:06 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:38:06 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:38:06 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:38:06 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:38:06 INFO DAGScheduler: Missing parents: List() +26/04/01 08:38:06 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:38:06 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 08:38:06 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 08:38:06 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:06 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:38:06 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:38:06 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 +26/04/01 08:38:06 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) +26/04/01 08:38:06 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) +26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2344 bytes result sent to driver +26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 13 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool +26/04/01 08:38:06 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.021 s +26/04/01 08:38:06 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished +26/04/01 08:38:06 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.023024 s +26/04/01 08:38:06 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 08:38:06 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:38:06 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:38:06 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:38:06 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:38:06 INFO DAGScheduler: Missing parents: List() +26/04/01 08:38:06 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:38:06 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 08:38:06 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 08:38:06 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:06 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:38:06 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:38:06 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 +26/04/01 08:38:06 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) +26/04/01 08:38:06 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) +26/04/01 08:38:06 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1803 bytes result sent to driver +26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 9 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool +26/04/01 08:38:06 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.016 s +26/04/01 08:38:06 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished +26/04/01 08:38:06 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.018151 s +26/04/01 08:38:06 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. +26/04/01 08:38:06 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:38:06 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:38:06 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:38:06 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:38:06 INFO DAGScheduler: Missing parents: List() +26/04/01 08:38:06 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:38:06 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 08:38:06 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 08:38:06 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:06 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:38:06 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:38:06 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 +26/04/01 08:38:06 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) +26/04/01 08:38:06 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) +26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver +26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool +26/04/01 08:38:06 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s +26/04/01 08:38:06 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished +26/04/01 08:38:06 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.016603 s +26/04/01 08:38:06 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. +26/04/01 08:38:06 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:38:06 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:38:06 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:38:06 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:38:06 INFO DAGScheduler: Missing parents: List() +26/04/01 08:38:06 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:38:06 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 08:38:06 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 08:38:06 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:06 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:38:06 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:38:06 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 +26/04/01 08:38:06 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9134 bytes) +26/04/01 08:38:06 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) +26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver +26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool +26/04/01 08:38:06 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 08:38:06 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished +26/04/01 08:38:06 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.014998 s +26/04/01 08:38:06 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 08:38:06 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:38:06 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:38:06 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:38:06 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:38:06 INFO DAGScheduler: Missing parents: List() +26/04/01 08:38:06 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:38:06 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 08:38:06 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 08:38:06 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:06 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:06 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:38:06 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:38:06 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 +26/04/01 08:38:06 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) +26/04/01 08:38:06 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:06 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) +26/04/01 08:38:06 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver +26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool +26/04/01 08:38:06 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s +26/04/01 08:38:06 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished +26/04/01 08:38:06 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.015529 s +26/04/01 08:38:06 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. +26/04/01 08:38:06 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:38:06 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:38:06 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:38:06 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:38:06 INFO DAGScheduler: Missing parents: List() +26/04/01 08:38:06 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:38:06 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 08:38:06 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 08:38:06 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:06 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:06 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:38:06 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:38:06 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 +26/04/01 08:38:06 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) +26/04/01 08:38:06 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) +26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver +26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool +26/04/01 08:38:06 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s +26/04/01 08:38:06 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished +26/04/01 08:38:06 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.014484 s +26/04/01 08:38:06 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. +26/04/01 08:38:06 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 +26/04/01 08:38:06 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions +26/04/01 08:38:06 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) +26/04/01 08:38:06 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:38:06 INFO DAGScheduler: Missing parents: List() +26/04/01 08:38:06 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents +26/04/01 08:38:06 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) +26/04/01 08:38:06 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) +26/04/01 08:38:06 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:06 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:38:06 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:38:06 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 +26/04/01 08:38:06 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) +26/04/01 08:38:06 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) +26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver +26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 6 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool +26/04/01 08:38:06 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.012 s +26/04/01 08:38:06 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished +26/04/01 08:38:06 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.013930 s +26/04/01 08:38:06 INFO FileSourceStrategy: Pushed Filters: IsNotNull(p_name),StringContains(p_name,moccasin),IsNotNull(p_partkey) +26/04/01 08:38:06 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(p_name#75),Contains(p_name#75, moccasin),isnotnull(p_partkey#74L) +26/04/01 08:38:06 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_partkey),IsNotNull(l_suppkey),IsNotNull(l_orderkey) +26/04/01 08:38:06 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_partkey#17L),isnotnull(l_suppkey#18L),isnotnull(l_orderkey#16L) +26/04/01 08:38:06 INFO FileSourceStrategy: Pushed Filters: IsNotNull(s_suppkey),IsNotNull(s_nationkey) +26/04/01 08:38:06 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(s_suppkey#108L),isnotnull(s_nationkey#111L) +26/04/01 08:38:06 INFO FileSourceStrategy: Pushed Filters: IsNotNull(ps_suppkey),IsNotNull(ps_partkey) +26/04/01 08:38:06 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(ps_suppkey#93L),isnotnull(ps_partkey#92L) +26/04/01 08:38:06 INFO FileSourceStrategy: Pushed Filters: IsNotNull(o_orderkey) +26/04/01 08:38:06 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(o_orderkey#56L) +26/04/01 08:38:06 INFO FileSourceStrategy: Pushed Filters: IsNotNull(n_nationkey) +26/04/01 08:38:06 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(n_nationkey#48L) +26/04/01 08:38:07 INFO CodeGenerator: Code generated in 70.492667 ms +26/04/01 08:38:07 INFO CodeGenerator: Code generated in 71.013458 ms +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:59288 (size: 35.0 KiB, free: 8.6 GiB) +26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:59288 (size: 35.0 KiB, free: 8.6 GiB) +26/04/01 08:38:07 INFO SparkContext: Created broadcast 9 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:38:07 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:38:07 INFO FileSourceScanExec: Planning scan with bin packing, max size: 102898790 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:38:07 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:38:07 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:07 INFO DAGScheduler: Registering RDD 23 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 +26/04/01 08:38:07 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) +26/04/01 08:38:07 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 8 output partitions +26/04/01 08:38:07 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:38:07 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:38:07 INFO DAGScheduler: Missing parents: List() +26/04/01 08:38:07 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[23] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:38:07 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 18.2 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 8.1 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:59288 (size: 8.1 KiB, free: 8.6 GiB) +26/04/01 08:38:07 INFO SparkContext: Created broadcast 10 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:38:07 INFO DAGScheduler: Submitting 8 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[23] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7)) +26/04/01 08:38:07 INFO TaskSchedulerImpl: Adding task set 8.0 with 8 tasks resource profile 0 +26/04/01 08:38:07 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 08:38:07 INFO DAGScheduler: Final stage: ResultStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:38:07 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:38:07 INFO DAGScheduler: Missing parents: List() +26/04/01 08:38:07 INFO DAGScheduler: Submitting ResultStage 9 (MapPartitionsRDD[22] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:38:07 INFO CodeGenerator: Code generated in 12.31725 ms +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 15.2 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9891 bytes) +26/04/01 08:38:07 INFO TaskSetManager: Starting task 1.0 in stage 8.0 (TID 9) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9893 bytes) +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 6.5 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO TaskSetManager: Starting task 2.0 in stage 8.0 (TID 10) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9893 bytes) +26/04/01 08:38:07 INFO TaskSetManager: Starting task 3.0 in stage 8.0 (TID 11) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9893 bytes) +26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:59288 (size: 6.5 KiB, free: 8.6 GiB) +26/04/01 08:38:07 INFO TaskSetManager: Starting task 4.0 in stage 8.0 (TID 12) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9892 bytes) +26/04/01 08:38:07 INFO TaskSetManager: Starting task 5.0 in stage 8.0 (TID 13) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9894 bytes) +26/04/01 08:38:07 INFO TaskSetManager: Starting task 6.0 in stage 8.0 (TID 14) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9893 bytes) +26/04/01 08:38:07 INFO SparkContext: Created broadcast 11 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:38:07 INFO TaskSetManager: Starting task 7.0 in stage 8.0 (TID 15) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9894 bytes) +26/04/01 08:38:07 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 9 (MapPartitionsRDD[22] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:38:07 INFO TaskSchedulerImpl: Adding task set 9.0 with 1 tasks resource profile 0 +26/04/01 08:38:07 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) +26/04/01 08:38:07 INFO Executor: Running task 1.0 in stage 8.0 (TID 9) +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 202.3 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO Executor: Running task 2.0 in stage 8.0 (TID 10) +26/04/01 08:38:07 INFO Executor: Running task 3.0 in stage 8.0 (TID 11) +26/04/01 08:38:07 INFO Executor: Running task 4.0 in stage 8.0 (TID 12) +26/04/01 08:38:07 INFO Executor: Running task 5.0 in stage 8.0 (TID 13) +26/04/01 08:38:07 INFO Executor: Running task 6.0 in stage 8.0 (TID 14) +26/04/01 08:38:07 INFO Executor: Running task 7.0 in stage 8.0 (TID 15) +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 35.2 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:59288 (size: 35.2 KiB, free: 8.6 GiB) +26/04/01 08:38:07 INFO SparkContext: Created broadcast 12 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:38:07 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:38:07 INFO DAGScheduler: Registering RDD 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 +26/04/01 08:38:07 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions +26/04/01 08:38:07 INFO DAGScheduler: Final stage: ShuffleMapStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:38:07 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:38:07 INFO DAGScheduler: Missing parents: List() +26/04/01 08:38:07 INFO DAGScheduler: Submitting ShuffleMapStage 10 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_13 stored as values in memory (estimated size 20.4 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_13_piece0 stored as bytes in memory (estimated size 8.5 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_13_piece0 in memory on 10.0.0.133:59288 (size: 8.5 KiB, free: 8.6 GiB) +26/04/01 08:38:07 INFO SparkContext: Created broadcast 13 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:38:07 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 10 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:38:07 INFO TaskSchedulerImpl: Adding task set 10.0 with 208 tasks resource profile 0 +26/04/01 08:38:07 INFO CodeGenerator: Code generated in 6.977 ms +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_14 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_14_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_14_piece0 in memory on 10.0.0.133:59288 (size: 35.0 KiB, free: 8.6 GiB) +26/04/01 08:38:07 INFO SparkContext: Created broadcast 14 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:38:07 INFO FileSourceScanExec: Planning scan with bin packing, max size: 27960063 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:38:07 INFO CodeGenerator: Code generated in 5.3325 ms +26/04/01 08:38:07 INFO DAGScheduler: Registering RDD 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 2 +26/04/01 08:38:07 INFO DAGScheduler: Got map stage job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 8 output partitions +26/04/01 08:38:07 INFO DAGScheduler: Final stage: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:38:07 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:38:07 INFO DAGScheduler: Missing parents: List() +26/04/01 08:38:07 INFO DAGScheduler: Submitting ShuffleMapStage 11 (MapPartitionsRDD[31] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_15 stored as values in memory (estimated size 17.7 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_15_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_15_piece0 in memory on 10.0.0.133:59288 (size: 7.9 KiB, free: 8.6 GiB) +26/04/01 08:38:07 INFO SparkContext: Created broadcast 15 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:38:07 INFO DAGScheduler: Submitting 8 missing tasks from ShuffleMapStage 11 (MapPartitionsRDD[31] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7)) +26/04/01 08:38:07 INFO TaskSchedulerImpl: Adding task set 11.0 with 8 tasks resource profile 0 +26/04/01 08:38:07 INFO CodeGenerator: Code generated in 5.675916 ms +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_16 stored as values in memory (estimated size 201.9 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO CodeGenerator: Code generated in 5.802083 ms +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_16_piece0 stored as bytes in memory (estimated size 35.1 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_16_piece0 in memory on 10.0.0.133:59288 (size: 35.1 KiB, free: 8.6 GiB) +26/04/01 08:38:07 INFO SparkContext: Created broadcast 16 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:38:07 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.17.parquet, range: 0-21536094, partition values: [empty row] +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.19.parquet, range: 0-21528527, partition values: [empty row] +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.1.parquet, range: 0-21530360, partition values: [empty row] +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.8.parquet, range: 0-21539554, partition values: [empty row] +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.14.parquet, range: 0-21526139, partition values: [empty row] +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.3.parquet, range: 0-21532480, partition values: [empty row] +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.30.parquet, range: 0-21523460, partition values: [empty row] +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.6.parquet, range: 0-21534095, partition values: [empty row] +26/04/01 08:38:07 INFO DAGScheduler: Registering RDD 35 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 3 +26/04/01 08:38:07 INFO DAGScheduler: Got map stage job 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 37 output partitions +26/04/01 08:38:07 INFO DAGScheduler: Final stage: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:38:07 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:38:07 INFO DAGScheduler: Missing parents: List() +26/04/01 08:38:07 INFO DAGScheduler: Submitting ShuffleMapStage 12 (MapPartitionsRDD[35] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_17 stored as values in memory (estimated size 18.6 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_17_piece0 stored as bytes in memory (estimated size 8.2 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_17_piece0 in memory on 10.0.0.133:59288 (size: 8.2 KiB, free: 8.6 GiB) +26/04/01 08:38:07 INFO SparkContext: Created broadcast 17 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:38:07 INFO DAGScheduler: Submitting 37 missing tasks from ShuffleMapStage 12 (MapPartitionsRDD[35] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:38:07 INFO TaskSchedulerImpl: Adding task set 12.0 with 37 tasks resource profile 0 +26/04/01 08:38:07 INFO CodeGenerator: Code generated in 4.214833 ms +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_18 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_18_piece0 stored as bytes in memory (estimated size 35.1 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_18_piece0 in memory on 10.0.0.133:59288 (size: 35.1 KiB, free: 8.6 GiB) +26/04/01 08:38:07 INFO SparkContext: Created broadcast 18 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:38:07 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. +26/04/01 08:38:07 INFO DAGScheduler: Registering RDD 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 4 +26/04/01 08:38:07 INFO DAGScheduler: Got map stage job 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 64 output partitions +26/04/01 08:38:07 INFO DAGScheduler: Final stage: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:38:07 INFO DAGScheduler: Parents of final stage: List() +26/04/01 08:38:07 INFO DAGScheduler: Missing parents: List() +26/04/01 08:38:07 INFO DAGScheduler: Submitting ShuffleMapStage 13 (MapPartitionsRDD[39] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_19 stored as values in memory (estimated size 17.8 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO MemoryStore: Block broadcast_19_piece0 stored as bytes in memory (estimated size 8.0 KiB, free 8.6 GiB) +26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_19_piece0 in memory on 10.0.0.133:59288 (size: 8.0 KiB, free: 8.6 GiB) +26/04/01 08:38:07 INFO SparkContext: Created broadcast 19 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:38:07 INFO DAGScheduler: Submitting 64 missing tasks from ShuffleMapStage 13 (MapPartitionsRDD[39] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:38:07 INFO TaskSchedulerImpl: Adding task set 13.0 with 64 tasks resource profile 0 +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@d2db547)), noteq(p_partkey, null)) +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@2bc93fdf)), noteq(p_partkey, null)) +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@741d8897)), noteq(p_partkey, null)) +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@703f69d4)), noteq(p_partkey, null)) +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@74395742)), noteq(p_partkey, null)) +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@341dd6e7)), noteq(p_partkey, null)) +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@230b76c2)), noteq(p_partkey, null)) +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@e76cdec)), noteq(p_partkey, null)) +26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.13.parquet, range: 0-21529438, partition values: [empty row] +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.22.parquet, range: 0-21531915, partition values: [empty row] +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.5.parquet, range: 0-21524978, partition values: [empty row] +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@3cea020e)), noteq(p_partkey, null)) +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@556f4016)), noteq(p_partkey, null)) +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.16.parquet, range: 0-21533858, partition values: [empty row] +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.28.parquet, range: 0-21538351, partition values: [empty row] +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.10.parquet, range: 0-21523089, partition values: [empty row] +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.18.parquet, range: 0-21535985, partition values: [empty row] +26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.20.parquet, range: 0-21528010, partition values: [empty row] +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@5a63b0e4)), noteq(p_partkey, null)) +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@5af179ca)), noteq(p_partkey, null)) +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@4dd47702)), noteq(p_partkey, null)) +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@7b24ed9e)), noteq(p_partkey, null)) +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@51a83230)), noteq(p_partkey, null)) +26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@f77f37d)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.11.parquet, range: 0-21529268, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.12.parquet, range: 0-21523797, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.9.parquet, range: 0-21537495, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.25.parquet, range: 0-21531839, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.27.parquet, range: 0-21527815, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.4.parquet, range: 0-21535415, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.31.parquet, range: 0-21533701, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.24.parquet, range: 0-21520965, partition values: [empty row] +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@43866115)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@529ae805)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@56949e42)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@7985d4fa)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@3a186a9e)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@6a92709e)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@7d20947c)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@41766d96)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.23.parquet, range: 0-21531357, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.7.parquet, range: 0-21528633, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.29.parquet, range: 0-21523744, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.32.parquet, range: 0-21526986, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.15.parquet, range: 0-21520086, partition values: [empty row] +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@335cdde4)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.2.parquet, range: 0-21537447, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.26.parquet, range: 0-21533511, partition values: [empty row] +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@5945f1c7)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.21.parquet, range: 0-21534200, partition values: [empty row] +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@b6cdab2)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@a478e01)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@54ba13c4)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@50a9de9d)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@3772abc4)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@46d046b1)), noteq(p_partkey, null)) +26/04/01 08:38:08 INFO Executor: Finished task 6.0 in stage 8.0 (TID 14). 2294 bytes result sent to driver +26/04/01 08:38:08 INFO Executor: Finished task 3.0 in stage 8.0 (TID 11). 2294 bytes result sent to driver +26/04/01 08:38:08 INFO TaskSetManager: Starting task 0.0 in stage 9.0 (TID 16) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9608 bytes) +26/04/01 08:38:08 INFO Executor: Finished task 5.0 in stage 8.0 (TID 13). 2294 bytes result sent to driver +26/04/01 08:38:08 INFO Executor: Running task 0.0 in stage 9.0 (TID 16) +26/04/01 08:38:08 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 17) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:08 INFO TaskSetManager: Finished task 6.0 in stage 8.0 (TID 14) in 1095 ms on 10.0.0.133 (executor driver) (1/8) +26/04/01 08:38:08 INFO Executor: Running task 0.0 in stage 10.0 (TID 17) +26/04/01 08:38:08 INFO TaskSetManager: Starting task 1.0 in stage 10.0 (TID 18) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:08 INFO TaskSetManager: Finished task 3.0 in stage 8.0 (TID 11) in 1096 ms on 10.0.0.133 (executor driver) (2/8) +26/04/01 08:38:08 INFO TaskSetManager: Finished task 5.0 in stage 8.0 (TID 13) in 1096 ms on 10.0.0.133 (executor driver) (3/8) +26/04/01 08:38:08 INFO Executor: Running task 1.0 in stage 10.0 (TID 18) +26/04/01 08:38:08 INFO Executor: Finished task 7.0 in stage 8.0 (TID 15). 2294 bytes result sent to driver +26/04/01 08:38:08 INFO TaskSetManager: Starting task 2.0 in stage 10.0 (TID 19) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:08 INFO Executor: Running task 2.0 in stage 10.0 (TID 19) +26/04/01 08:38:08 INFO TaskSetManager: Finished task 7.0 in stage 8.0 (TID 15) in 1097 ms on 10.0.0.133 (executor driver) (4/8) +26/04/01 08:38:08 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 2294 bytes result sent to driver +26/04/01 08:38:08 INFO TaskSetManager: Starting task 3.0 in stage 10.0 (TID 20) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:08 INFO Executor: Running task 3.0 in stage 10.0 (TID 20) +26/04/01 08:38:08 INFO Executor: Finished task 1.0 in stage 8.0 (TID 9). 2294 bytes result sent to driver +26/04/01 08:38:08 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 1101 ms on 10.0.0.133 (executor driver) (5/8) +26/04/01 08:38:08 INFO TaskSetManager: Starting task 4.0 in stage 10.0 (TID 21) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:08 INFO TaskSetManager: Finished task 1.0 in stage 8.0 (TID 9) in 1100 ms on 10.0.0.133 (executor driver) (6/8) +26/04/01 08:38:08 INFO Executor: Running task 4.0 in stage 10.0 (TID 21) +26/04/01 08:38:08 INFO Executor: Finished task 4.0 in stage 8.0 (TID 12). 2294 bytes result sent to driver +26/04/01 08:38:08 INFO TaskSetManager: Starting task 5.0 in stage 10.0 (TID 22) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:08 INFO Executor: Running task 5.0 in stage 10.0 (TID 22) +26/04/01 08:38:08 INFO TaskSetManager: Finished task 4.0 in stage 8.0 (TID 12) in 1100 ms on 10.0.0.133 (executor driver) (7/8) +26/04/01 08:38:08 INFO Executor: Finished task 2.0 in stage 8.0 (TID 10). 2294 bytes result sent to driver +26/04/01 08:38:08 INFO TaskSetManager: Starting task 6.0 in stage 10.0 (TID 23) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:08 INFO Executor: Running task 6.0 in stage 10.0 (TID 23) +26/04/01 08:38:08 INFO TaskSetManager: Finished task 2.0 in stage 8.0 (TID 10) in 1102 ms on 10.0.0.133 (executor driver) (8/8) +26/04/01 08:38:08 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool +26/04/01 08:38:08 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 1.111 s +26/04/01 08:38:08 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:38:08 INFO DAGScheduler: running: Set(ShuffleMapStage 12, ResultStage 9, ShuffleMapStage 13, ShuffleMapStage 10, ShuffleMapStage 11) +26/04/01 08:38:08 INFO DAGScheduler: waiting: Set() +26/04/01 08:38:08 INFO DAGScheduler: failed: Set() +26/04/01 08:38:08 INFO CodeGenerator: Code generated in 7.670084 ms +26/04/01 08:38:08 INFO CodeGenerator: Code generated in 10.964417 ms +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/nation/nation.1.parquet, range: 0-2670, partition values: [empty row] +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: noteq(n_nationkey, null) +26/04/01 08:38:08 INFO Executor: Finished task 0.0 in stage 9.0 (TID 16). 2376 bytes result sent to driver +26/04/01 08:38:08 INFO TaskSetManager: Starting task 7.0 in stage 10.0 (TID 24) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:08 INFO Executor: Running task 7.0 in stage 10.0 (TID 24) +26/04/01 08:38:08 INFO TaskSetManager: Finished task 0.0 in stage 9.0 (TID 16) in 21 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:38:08 INFO TaskSchedulerImpl: Removed TaskSet 9.0, whose tasks have all completed, from pool +26/04/01 08:38:08 INFO DAGScheduler: ResultStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 1.116 s +26/04/01 08:38:08 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:38:08 INFO TaskSchedulerImpl: Killing all running tasks in stage 9: Stage finished +26/04/01 08:38:08 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 1.122861 s +26/04/01 08:38:08 INFO CodeGenerator: Code generated in 5.074958 ms +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:08 INFO CodeGenerator: Code generated in 3.289791 ms +26/04/01 08:38:08 INFO MemoryStore: Block broadcast_20 stored as values in memory (estimated size 1024.2 KiB, free 8.6 GiB) +26/04/01 08:38:08 INFO MemoryStore: Block broadcast_20_piece0 stored as bytes in memory (estimated size 828.0 B, free 8.6 GiB) +26/04/01 08:38:08 INFO BlockManagerInfo: Added broadcast_20_piece0 in memory on 10.0.0.133:59288 (size: 828.0 B, free: 8.6 GiB) +26/04/01 08:38:08 INFO SparkContext: Created broadcast 20 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:09 INFO Executor: Finished task 5.0 in stage 10.0 (TID 22). 2294 bytes result sent to driver +26/04/01 08:38:09 INFO TaskSetManager: Starting task 8.0 in stage 10.0 (TID 25) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:09 INFO Executor: Running task 8.0 in stage 10.0 (TID 25) +26/04/01 08:38:09 INFO TaskSetManager: Finished task 5.0 in stage 10.0 (TID 22) in 1260 ms on 10.0.0.133 (executor driver) (1/208) +26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:09 INFO Executor: Finished task 4.0 in stage 10.0 (TID 21). 2294 bytes result sent to driver +26/04/01 08:38:09 INFO TaskSetManager: Starting task 9.0 in stage 10.0 (TID 26) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:09 INFO Executor: Running task 9.0 in stage 10.0 (TID 26) +26/04/01 08:38:09 INFO TaskSetManager: Finished task 4.0 in stage 10.0 (TID 21) in 1452 ms on 10.0.0.133 (executor driver) (2/208) +26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:09 INFO Executor: Finished task 7.0 in stage 10.0 (TID 24). 2294 bytes result sent to driver +26/04/01 08:38:09 INFO TaskSetManager: Starting task 10.0 in stage 10.0 (TID 27) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:09 INFO TaskSetManager: Finished task 7.0 in stage 10.0 (TID 24) in 1443 ms on 10.0.0.133 (executor driver) (3/208) +26/04/01 08:38:09 INFO Executor: Running task 10.0 in stage 10.0 (TID 27) +26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:09 INFO Executor: Finished task 6.0 in stage 10.0 (TID 23). 2294 bytes result sent to driver +26/04/01 08:38:09 INFO TaskSetManager: Starting task 11.0 in stage 10.0 (TID 28) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:09 INFO TaskSetManager: Finished task 6.0 in stage 10.0 (TID 23) in 1461 ms on 10.0.0.133 (executor driver) (4/208) +26/04/01 08:38:09 INFO Executor: Running task 11.0 in stage 10.0 (TID 28) +26/04/01 08:38:09 INFO Executor: Finished task 3.0 in stage 10.0 (TID 20). 2294 bytes result sent to driver +26/04/01 08:38:09 INFO TaskSetManager: Starting task 12.0 in stage 10.0 (TID 29) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:09 INFO Executor: Running task 12.0 in stage 10.0 (TID 29) +26/04/01 08:38:09 INFO TaskSetManager: Finished task 3.0 in stage 10.0 (TID 20) in 1465 ms on 10.0.0.133 (executor driver) (5/208) +26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:09 INFO Executor: Finished task 1.0 in stage 10.0 (TID 18). 2294 bytes result sent to driver +26/04/01 08:38:09 INFO TaskSetManager: Starting task 13.0 in stage 10.0 (TID 30) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:09 INFO TaskSetManager: Finished task 1.0 in stage 10.0 (TID 18) in 1469 ms on 10.0.0.133 (executor driver) (6/208) +26/04/01 08:38:09 INFO Executor: Finished task 0.0 in stage 10.0 (TID 17). 2294 bytes result sent to driver +26/04/01 08:38:09 INFO Executor: Running task 13.0 in stage 10.0 (TID 30) +26/04/01 08:38:09 INFO TaskSetManager: Starting task 14.0 in stage 10.0 (TID 31) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:09 INFO Executor: Running task 14.0 in stage 10.0 (TID 31) +26/04/01 08:38:09 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 17) in 1470 ms on 10.0.0.133 (executor driver) (7/208) +26/04/01 08:38:09 INFO Executor: Finished task 2.0 in stage 10.0 (TID 19). 2294 bytes result sent to driver +26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:09 INFO TaskSetManager: Starting task 15.0 in stage 10.0 (TID 32) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:09 INFO Executor: Running task 15.0 in stage 10.0 (TID 32) +26/04/01 08:38:09 INFO TaskSetManager: Finished task 2.0 in stage 10.0 (TID 19) in 1468 ms on 10.0.0.133 (executor driver) (8/208) +26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:10 INFO Executor: Finished task 8.0 in stage 10.0 (TID 25). 2294 bytes result sent to driver +26/04/01 08:38:10 INFO TaskSetManager: Starting task 16.0 in stage 10.0 (TID 33) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:10 INFO Executor: Running task 16.0 in stage 10.0 (TID 33) +26/04/01 08:38:10 INFO TaskSetManager: Finished task 8.0 in stage 10.0 (TID 25) in 1083 ms on 10.0.0.133 (executor driver) (9/208) +26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:10 INFO Executor: Finished task 9.0 in stage 10.0 (TID 26). 2294 bytes result sent to driver +26/04/01 08:38:10 INFO TaskSetManager: Starting task 17.0 in stage 10.0 (TID 34) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:10 INFO TaskSetManager: Finished task 9.0 in stage 10.0 (TID 26) in 1044 ms on 10.0.0.133 (executor driver) (10/208) +26/04/01 08:38:10 INFO Executor: Running task 17.0 in stage 10.0 (TID 34) +26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:10 INFO Executor: Finished task 10.0 in stage 10.0 (TID 27). 2294 bytes result sent to driver +26/04/01 08:38:10 INFO TaskSetManager: Starting task 18.0 in stage 10.0 (TID 35) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:10 INFO TaskSetManager: Finished task 10.0 in stage 10.0 (TID 27) in 1100 ms on 10.0.0.133 (executor driver) (11/208) +26/04/01 08:38:10 INFO Executor: Running task 18.0 in stage 10.0 (TID 35) +26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:10 INFO Executor: Finished task 14.0 in stage 10.0 (TID 31). 2251 bytes result sent to driver +26/04/01 08:38:10 INFO TaskSetManager: Starting task 19.0 in stage 10.0 (TID 36) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:10 INFO TaskSetManager: Finished task 14.0 in stage 10.0 (TID 31) in 1186 ms on 10.0.0.133 (executor driver) (12/208) +26/04/01 08:38:10 INFO Executor: Running task 19.0 in stage 10.0 (TID 36) +26/04/01 08:38:10 INFO Executor: Finished task 12.0 in stage 10.0 (TID 29). 2251 bytes result sent to driver +26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:10 INFO TaskSetManager: Starting task 20.0 in stage 10.0 (TID 37) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:10 INFO TaskSetManager: Finished task 12.0 in stage 10.0 (TID 29) in 1190 ms on 10.0.0.133 (executor driver) (13/208) +26/04/01 08:38:10 INFO Executor: Running task 20.0 in stage 10.0 (TID 37) +26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:10 INFO Executor: Finished task 11.0 in stage 10.0 (TID 28). 2251 bytes result sent to driver +26/04/01 08:38:10 INFO TaskSetManager: Starting task 21.0 in stage 10.0 (TID 38) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:10 INFO TaskSetManager: Finished task 11.0 in stage 10.0 (TID 28) in 1194 ms on 10.0.0.133 (executor driver) (14/208) +26/04/01 08:38:10 INFO Executor: Running task 21.0 in stage 10.0 (TID 38) +26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:10 INFO Executor: Finished task 13.0 in stage 10.0 (TID 30). 2294 bytes result sent to driver +26/04/01 08:38:10 INFO TaskSetManager: Starting task 22.0 in stage 10.0 (TID 39) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:10 INFO TaskSetManager: Finished task 13.0 in stage 10.0 (TID 30) in 1195 ms on 10.0.0.133 (executor driver) (15/208) +26/04/01 08:38:10 INFO Executor: Running task 22.0 in stage 10.0 (TID 39) +26/04/01 08:38:10 INFO Executor: Finished task 15.0 in stage 10.0 (TID 32). 2251 bytes result sent to driver +26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:10 INFO TaskSetManager: Starting task 23.0 in stage 10.0 (TID 40) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:10 INFO TaskSetManager: Finished task 15.0 in stage 10.0 (TID 32) in 1196 ms on 10.0.0.133 (executor driver) (16/208) +26/04/01 08:38:10 INFO Executor: Running task 23.0 in stage 10.0 (TID 40) +26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:11 INFO Executor: Finished task 16.0 in stage 10.0 (TID 33). 2251 bytes result sent to driver +26/04/01 08:38:11 INFO TaskSetManager: Starting task 24.0 in stage 10.0 (TID 41) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:11 INFO TaskSetManager: Finished task 16.0 in stage 10.0 (TID 33) in 1056 ms on 10.0.0.133 (executor driver) (17/208) +26/04/01 08:38:11 INFO Executor: Running task 24.0 in stage 10.0 (TID 41) +26/04/01 08:38:11 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:11 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:11 INFO Executor: Finished task 17.0 in stage 10.0 (TID 34). 2251 bytes result sent to driver +26/04/01 08:38:11 INFO TaskSetManager: Starting task 25.0 in stage 10.0 (TID 42) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:11 INFO TaskSetManager: Finished task 17.0 in stage 10.0 (TID 34) in 1021 ms on 10.0.0.133 (executor driver) (18/208) +26/04/01 08:38:11 INFO Executor: Running task 25.0 in stage 10.0 (TID 42) +26/04/01 08:38:11 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:11 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:11 INFO Executor: Finished task 18.0 in stage 10.0 (TID 35). 2251 bytes result sent to driver +26/04/01 08:38:11 INFO TaskSetManager: Starting task 26.0 in stage 10.0 (TID 43) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:11 INFO TaskSetManager: Finished task 18.0 in stage 10.0 (TID 35) in 1036 ms on 10.0.0.133 (executor driver) (19/208) +26/04/01 08:38:11 INFO Executor: Running task 26.0 in stage 10.0 (TID 43) +26/04/01 08:38:11 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:11 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:12 INFO Executor: Finished task 19.0 in stage 10.0 (TID 36). 2251 bytes result sent to driver +26/04/01 08:38:12 INFO TaskSetManager: Starting task 27.0 in stage 10.0 (TID 44) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:12 INFO TaskSetManager: Finished task 19.0 in stage 10.0 (TID 36) in 1121 ms on 10.0.0.133 (executor driver) (20/208) +26/04/01 08:38:12 INFO Executor: Running task 27.0 in stage 10.0 (TID 44) +26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:12 INFO Executor: Finished task 23.0 in stage 10.0 (TID 40). 2251 bytes result sent to driver +26/04/01 08:38:12 INFO TaskSetManager: Starting task 28.0 in stage 10.0 (TID 45) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:12 INFO TaskSetManager: Finished task 23.0 in stage 10.0 (TID 40) in 1113 ms on 10.0.0.133 (executor driver) (21/208) +26/04/01 08:38:12 INFO Executor: Running task 28.0 in stage 10.0 (TID 45) +26/04/01 08:38:12 INFO Executor: Finished task 20.0 in stage 10.0 (TID 37). 2251 bytes result sent to driver +26/04/01 08:38:12 INFO TaskSetManager: Starting task 29.0 in stage 10.0 (TID 46) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:12 INFO TaskSetManager: Finished task 20.0 in stage 10.0 (TID 37) in 1123 ms on 10.0.0.133 (executor driver) (22/208) +26/04/01 08:38:12 INFO Executor: Running task 29.0 in stage 10.0 (TID 46) +26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:12 INFO Executor: Finished task 22.0 in stage 10.0 (TID 39). 2251 bytes result sent to driver +26/04/01 08:38:12 INFO TaskSetManager: Starting task 30.0 in stage 10.0 (TID 47) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:12 INFO TaskSetManager: Finished task 22.0 in stage 10.0 (TID 39) in 1120 ms on 10.0.0.133 (executor driver) (23/208) +26/04/01 08:38:12 INFO Executor: Running task 30.0 in stage 10.0 (TID 47) +26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:12 INFO Executor: Finished task 21.0 in stage 10.0 (TID 38). 2251 bytes result sent to driver +26/04/01 08:38:12 INFO TaskSetManager: Starting task 31.0 in stage 10.0 (TID 48) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:12 INFO Executor: Running task 31.0 in stage 10.0 (TID 48) +26/04/01 08:38:12 INFO TaskSetManager: Finished task 21.0 in stage 10.0 (TID 38) in 1130 ms on 10.0.0.133 (executor driver) (24/208) +26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:12 INFO Executor: Finished task 24.0 in stage 10.0 (TID 41). 2251 bytes result sent to driver +26/04/01 08:38:12 INFO TaskSetManager: Starting task 32.0 in stage 10.0 (TID 49) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:12 INFO TaskSetManager: Finished task 24.0 in stage 10.0 (TID 41) in 1017 ms on 10.0.0.133 (executor driver) (25/208) +26/04/01 08:38:12 INFO Executor: Running task 32.0 in stage 10.0 (TID 49) +26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:12 INFO Executor: Finished task 25.0 in stage 10.0 (TID 42). 2251 bytes result sent to driver +26/04/01 08:38:12 INFO TaskSetManager: Starting task 33.0 in stage 10.0 (TID 50) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:12 INFO TaskSetManager: Finished task 25.0 in stage 10.0 (TID 42) in 979 ms on 10.0.0.133 (executor driver) (26/208) +26/04/01 08:38:12 INFO Executor: Running task 33.0 in stage 10.0 (TID 50) +26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:12 INFO Executor: Finished task 26.0 in stage 10.0 (TID 43). 2251 bytes result sent to driver +26/04/01 08:38:12 INFO TaskSetManager: Starting task 34.0 in stage 10.0 (TID 51) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:12 INFO TaskSetManager: Finished task 26.0 in stage 10.0 (TID 43) in 975 ms on 10.0.0.133 (executor driver) (27/208) +26/04/01 08:38:12 INFO Executor: Running task 34.0 in stage 10.0 (TID 51) +26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:13 INFO Executor: Finished task 31.0 in stage 10.0 (TID 48). 2251 bytes result sent to driver +26/04/01 08:38:13 INFO TaskSetManager: Starting task 35.0 in stage 10.0 (TID 52) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:13 INFO TaskSetManager: Finished task 31.0 in stage 10.0 (TID 48) in 1054 ms on 10.0.0.133 (executor driver) (28/208) +26/04/01 08:38:13 INFO Executor: Running task 35.0 in stage 10.0 (TID 52) +26/04/01 08:38:13 INFO Executor: Finished task 27.0 in stage 10.0 (TID 44). 2251 bytes result sent to driver +26/04/01 08:38:13 INFO TaskSetManager: Starting task 36.0 in stage 10.0 (TID 53) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:13 INFO TaskSetManager: Finished task 27.0 in stage 10.0 (TID 44) in 1068 ms on 10.0.0.133 (executor driver) (29/208) +26/04/01 08:38:13 INFO Executor: Running task 36.0 in stage 10.0 (TID 53) +26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:13 INFO Executor: Finished task 30.0 in stage 10.0 (TID 47). 2251 bytes result sent to driver +26/04/01 08:38:13 INFO TaskSetManager: Starting task 37.0 in stage 10.0 (TID 54) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:13 INFO TaskSetManager: Finished task 30.0 in stage 10.0 (TID 47) in 1066 ms on 10.0.0.133 (executor driver) (30/208) +26/04/01 08:38:13 INFO Executor: Running task 37.0 in stage 10.0 (TID 54) +26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:13 INFO Executor: Finished task 28.0 in stage 10.0 (TID 45). 2251 bytes result sent to driver +26/04/01 08:38:13 INFO TaskSetManager: Starting task 38.0 in stage 10.0 (TID 55) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:13 INFO TaskSetManager: Finished task 28.0 in stage 10.0 (TID 45) in 1080 ms on 10.0.0.133 (executor driver) (31/208) +26/04/01 08:38:13 INFO Executor: Running task 38.0 in stage 10.0 (TID 55) +26/04/01 08:38:13 INFO Executor: Finished task 29.0 in stage 10.0 (TID 46). 2251 bytes result sent to driver +26/04/01 08:38:13 INFO TaskSetManager: Starting task 39.0 in stage 10.0 (TID 56) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:13 INFO TaskSetManager: Finished task 29.0 in stage 10.0 (TID 46) in 1079 ms on 10.0.0.133 (executor driver) (32/208) +26/04/01 08:38:13 INFO Executor: Running task 39.0 in stage 10.0 (TID 56) +26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:13 INFO Executor: Finished task 32.0 in stage 10.0 (TID 49). 2251 bytes result sent to driver +26/04/01 08:38:13 INFO TaskSetManager: Starting task 40.0 in stage 10.0 (TID 57) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:13 INFO TaskSetManager: Finished task 32.0 in stage 10.0 (TID 49) in 984 ms on 10.0.0.133 (executor driver) (33/208) +26/04/01 08:38:13 INFO Executor: Running task 40.0 in stage 10.0 (TID 57) +26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:13 INFO Executor: Finished task 33.0 in stage 10.0 (TID 50). 2251 bytes result sent to driver +26/04/01 08:38:13 INFO TaskSetManager: Starting task 41.0 in stage 10.0 (TID 58) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:13 INFO TaskSetManager: Finished task 33.0 in stage 10.0 (TID 50) in 1000 ms on 10.0.0.133 (executor driver) (34/208) +26/04/01 08:38:13 INFO Executor: Running task 41.0 in stage 10.0 (TID 58) +26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:13 INFO Executor: Finished task 34.0 in stage 10.0 (TID 51). 2251 bytes result sent to driver +26/04/01 08:38:13 INFO TaskSetManager: Starting task 42.0 in stage 10.0 (TID 59) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:13 INFO TaskSetManager: Finished task 34.0 in stage 10.0 (TID 51) in 972 ms on 10.0.0.133 (executor driver) (35/208) +26/04/01 08:38:13 INFO Executor: Running task 42.0 in stage 10.0 (TID 59) +26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:14 INFO Executor: Finished task 35.0 in stage 10.0 (TID 52). 2251 bytes result sent to driver +26/04/01 08:38:14 INFO TaskSetManager: Starting task 43.0 in stage 10.0 (TID 60) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:14 INFO Executor: Running task 43.0 in stage 10.0 (TID 60) +26/04/01 08:38:14 INFO TaskSetManager: Finished task 35.0 in stage 10.0 (TID 52) in 1003 ms on 10.0.0.133 (executor driver) (36/208) +26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:14 INFO Executor: Finished task 39.0 in stage 10.0 (TID 56). 2251 bytes result sent to driver +26/04/01 08:38:14 INFO TaskSetManager: Starting task 44.0 in stage 10.0 (TID 61) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:14 INFO TaskSetManager: Finished task 39.0 in stage 10.0 (TID 56) in 1054 ms on 10.0.0.133 (executor driver) (37/208) +26/04/01 08:38:14 INFO Executor: Running task 44.0 in stage 10.0 (TID 61) +26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:14 INFO Executor: Finished task 36.0 in stage 10.0 (TID 53). 2251 bytes result sent to driver +26/04/01 08:38:14 INFO TaskSetManager: Starting task 45.0 in stage 10.0 (TID 62) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:14 INFO TaskSetManager: Finished task 36.0 in stage 10.0 (TID 53) in 1072 ms on 10.0.0.133 (executor driver) (38/208) +26/04/01 08:38:14 INFO Executor: Running task 45.0 in stage 10.0 (TID 62) +26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:14 INFO Executor: Finished task 37.0 in stage 10.0 (TID 54). 2251 bytes result sent to driver +26/04/01 08:38:14 INFO TaskSetManager: Starting task 46.0 in stage 10.0 (TID 63) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:14 INFO TaskSetManager: Finished task 37.0 in stage 10.0 (TID 54) in 1069 ms on 10.0.0.133 (executor driver) (39/208) +26/04/01 08:38:14 INFO Executor: Running task 46.0 in stage 10.0 (TID 63) +26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:14 INFO Executor: Finished task 38.0 in stage 10.0 (TID 55). 2251 bytes result sent to driver +26/04/01 08:38:14 INFO TaskSetManager: Starting task 47.0 in stage 10.0 (TID 64) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:14 INFO TaskSetManager: Finished task 38.0 in stage 10.0 (TID 55) in 1062 ms on 10.0.0.133 (executor driver) (40/208) +26/04/01 08:38:14 INFO Executor: Running task 47.0 in stage 10.0 (TID 64) +26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:14 INFO Executor: Finished task 40.0 in stage 10.0 (TID 57). 2251 bytes result sent to driver +26/04/01 08:38:14 INFO TaskSetManager: Starting task 48.0 in stage 10.0 (TID 65) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:14 INFO TaskSetManager: Finished task 40.0 in stage 10.0 (TID 57) in 1022 ms on 10.0.0.133 (executor driver) (41/208) +26/04/01 08:38:14 INFO Executor: Running task 48.0 in stage 10.0 (TID 65) +26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:14 INFO Executor: Finished task 41.0 in stage 10.0 (TID 58). 2251 bytes result sent to driver +26/04/01 08:38:14 INFO TaskSetManager: Starting task 49.0 in stage 10.0 (TID 66) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:14 INFO TaskSetManager: Finished task 41.0 in stage 10.0 (TID 58) in 1041 ms on 10.0.0.133 (executor driver) (42/208) +26/04/01 08:38:14 INFO Executor: Running task 49.0 in stage 10.0 (TID 66) +26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:14 INFO Executor: Finished task 42.0 in stage 10.0 (TID 59). 2251 bytes result sent to driver +26/04/01 08:38:14 INFO TaskSetManager: Starting task 50.0 in stage 10.0 (TID 67) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:14 INFO TaskSetManager: Finished task 42.0 in stage 10.0 (TID 59) in 1028 ms on 10.0.0.133 (executor driver) (43/208) +26/04/01 08:38:14 INFO Executor: Running task 50.0 in stage 10.0 (TID 67) +26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:15 INFO Executor: Finished task 43.0 in stage 10.0 (TID 60). 2251 bytes result sent to driver +26/04/01 08:38:15 INFO TaskSetManager: Starting task 51.0 in stage 10.0 (TID 68) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:15 INFO TaskSetManager: Finished task 43.0 in stage 10.0 (TID 60) in 990 ms on 10.0.0.133 (executor driver) (44/208) +26/04/01 08:38:15 INFO Executor: Running task 51.0 in stage 10.0 (TID 68) +26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:15 INFO Executor: Finished task 47.0 in stage 10.0 (TID 64). 2251 bytes result sent to driver +26/04/01 08:38:15 INFO TaskSetManager: Starting task 52.0 in stage 10.0 (TID 69) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:15 INFO TaskSetManager: Finished task 47.0 in stage 10.0 (TID 64) in 1051 ms on 10.0.0.133 (executor driver) (45/208) +26/04/01 08:38:15 INFO Executor: Running task 52.0 in stage 10.0 (TID 69) +26/04/01 08:38:15 INFO Executor: Finished task 45.0 in stage 10.0 (TID 62). 2251 bytes result sent to driver +26/04/01 08:38:15 INFO TaskSetManager: Starting task 53.0 in stage 10.0 (TID 70) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:15 INFO TaskSetManager: Finished task 45.0 in stage 10.0 (TID 62) in 1055 ms on 10.0.0.133 (executor driver) (46/208) +26/04/01 08:38:15 INFO Executor: Running task 53.0 in stage 10.0 (TID 70) +26/04/01 08:38:15 INFO Executor: Finished task 44.0 in stage 10.0 (TID 61). 2251 bytes result sent to driver +26/04/01 08:38:15 INFO TaskSetManager: Starting task 54.0 in stage 10.0 (TID 71) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:15 INFO TaskSetManager: Finished task 44.0 in stage 10.0 (TID 61) in 1059 ms on 10.0.0.133 (executor driver) (47/208) +26/04/01 08:38:15 INFO Executor: Running task 54.0 in stage 10.0 (TID 71) +26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:15 INFO Executor: Finished task 46.0 in stage 10.0 (TID 63). 2251 bytes result sent to driver +26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:15 INFO TaskSetManager: Starting task 55.0 in stage 10.0 (TID 72) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:15 INFO TaskSetManager: Finished task 46.0 in stage 10.0 (TID 63) in 1056 ms on 10.0.0.133 (executor driver) (48/208) +26/04/01 08:38:15 INFO Executor: Running task 55.0 in stage 10.0 (TID 72) +26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:15 INFO Executor: Finished task 48.0 in stage 10.0 (TID 65). 2294 bytes result sent to driver +26/04/01 08:38:15 INFO TaskSetManager: Starting task 56.0 in stage 10.0 (TID 73) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:15 INFO Executor: Running task 56.0 in stage 10.0 (TID 73) +26/04/01 08:38:15 INFO TaskSetManager: Finished task 48.0 in stage 10.0 (TID 65) in 1023 ms on 10.0.0.133 (executor driver) (49/208) +26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:15 INFO Executor: Finished task 49.0 in stage 10.0 (TID 66). 2251 bytes result sent to driver +26/04/01 08:38:15 INFO TaskSetManager: Starting task 57.0 in stage 10.0 (TID 74) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:15 INFO TaskSetManager: Finished task 49.0 in stage 10.0 (TID 66) in 1006 ms on 10.0.0.133 (executor driver) (50/208) +26/04/01 08:38:15 INFO Executor: Running task 57.0 in stage 10.0 (TID 74) +26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:15 INFO Executor: Finished task 50.0 in stage 10.0 (TID 67). 2251 bytes result sent to driver +26/04/01 08:38:15 INFO TaskSetManager: Starting task 58.0 in stage 10.0 (TID 75) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:15 INFO Executor: Running task 58.0 in stage 10.0 (TID 75) +26/04/01 08:38:15 INFO TaskSetManager: Finished task 50.0 in stage 10.0 (TID 67) in 989 ms on 10.0.0.133 (executor driver) (51/208) +26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:16 INFO Executor: Finished task 51.0 in stage 10.0 (TID 68). 2251 bytes result sent to driver +26/04/01 08:38:16 INFO TaskSetManager: Starting task 59.0 in stage 10.0 (TID 76) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:16 INFO TaskSetManager: Finished task 51.0 in stage 10.0 (TID 68) in 1002 ms on 10.0.0.133 (executor driver) (52/208) +26/04/01 08:38:16 INFO Executor: Running task 59.0 in stage 10.0 (TID 76) +26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:16 INFO Executor: Finished task 53.0 in stage 10.0 (TID 70). 2251 bytes result sent to driver +26/04/01 08:38:16 INFO TaskSetManager: Starting task 60.0 in stage 10.0 (TID 77) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:16 INFO TaskSetManager: Finished task 53.0 in stage 10.0 (TID 70) in 1075 ms on 10.0.0.133 (executor driver) (53/208) +26/04/01 08:38:16 INFO Executor: Running task 60.0 in stage 10.0 (TID 77) +26/04/01 08:38:16 INFO Executor: Finished task 52.0 in stage 10.0 (TID 69). 2251 bytes result sent to driver +26/04/01 08:38:16 INFO TaskSetManager: Starting task 61.0 in stage 10.0 (TID 78) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:16 INFO Executor: Running task 61.0 in stage 10.0 (TID 78) +26/04/01 08:38:16 INFO TaskSetManager: Finished task 52.0 in stage 10.0 (TID 69) in 1077 ms on 10.0.0.133 (executor driver) (54/208) +26/04/01 08:38:16 INFO Executor: Finished task 55.0 in stage 10.0 (TID 72). 2251 bytes result sent to driver +26/04/01 08:38:16 INFO TaskSetManager: Starting task 62.0 in stage 10.0 (TID 79) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:16 INFO TaskSetManager: Finished task 55.0 in stage 10.0 (TID 72) in 1074 ms on 10.0.0.133 (executor driver) (55/208) +26/04/01 08:38:16 INFO Executor: Running task 62.0 in stage 10.0 (TID 79) +26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:16 INFO Executor: Finished task 54.0 in stage 10.0 (TID 71). 2251 bytes result sent to driver +26/04/01 08:38:16 INFO TaskSetManager: Starting task 63.0 in stage 10.0 (TID 80) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:16 INFO TaskSetManager: Finished task 54.0 in stage 10.0 (TID 71) in 1077 ms on 10.0.0.133 (executor driver) (56/208) +26/04/01 08:38:16 INFO Executor: Running task 63.0 in stage 10.0 (TID 80) +26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:16 INFO Executor: Finished task 56.0 in stage 10.0 (TID 73). 2251 bytes result sent to driver +26/04/01 08:38:16 INFO TaskSetManager: Starting task 64.0 in stage 10.0 (TID 81) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:16 INFO TaskSetManager: Finished task 56.0 in stage 10.0 (TID 73) in 1032 ms on 10.0.0.133 (executor driver) (57/208) +26/04/01 08:38:16 INFO Executor: Running task 64.0 in stage 10.0 (TID 81) +26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:16 INFO Executor: Finished task 57.0 in stage 10.0 (TID 74). 2251 bytes result sent to driver +26/04/01 08:38:16 INFO TaskSetManager: Starting task 65.0 in stage 10.0 (TID 82) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:16 INFO TaskSetManager: Finished task 57.0 in stage 10.0 (TID 74) in 1050 ms on 10.0.0.133 (executor driver) (58/208) +26/04/01 08:38:16 INFO Executor: Running task 65.0 in stage 10.0 (TID 82) +26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:16 INFO Executor: Finished task 58.0 in stage 10.0 (TID 75). 2251 bytes result sent to driver +26/04/01 08:38:16 INFO TaskSetManager: Starting task 66.0 in stage 10.0 (TID 83) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:16 INFO TaskSetManager: Finished task 58.0 in stage 10.0 (TID 75) in 1046 ms on 10.0.0.133 (executor driver) (59/208) +26/04/01 08:38:16 INFO Executor: Running task 66.0 in stage 10.0 (TID 83) +26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:17 INFO Executor: Finished task 59.0 in stage 10.0 (TID 76). 2251 bytes result sent to driver +26/04/01 08:38:17 INFO TaskSetManager: Starting task 67.0 in stage 10.0 (TID 84) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:17 INFO TaskSetManager: Finished task 59.0 in stage 10.0 (TID 76) in 1071 ms on 10.0.0.133 (executor driver) (60/208) +26/04/01 08:38:17 INFO Executor: Running task 67.0 in stage 10.0 (TID 84) +26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:17 INFO Executor: Finished task 63.0 in stage 10.0 (TID 80). 2251 bytes result sent to driver +26/04/01 08:38:17 INFO TaskSetManager: Starting task 68.0 in stage 10.0 (TID 85) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:17 INFO TaskSetManager: Finished task 63.0 in stage 10.0 (TID 80) in 1115 ms on 10.0.0.133 (executor driver) (61/208) +26/04/01 08:38:17 INFO Executor: Running task 68.0 in stage 10.0 (TID 85) +26/04/01 08:38:17 INFO Executor: Finished task 61.0 in stage 10.0 (TID 78). 2251 bytes result sent to driver +26/04/01 08:38:17 INFO TaskSetManager: Starting task 69.0 in stage 10.0 (TID 86) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:17 INFO TaskSetManager: Finished task 61.0 in stage 10.0 (TID 78) in 1119 ms on 10.0.0.133 (executor driver) (62/208) +26/04/01 08:38:17 INFO Executor: Running task 69.0 in stage 10.0 (TID 86) +26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:17 INFO Executor: Finished task 62.0 in stage 10.0 (TID 79). 2251 bytes result sent to driver +26/04/01 08:38:17 INFO TaskSetManager: Starting task 70.0 in stage 10.0 (TID 87) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:17 INFO TaskSetManager: Finished task 62.0 in stage 10.0 (TID 79) in 1121 ms on 10.0.0.133 (executor driver) (63/208) +26/04/01 08:38:17 INFO Executor: Running task 70.0 in stage 10.0 (TID 87) +26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:17 INFO Executor: Finished task 60.0 in stage 10.0 (TID 77). 2251 bytes result sent to driver +26/04/01 08:38:17 INFO TaskSetManager: Starting task 71.0 in stage 10.0 (TID 88) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:17 INFO TaskSetManager: Finished task 60.0 in stage 10.0 (TID 77) in 1129 ms on 10.0.0.133 (executor driver) (64/208) +26/04/01 08:38:17 INFO Executor: Running task 71.0 in stage 10.0 (TID 88) +26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:17 INFO Executor: Finished task 64.0 in stage 10.0 (TID 81). 2251 bytes result sent to driver +26/04/01 08:38:17 INFO TaskSetManager: Starting task 72.0 in stage 10.0 (TID 89) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:17 INFO TaskSetManager: Finished task 64.0 in stage 10.0 (TID 81) in 1065 ms on 10.0.0.133 (executor driver) (65/208) +26/04/01 08:38:17 INFO Executor: Running task 72.0 in stage 10.0 (TID 89) +26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:17 INFO Executor: Finished task 66.0 in stage 10.0 (TID 83). 2251 bytes result sent to driver +26/04/01 08:38:17 INFO TaskSetManager: Starting task 73.0 in stage 10.0 (TID 90) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:17 INFO Executor: Running task 73.0 in stage 10.0 (TID 90) +26/04/01 08:38:17 INFO TaskSetManager: Finished task 66.0 in stage 10.0 (TID 83) in 1076 ms on 10.0.0.133 (executor driver) (66/208) +26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:17 INFO Executor: Finished task 65.0 in stage 10.0 (TID 82). 2251 bytes result sent to driver +26/04/01 08:38:17 INFO TaskSetManager: Starting task 74.0 in stage 10.0 (TID 91) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:17 INFO TaskSetManager: Finished task 65.0 in stage 10.0 (TID 82) in 1096 ms on 10.0.0.133 (executor driver) (67/208) +26/04/01 08:38:17 INFO Executor: Running task 74.0 in stage 10.0 (TID 91) +26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:18 INFO Executor: Finished task 67.0 in stage 10.0 (TID 84). 2251 bytes result sent to driver +26/04/01 08:38:18 INFO TaskSetManager: Starting task 75.0 in stage 10.0 (TID 92) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:18 INFO TaskSetManager: Finished task 67.0 in stage 10.0 (TID 84) in 1097 ms on 10.0.0.133 (executor driver) (68/208) +26/04/01 08:38:18 INFO Executor: Running task 75.0 in stage 10.0 (TID 92) +26/04/01 08:38:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:18 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:18 INFO Executor: Finished task 69.0 in stage 10.0 (TID 86). 2251 bytes result sent to driver +26/04/01 08:38:18 INFO TaskSetManager: Starting task 76.0 in stage 10.0 (TID 93) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:18 INFO TaskSetManager: Finished task 69.0 in stage 10.0 (TID 86) in 1239 ms on 10.0.0.133 (executor driver) (69/208) +26/04/01 08:38:18 INFO Executor: Running task 76.0 in stage 10.0 (TID 93) +26/04/01 08:38:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:18 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:18 INFO Executor: Finished task 71.0 in stage 10.0 (TID 88). 2251 bytes result sent to driver +26/04/01 08:38:18 INFO TaskSetManager: Starting task 77.0 in stage 10.0 (TID 94) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:18 INFO Executor: Finished task 70.0 in stage 10.0 (TID 87). 2251 bytes result sent to driver +26/04/01 08:38:18 INFO TaskSetManager: Finished task 71.0 in stage 10.0 (TID 88) in 1247 ms on 10.0.0.133 (executor driver) (70/208) +26/04/01 08:38:18 INFO Executor: Running task 77.0 in stage 10.0 (TID 94) +26/04/01 08:38:18 INFO TaskSetManager: Starting task 78.0 in stage 10.0 (TID 95) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:18 INFO TaskSetManager: Finished task 70.0 in stage 10.0 (TID 87) in 1255 ms on 10.0.0.133 (executor driver) (71/208) +26/04/01 08:38:18 INFO Executor: Running task 78.0 in stage 10.0 (TID 95) +26/04/01 08:38:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:18 INFO Executor: Finished task 68.0 in stage 10.0 (TID 85). 2251 bytes result sent to driver +26/04/01 08:38:18 INFO TaskSetManager: Starting task 79.0 in stage 10.0 (TID 96) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:18 INFO TaskSetManager: Finished task 68.0 in stage 10.0 (TID 85) in 1265 ms on 10.0.0.133 (executor driver) (72/208) +26/04/01 08:38:18 INFO Executor: Running task 79.0 in stage 10.0 (TID 96) +26/04/01 08:38:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:18 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:18 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:18 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:18 INFO Executor: Finished task 72.0 in stage 10.0 (TID 89). 2251 bytes result sent to driver +26/04/01 08:38:18 INFO TaskSetManager: Starting task 80.0 in stage 10.0 (TID 97) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:18 INFO TaskSetManager: Finished task 72.0 in stage 10.0 (TID 89) in 1122 ms on 10.0.0.133 (executor driver) (73/208) +26/04/01 08:38:18 INFO Executor: Running task 80.0 in stage 10.0 (TID 97) +26/04/01 08:38:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:18 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:19 INFO Executor: Finished task 73.0 in stage 10.0 (TID 90). 2251 bytes result sent to driver +26/04/01 08:38:19 INFO TaskSetManager: Starting task 81.0 in stage 10.0 (TID 98) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:19 INFO TaskSetManager: Finished task 73.0 in stage 10.0 (TID 90) in 1088 ms on 10.0.0.133 (executor driver) (74/208) +26/04/01 08:38:19 INFO Executor: Running task 81.0 in stage 10.0 (TID 98) +26/04/01 08:38:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:19 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:19 INFO Executor: Finished task 74.0 in stage 10.0 (TID 91). 2251 bytes result sent to driver +26/04/01 08:38:19 INFO TaskSetManager: Starting task 82.0 in stage 10.0 (TID 99) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:19 INFO TaskSetManager: Finished task 74.0 in stage 10.0 (TID 91) in 1177 ms on 10.0.0.133 (executor driver) (75/208) +26/04/01 08:38:19 INFO Executor: Running task 82.0 in stage 10.0 (TID 99) +26/04/01 08:38:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:19 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:19 INFO Executor: Finished task 75.0 in stage 10.0 (TID 92). 2251 bytes result sent to driver +26/04/01 08:38:19 INFO TaskSetManager: Starting task 83.0 in stage 10.0 (TID 100) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:19 INFO TaskSetManager: Finished task 75.0 in stage 10.0 (TID 92) in 1173 ms on 10.0.0.133 (executor driver) (76/208) +26/04/01 08:38:19 INFO Executor: Running task 83.0 in stage 10.0 (TID 100) +26/04/01 08:38:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:19 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:19 INFO Executor: Finished task 76.0 in stage 10.0 (TID 93). 2251 bytes result sent to driver +26/04/01 08:38:19 INFO TaskSetManager: Starting task 84.0 in stage 10.0 (TID 101) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:19 INFO Executor: Running task 84.0 in stage 10.0 (TID 101) +26/04/01 08:38:19 INFO TaskSetManager: Finished task 76.0 in stage 10.0 (TID 93) in 1106 ms on 10.0.0.133 (executor driver) (77/208) +26/04/01 08:38:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:19 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:19 INFO Executor: Finished task 77.0 in stage 10.0 (TID 94). 2251 bytes result sent to driver +26/04/01 08:38:19 INFO TaskSetManager: Starting task 85.0 in stage 10.0 (TID 102) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:19 INFO TaskSetManager: Finished task 77.0 in stage 10.0 (TID 94) in 1156 ms on 10.0.0.133 (executor driver) (78/208) +26/04/01 08:38:19 INFO Executor: Running task 85.0 in stage 10.0 (TID 102) +26/04/01 08:38:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:19 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:19 INFO Executor: Finished task 79.0 in stage 10.0 (TID 96). 2251 bytes result sent to driver +26/04/01 08:38:19 INFO TaskSetManager: Starting task 86.0 in stage 10.0 (TID 103) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:19 INFO TaskSetManager: Finished task 79.0 in stage 10.0 (TID 96) in 1169 ms on 10.0.0.133 (executor driver) (79/208) +26/04/01 08:38:19 INFO Executor: Running task 86.0 in stage 10.0 (TID 103) +26/04/01 08:38:19 INFO Executor: Finished task 78.0 in stage 10.0 (TID 95). 2251 bytes result sent to driver +26/04/01 08:38:19 INFO TaskSetManager: Starting task 87.0 in stage 10.0 (TID 104) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:19 INFO TaskSetManager: Finished task 78.0 in stage 10.0 (TID 95) in 1177 ms on 10.0.0.133 (executor driver) (80/208) +26/04/01 08:38:19 INFO Executor: Running task 87.0 in stage 10.0 (TID 104) +26/04/01 08:38:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:19 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:19 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:20 INFO Executor: Finished task 80.0 in stage 10.0 (TID 97). 2251 bytes result sent to driver +26/04/01 08:38:20 INFO TaskSetManager: Starting task 88.0 in stage 10.0 (TID 105) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:20 INFO TaskSetManager: Finished task 80.0 in stage 10.0 (TID 97) in 1066 ms on 10.0.0.133 (executor driver) (81/208) +26/04/01 08:38:20 INFO Executor: Running task 88.0 in stage 10.0 (TID 105) +26/04/01 08:38:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:20 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:20 INFO Executor: Finished task 81.0 in stage 10.0 (TID 98). 2251 bytes result sent to driver +26/04/01 08:38:20 INFO TaskSetManager: Starting task 89.0 in stage 10.0 (TID 106) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:20 INFO Executor: Running task 89.0 in stage 10.0 (TID 106) +26/04/01 08:38:20 INFO TaskSetManager: Finished task 81.0 in stage 10.0 (TID 98) in 1075 ms on 10.0.0.133 (executor driver) (82/208) +26/04/01 08:38:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:20 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:20 INFO Executor: Finished task 82.0 in stage 10.0 (TID 99). 2251 bytes result sent to driver +26/04/01 08:38:20 INFO TaskSetManager: Starting task 90.0 in stage 10.0 (TID 107) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:20 INFO TaskSetManager: Finished task 82.0 in stage 10.0 (TID 99) in 1093 ms on 10.0.0.133 (executor driver) (83/208) +26/04/01 08:38:20 INFO Executor: Running task 90.0 in stage 10.0 (TID 107) +26/04/01 08:38:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:20 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:20 INFO Executor: Finished task 83.0 in stage 10.0 (TID 100). 2251 bytes result sent to driver +26/04/01 08:38:20 INFO TaskSetManager: Starting task 91.0 in stage 10.0 (TID 108) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:20 INFO TaskSetManager: Finished task 83.0 in stage 10.0 (TID 100) in 1079 ms on 10.0.0.133 (executor driver) (84/208) +26/04/01 08:38:20 INFO Executor: Running task 91.0 in stage 10.0 (TID 108) +26/04/01 08:38:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:20 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:20 INFO Executor: Finished task 84.0 in stage 10.0 (TID 101). 2294 bytes result sent to driver +26/04/01 08:38:20 INFO TaskSetManager: Starting task 92.0 in stage 10.0 (TID 109) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:20 INFO TaskSetManager: Finished task 84.0 in stage 10.0 (TID 101) in 1087 ms on 10.0.0.133 (executor driver) (85/208) +26/04/01 08:38:20 INFO Executor: Running task 92.0 in stage 10.0 (TID 109) +26/04/01 08:38:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:20 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:21 INFO Executor: Finished task 85.0 in stage 10.0 (TID 102). 2251 bytes result sent to driver +26/04/01 08:38:21 INFO TaskSetManager: Starting task 93.0 in stage 10.0 (TID 110) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:21 INFO Executor: Running task 93.0 in stage 10.0 (TID 110) +26/04/01 08:38:21 INFO TaskSetManager: Finished task 85.0 in stage 10.0 (TID 102) in 1154 ms on 10.0.0.133 (executor driver) (86/208) +26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:21 INFO Executor: Finished task 87.0 in stage 10.0 (TID 104). 2251 bytes result sent to driver +26/04/01 08:38:21 INFO TaskSetManager: Starting task 94.0 in stage 10.0 (TID 111) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:21 INFO Executor: Running task 94.0 in stage 10.0 (TID 111) +26/04/01 08:38:21 INFO TaskSetManager: Finished task 87.0 in stage 10.0 (TID 104) in 1137 ms on 10.0.0.133 (executor driver) (87/208) +26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:21 INFO Executor: Finished task 86.0 in stage 10.0 (TID 103). 2251 bytes result sent to driver +26/04/01 08:38:21 INFO TaskSetManager: Starting task 95.0 in stage 10.0 (TID 112) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:21 INFO TaskSetManager: Finished task 86.0 in stage 10.0 (TID 103) in 1145 ms on 10.0.0.133 (executor driver) (88/208) +26/04/01 08:38:21 INFO Executor: Running task 95.0 in stage 10.0 (TID 112) +26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:21 INFO Executor: Finished task 88.0 in stage 10.0 (TID 105). 2251 bytes result sent to driver +26/04/01 08:38:21 INFO TaskSetManager: Starting task 96.0 in stage 10.0 (TID 113) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:21 INFO Executor: Running task 96.0 in stage 10.0 (TID 113) +26/04/01 08:38:21 INFO TaskSetManager: Finished task 88.0 in stage 10.0 (TID 105) in 1094 ms on 10.0.0.133 (executor driver) (89/208) +26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:21 INFO Executor: Finished task 89.0 in stage 10.0 (TID 106). 2251 bytes result sent to driver +26/04/01 08:38:21 INFO TaskSetManager: Starting task 97.0 in stage 10.0 (TID 114) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:21 INFO TaskSetManager: Finished task 89.0 in stage 10.0 (TID 106) in 1042 ms on 10.0.0.133 (executor driver) (90/208) +26/04/01 08:38:21 INFO Executor: Running task 97.0 in stage 10.0 (TID 114) +26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:21 INFO Executor: Finished task 90.0 in stage 10.0 (TID 107). 2251 bytes result sent to driver +26/04/01 08:38:21 INFO TaskSetManager: Starting task 98.0 in stage 10.0 (TID 115) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:21 INFO TaskSetManager: Finished task 90.0 in stage 10.0 (TID 107) in 1142 ms on 10.0.0.133 (executor driver) (91/208) +26/04/01 08:38:21 INFO Executor: Running task 98.0 in stage 10.0 (TID 115) +26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:21 INFO Executor: Finished task 91.0 in stage 10.0 (TID 108). 2251 bytes result sent to driver +26/04/01 08:38:21 INFO TaskSetManager: Starting task 99.0 in stage 10.0 (TID 116) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:21 INFO TaskSetManager: Finished task 91.0 in stage 10.0 (TID 108) in 1131 ms on 10.0.0.133 (executor driver) (92/208) +26/04/01 08:38:21 INFO Executor: Running task 99.0 in stage 10.0 (TID 116) +26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:21 INFO Executor: Finished task 92.0 in stage 10.0 (TID 109). 2251 bytes result sent to driver +26/04/01 08:38:21 INFO TaskSetManager: Starting task 100.0 in stage 10.0 (TID 117) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:21 INFO TaskSetManager: Finished task 92.0 in stage 10.0 (TID 109) in 1102 ms on 10.0.0.133 (executor driver) (93/208) +26/04/01 08:38:21 INFO Executor: Running task 100.0 in stage 10.0 (TID 117) +26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:22 INFO Executor: Finished task 93.0 in stage 10.0 (TID 110). 2251 bytes result sent to driver +26/04/01 08:38:22 INFO TaskSetManager: Starting task 101.0 in stage 10.0 (TID 118) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:22 INFO TaskSetManager: Finished task 93.0 in stage 10.0 (TID 110) in 1158 ms on 10.0.0.133 (executor driver) (94/208) +26/04/01 08:38:22 INFO Executor: Running task 101.0 in stage 10.0 (TID 118) +26/04/01 08:38:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:22 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:22 INFO Executor: Finished task 95.0 in stage 10.0 (TID 112). 2251 bytes result sent to driver +26/04/01 08:38:22 INFO TaskSetManager: Starting task 102.0 in stage 10.0 (TID 119) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:22 INFO TaskSetManager: Finished task 95.0 in stage 10.0 (TID 112) in 1180 ms on 10.0.0.133 (executor driver) (95/208) +26/04/01 08:38:22 INFO Executor: Running task 102.0 in stage 10.0 (TID 119) +26/04/01 08:38:22 INFO Executor: Finished task 94.0 in stage 10.0 (TID 111). 2251 bytes result sent to driver +26/04/01 08:38:22 INFO TaskSetManager: Starting task 103.0 in stage 10.0 (TID 120) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:22 INFO TaskSetManager: Finished task 94.0 in stage 10.0 (TID 111) in 1188 ms on 10.0.0.133 (executor driver) (96/208) +26/04/01 08:38:22 INFO Executor: Running task 103.0 in stage 10.0 (TID 120) +26/04/01 08:38:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:22 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:22 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:22 INFO Executor: Finished task 96.0 in stage 10.0 (TID 113). 2251 bytes result sent to driver +26/04/01 08:38:22 INFO TaskSetManager: Starting task 104.0 in stage 10.0 (TID 121) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:22 INFO TaskSetManager: Finished task 96.0 in stage 10.0 (TID 113) in 1138 ms on 10.0.0.133 (executor driver) (97/208) +26/04/01 08:38:22 INFO Executor: Running task 104.0 in stage 10.0 (TID 121) +26/04/01 08:38:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:22 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:22 INFO Executor: Finished task 97.0 in stage 10.0 (TID 114). 2251 bytes result sent to driver +26/04/01 08:38:22 INFO TaskSetManager: Starting task 105.0 in stage 10.0 (TID 122) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:22 INFO TaskSetManager: Finished task 97.0 in stage 10.0 (TID 114) in 1156 ms on 10.0.0.133 (executor driver) (98/208) +26/04/01 08:38:22 INFO Executor: Running task 105.0 in stage 10.0 (TID 122) +26/04/01 08:38:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:22 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:22 INFO Executor: Finished task 98.0 in stage 10.0 (TID 115). 2251 bytes result sent to driver +26/04/01 08:38:22 INFO TaskSetManager: Starting task 106.0 in stage 10.0 (TID 123) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:22 INFO TaskSetManager: Finished task 98.0 in stage 10.0 (TID 115) in 1103 ms on 10.0.0.133 (executor driver) (99/208) +26/04/01 08:38:22 INFO Executor: Running task 106.0 in stage 10.0 (TID 123) +26/04/01 08:38:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:22 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:22 INFO Executor: Finished task 99.0 in stage 10.0 (TID 116). 2251 bytes result sent to driver +26/04/01 08:38:22 INFO TaskSetManager: Starting task 107.0 in stage 10.0 (TID 124) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:22 INFO TaskSetManager: Finished task 99.0 in stage 10.0 (TID 116) in 1147 ms on 10.0.0.133 (executor driver) (100/208) +26/04/01 08:38:22 INFO Executor: Running task 107.0 in stage 10.0 (TID 124) +26/04/01 08:38:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:22 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:23 INFO Executor: Finished task 100.0 in stage 10.0 (TID 117). 2251 bytes result sent to driver +26/04/01 08:38:23 INFO TaskSetManager: Starting task 108.0 in stage 10.0 (TID 125) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:23 INFO Executor: Running task 108.0 in stage 10.0 (TID 125) +26/04/01 08:38:23 INFO TaskSetManager: Finished task 100.0 in stage 10.0 (TID 117) in 1158 ms on 10.0.0.133 (executor driver) (101/208) +26/04/01 08:38:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:23 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:23 INFO Executor: Finished task 101.0 in stage 10.0 (TID 118). 2251 bytes result sent to driver +26/04/01 08:38:23 INFO TaskSetManager: Starting task 109.0 in stage 10.0 (TID 126) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:23 INFO TaskSetManager: Finished task 101.0 in stage 10.0 (TID 118) in 1065 ms on 10.0.0.133 (executor driver) (102/208) +26/04/01 08:38:23 INFO Executor: Running task 109.0 in stage 10.0 (TID 126) +26/04/01 08:38:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:23 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:23 INFO Executor: Finished task 103.0 in stage 10.0 (TID 120). 2251 bytes result sent to driver +26/04/01 08:38:23 INFO TaskSetManager: Starting task 110.0 in stage 10.0 (TID 127) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:23 INFO TaskSetManager: Finished task 103.0 in stage 10.0 (TID 120) in 1298 ms on 10.0.0.133 (executor driver) (103/208) +26/04/01 08:38:23 INFO Executor: Running task 110.0 in stage 10.0 (TID 127) +26/04/01 08:38:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:23 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:23 INFO Executor: Finished task 102.0 in stage 10.0 (TID 119). 2251 bytes result sent to driver +26/04/01 08:38:23 INFO TaskSetManager: Starting task 111.0 in stage 10.0 (TID 128) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:23 INFO TaskSetManager: Finished task 102.0 in stage 10.0 (TID 119) in 1378 ms on 10.0.0.133 (executor driver) (104/208) +26/04/01 08:38:23 INFO Executor: Running task 111.0 in stage 10.0 (TID 128) +26/04/01 08:38:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:23 INFO Executor: Finished task 105.0 in stage 10.0 (TID 122). 2251 bytes result sent to driver +26/04/01 08:38:23 INFO TaskSetManager: Starting task 112.0 in stage 10.0 (TID 129) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:23 INFO TaskSetManager: Finished task 105.0 in stage 10.0 (TID 122) in 1266 ms on 10.0.0.133 (executor driver) (105/208) +26/04/01 08:38:23 INFO Executor: Running task 112.0 in stage 10.0 (TID 129) +26/04/01 08:38:23 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:23 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:23 INFO Executor: Finished task 106.0 in stage 10.0 (TID 123). 2251 bytes result sent to driver +26/04/01 08:38:23 INFO TaskSetManager: Starting task 113.0 in stage 10.0 (TID 130) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:23 INFO TaskSetManager: Finished task 106.0 in stage 10.0 (TID 123) in 1239 ms on 10.0.0.133 (executor driver) (106/208) +26/04/01 08:38:23 INFO Executor: Running task 113.0 in stage 10.0 (TID 130) +26/04/01 08:38:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:23 INFO Executor: Finished task 104.0 in stage 10.0 (TID 121). 2251 bytes result sent to driver +26/04/01 08:38:23 INFO TaskSetManager: Starting task 114.0 in stage 10.0 (TID 131) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:23 INFO Executor: Running task 114.0 in stage 10.0 (TID 131) +26/04/01 08:38:23 INFO TaskSetManager: Finished task 104.0 in stage 10.0 (TID 121) in 1489 ms on 10.0.0.133 (executor driver) (107/208) +26/04/01 08:38:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:23 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:23 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:24 INFO Executor: Finished task 107.0 in stage 10.0 (TID 124). 2251 bytes result sent to driver +26/04/01 08:38:24 INFO TaskSetManager: Starting task 115.0 in stage 10.0 (TID 132) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:24 INFO TaskSetManager: Finished task 107.0 in stage 10.0 (TID 124) in 1204 ms on 10.0.0.133 (executor driver) (108/208) +26/04/01 08:38:24 INFO Executor: Running task 115.0 in stage 10.0 (TID 132) +26/04/01 08:38:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:24 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:24 INFO Executor: Finished task 108.0 in stage 10.0 (TID 125). 2251 bytes result sent to driver +26/04/01 08:38:24 INFO TaskSetManager: Starting task 116.0 in stage 10.0 (TID 133) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:24 INFO TaskSetManager: Finished task 108.0 in stage 10.0 (TID 125) in 1168 ms on 10.0.0.133 (executor driver) (109/208) +26/04/01 08:38:24 INFO Executor: Running task 116.0 in stage 10.0 (TID 133) +26/04/01 08:38:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:24 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:24 INFO Executor: Finished task 109.0 in stage 10.0 (TID 126). 2251 bytes result sent to driver +26/04/01 08:38:24 INFO TaskSetManager: Starting task 117.0 in stage 10.0 (TID 134) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:24 INFO Executor: Running task 117.0 in stage 10.0 (TID 134) +26/04/01 08:38:24 INFO TaskSetManager: Finished task 109.0 in stage 10.0 (TID 126) in 1150 ms on 10.0.0.133 (executor driver) (110/208) +26/04/01 08:38:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:24 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:24 INFO Executor: Finished task 110.0 in stage 10.0 (TID 127). 2251 bytes result sent to driver +26/04/01 08:38:24 INFO TaskSetManager: Starting task 118.0 in stage 10.0 (TID 135) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:24 INFO Executor: Running task 118.0 in stage 10.0 (TID 135) +26/04/01 08:38:24 INFO TaskSetManager: Finished task 110.0 in stage 10.0 (TID 127) in 1097 ms on 10.0.0.133 (executor driver) (111/208) +26/04/01 08:38:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:24 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:25 INFO Executor: Finished task 112.0 in stage 10.0 (TID 129). 2251 bytes result sent to driver +26/04/01 08:38:25 INFO TaskSetManager: Starting task 119.0 in stage 10.0 (TID 136) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:25 INFO Executor: Finished task 111.0 in stage 10.0 (TID 128). 2251 bytes result sent to driver +26/04/01 08:38:25 INFO Executor: Running task 119.0 in stage 10.0 (TID 136) +26/04/01 08:38:25 INFO TaskSetManager: Starting task 120.0 in stage 10.0 (TID 137) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:25 INFO Executor: Running task 120.0 in stage 10.0 (TID 137) +26/04/01 08:38:25 INFO TaskSetManager: Finished task 112.0 in stage 10.0 (TID 129) in 1607 ms on 10.0.0.133 (executor driver) (112/208) +26/04/01 08:38:25 INFO TaskSetManager: Finished task 111.0 in stage 10.0 (TID 128) in 1613 ms on 10.0.0.133 (executor driver) (113/208) +26/04/01 08:38:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:25 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:25 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:25 INFO Executor: Finished task 114.0 in stage 10.0 (TID 131). 2251 bytes result sent to driver +26/04/01 08:38:25 INFO TaskSetManager: Starting task 121.0 in stage 10.0 (TID 138) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:25 INFO TaskSetManager: Finished task 114.0 in stage 10.0 (TID 131) in 1511 ms on 10.0.0.133 (executor driver) (114/208) +26/04/01 08:38:25 INFO Executor: Running task 121.0 in stage 10.0 (TID 138) +26/04/01 08:38:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:25 INFO Executor: Finished task 113.0 in stage 10.0 (TID 130). 2251 bytes result sent to driver +26/04/01 08:38:25 INFO TaskSetManager: Starting task 122.0 in stage 10.0 (TID 139) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:25 INFO TaskSetManager: Finished task 113.0 in stage 10.0 (TID 130) in 1519 ms on 10.0.0.133 (executor driver) (115/208) +26/04/01 08:38:25 INFO Executor: Running task 122.0 in stage 10.0 (TID 139) +26/04/01 08:38:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:25 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:25 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:25 INFO Executor: Finished task 115.0 in stage 10.0 (TID 132). 2251 bytes result sent to driver +26/04/01 08:38:25 INFO TaskSetManager: Starting task 123.0 in stage 10.0 (TID 140) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:25 INFO TaskSetManager: Finished task 115.0 in stage 10.0 (TID 132) in 1454 ms on 10.0.0.133 (executor driver) (116/208) +26/04/01 08:38:25 INFO Executor: Running task 123.0 in stage 10.0 (TID 140) +26/04/01 08:38:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:25 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:25 INFO BlockManagerInfo: Removed broadcast_10_piece0 on 10.0.0.133:59288 in memory (size: 8.1 KiB, free: 8.6 GiB) +26/04/01 08:38:25 INFO BlockManagerInfo: Removed broadcast_11_piece0 on 10.0.0.133:59288 in memory (size: 6.5 KiB, free: 8.6 GiB) +26/04/01 08:38:25 INFO Executor: Finished task 116.0 in stage 10.0 (TID 133). 2251 bytes result sent to driver +26/04/01 08:38:25 INFO TaskSetManager: Starting task 124.0 in stage 10.0 (TID 141) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:25 INFO TaskSetManager: Finished task 116.0 in stage 10.0 (TID 133) in 1471 ms on 10.0.0.133 (executor driver) (117/208) +26/04/01 08:38:25 INFO Executor: Running task 124.0 in stage 10.0 (TID 141) +26/04/01 08:38:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:25 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:25 INFO Executor: Finished task 117.0 in stage 10.0 (TID 134). 2251 bytes result sent to driver +26/04/01 08:38:25 INFO TaskSetManager: Starting task 125.0 in stage 10.0 (TID 142) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:25 INFO TaskSetManager: Finished task 117.0 in stage 10.0 (TID 134) in 1453 ms on 10.0.0.133 (executor driver) (118/208) +26/04/01 08:38:25 INFO Executor: Running task 125.0 in stage 10.0 (TID 142) +26/04/01 08:38:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:25 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:26 INFO Executor: Finished task 118.0 in stage 10.0 (TID 135). 2251 bytes result sent to driver +26/04/01 08:38:26 INFO TaskSetManager: Starting task 126.0 in stage 10.0 (TID 143) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:26 INFO TaskSetManager: Finished task 118.0 in stage 10.0 (TID 135) in 1446 ms on 10.0.0.133 (executor driver) (119/208) +26/04/01 08:38:26 INFO Executor: Running task 126.0 in stage 10.0 (TID 143) +26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:26 INFO Executor: Finished task 119.0 in stage 10.0 (TID 136). 2251 bytes result sent to driver +26/04/01 08:38:26 INFO TaskSetManager: Starting task 127.0 in stage 10.0 (TID 144) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:26 INFO TaskSetManager: Finished task 119.0 in stage 10.0 (TID 136) in 1063 ms on 10.0.0.133 (executor driver) (120/208) +26/04/01 08:38:26 INFO Executor: Running task 127.0 in stage 10.0 (TID 144) +26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:26 INFO Executor: Finished task 120.0 in stage 10.0 (TID 137). 2251 bytes result sent to driver +26/04/01 08:38:26 INFO TaskSetManager: Starting task 128.0 in stage 10.0 (TID 145) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:26 INFO TaskSetManager: Finished task 120.0 in stage 10.0 (TID 137) in 1098 ms on 10.0.0.133 (executor driver) (121/208) +26/04/01 08:38:26 INFO Executor: Running task 128.0 in stage 10.0 (TID 145) +26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:26 INFO Executor: Finished task 122.0 in stage 10.0 (TID 139). 2251 bytes result sent to driver +26/04/01 08:38:26 INFO TaskSetManager: Starting task 129.0 in stage 10.0 (TID 146) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:26 INFO Executor: Running task 129.0 in stage 10.0 (TID 146) +26/04/01 08:38:26 INFO TaskSetManager: Finished task 122.0 in stage 10.0 (TID 139) in 1110 ms on 10.0.0.133 (executor driver) (122/208) +26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:26 INFO Executor: Finished task 121.0 in stage 10.0 (TID 138). 2251 bytes result sent to driver +26/04/01 08:38:26 INFO TaskSetManager: Starting task 130.0 in stage 10.0 (TID 147) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:26 INFO TaskSetManager: Finished task 121.0 in stage 10.0 (TID 138) in 1114 ms on 10.0.0.133 (executor driver) (123/208) +26/04/01 08:38:26 INFO Executor: Running task 130.0 in stage 10.0 (TID 147) +26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:26 INFO Executor: Finished task 123.0 in stage 10.0 (TID 140). 2251 bytes result sent to driver +26/04/01 08:38:26 INFO TaskSetManager: Starting task 131.0 in stage 10.0 (TID 148) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:26 INFO TaskSetManager: Finished task 123.0 in stage 10.0 (TID 140) in 1075 ms on 10.0.0.133 (executor driver) (124/208) +26/04/01 08:38:26 INFO Executor: Running task 131.0 in stage 10.0 (TID 148) +26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:26 INFO Executor: Finished task 124.0 in stage 10.0 (TID 141). 2251 bytes result sent to driver +26/04/01 08:38:26 INFO TaskSetManager: Starting task 132.0 in stage 10.0 (TID 149) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:26 INFO TaskSetManager: Finished task 124.0 in stage 10.0 (TID 141) in 1028 ms on 10.0.0.133 (executor driver) (125/208) +26/04/01 08:38:26 INFO Executor: Running task 132.0 in stage 10.0 (TID 149) +26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:26 INFO Executor: Finished task 125.0 in stage 10.0 (TID 142). 2294 bytes result sent to driver +26/04/01 08:38:26 INFO TaskSetManager: Starting task 133.0 in stage 10.0 (TID 150) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:26 INFO TaskSetManager: Finished task 125.0 in stage 10.0 (TID 142) in 1036 ms on 10.0.0.133 (executor driver) (126/208) +26/04/01 08:38:26 INFO Executor: Running task 133.0 in stage 10.0 (TID 150) +26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:27 INFO Executor: Finished task 126.0 in stage 10.0 (TID 143). 2251 bytes result sent to driver +26/04/01 08:38:27 INFO TaskSetManager: Starting task 134.0 in stage 10.0 (TID 151) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:27 INFO TaskSetManager: Finished task 126.0 in stage 10.0 (TID 143) in 1035 ms on 10.0.0.133 (executor driver) (127/208) +26/04/01 08:38:27 INFO Executor: Running task 134.0 in stage 10.0 (TID 151) +26/04/01 08:38:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:27 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:27 INFO Executor: Finished task 127.0 in stage 10.0 (TID 144). 2251 bytes result sent to driver +26/04/01 08:38:27 INFO TaskSetManager: Starting task 135.0 in stage 10.0 (TID 152) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:27 INFO TaskSetManager: Finished task 127.0 in stage 10.0 (TID 144) in 1010 ms on 10.0.0.133 (executor driver) (128/208) +26/04/01 08:38:27 INFO Executor: Running task 135.0 in stage 10.0 (TID 152) +26/04/01 08:38:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:27 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:27 INFO Executor: Finished task 128.0 in stage 10.0 (TID 145). 2251 bytes result sent to driver +26/04/01 08:38:27 INFO TaskSetManager: Starting task 136.0 in stage 10.0 (TID 153) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:27 INFO TaskSetManager: Finished task 128.0 in stage 10.0 (TID 145) in 1130 ms on 10.0.0.133 (executor driver) (129/208) +26/04/01 08:38:27 INFO Executor: Running task 136.0 in stage 10.0 (TID 153) +26/04/01 08:38:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:27 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:27 INFO Executor: Finished task 129.0 in stage 10.0 (TID 146). 2251 bytes result sent to driver +26/04/01 08:38:27 INFO TaskSetManager: Starting task 137.0 in stage 10.0 (TID 154) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:27 INFO TaskSetManager: Finished task 129.0 in stage 10.0 (TID 146) in 1110 ms on 10.0.0.133 (executor driver) (130/208) +26/04/01 08:38:27 INFO Executor: Running task 137.0 in stage 10.0 (TID 154) +26/04/01 08:38:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:27 INFO Executor: Finished task 130.0 in stage 10.0 (TID 147). 2251 bytes result sent to driver +26/04/01 08:38:27 INFO TaskSetManager: Starting task 138.0 in stage 10.0 (TID 155) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:27 INFO TaskSetManager: Finished task 130.0 in stage 10.0 (TID 147) in 1110 ms on 10.0.0.133 (executor driver) (131/208) +26/04/01 08:38:27 INFO Executor: Running task 138.0 in stage 10.0 (TID 155) +26/04/01 08:38:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:27 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:27 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:27 INFO Executor: Finished task 131.0 in stage 10.0 (TID 148). 2251 bytes result sent to driver +26/04/01 08:38:27 INFO TaskSetManager: Starting task 139.0 in stage 10.0 (TID 156) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:27 INFO TaskSetManager: Finished task 131.0 in stage 10.0 (TID 148) in 1096 ms on 10.0.0.133 (executor driver) (132/208) +26/04/01 08:38:27 INFO Executor: Running task 139.0 in stage 10.0 (TID 156) +26/04/01 08:38:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:27 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:27 INFO Executor: Finished task 132.0 in stage 10.0 (TID 149). 2251 bytes result sent to driver +26/04/01 08:38:27 INFO TaskSetManager: Starting task 140.0 in stage 10.0 (TID 157) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:27 INFO Executor: Running task 140.0 in stage 10.0 (TID 157) +26/04/01 08:38:27 INFO TaskSetManager: Finished task 132.0 in stage 10.0 (TID 149) in 1123 ms on 10.0.0.133 (executor driver) (133/208) +26/04/01 08:38:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:27 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:28 INFO Executor: Finished task 133.0 in stage 10.0 (TID 150). 2251 bytes result sent to driver +26/04/01 08:38:28 INFO TaskSetManager: Starting task 141.0 in stage 10.0 (TID 158) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:28 INFO TaskSetManager: Finished task 133.0 in stage 10.0 (TID 150) in 1140 ms on 10.0.0.133 (executor driver) (134/208) +26/04/01 08:38:28 INFO Executor: Running task 141.0 in stage 10.0 (TID 158) +26/04/01 08:38:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:28 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:28 INFO Executor: Finished task 134.0 in stage 10.0 (TID 151). 2251 bytes result sent to driver +26/04/01 08:38:28 INFO TaskSetManager: Starting task 142.0 in stage 10.0 (TID 159) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:28 INFO Executor: Running task 142.0 in stage 10.0 (TID 159) +26/04/01 08:38:28 INFO TaskSetManager: Finished task 134.0 in stage 10.0 (TID 151) in 1139 ms on 10.0.0.133 (executor driver) (135/208) +26/04/01 08:38:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:28 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:28 INFO Executor: Finished task 135.0 in stage 10.0 (TID 152). 2251 bytes result sent to driver +26/04/01 08:38:28 INFO TaskSetManager: Starting task 143.0 in stage 10.0 (TID 160) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:28 INFO TaskSetManager: Finished task 135.0 in stage 10.0 (TID 152) in 1120 ms on 10.0.0.133 (executor driver) (136/208) +26/04/01 08:38:28 INFO Executor: Running task 143.0 in stage 10.0 (TID 160) +26/04/01 08:38:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:28 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:28 INFO Executor: Finished task 138.0 in stage 10.0 (TID 155). 2251 bytes result sent to driver +26/04/01 08:38:28 INFO TaskSetManager: Starting task 144.0 in stage 10.0 (TID 161) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:28 INFO TaskSetManager: Finished task 138.0 in stage 10.0 (TID 155) in 1165 ms on 10.0.0.133 (executor driver) (137/208) +26/04/01 08:38:28 INFO Executor: Running task 144.0 in stage 10.0 (TID 161) +26/04/01 08:38:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:28 INFO Executor: Finished task 137.0 in stage 10.0 (TID 154). 2251 bytes result sent to driver +26/04/01 08:38:28 INFO TaskSetManager: Starting task 145.0 in stage 10.0 (TID 162) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:28 INFO TaskSetManager: Finished task 137.0 in stage 10.0 (TID 154) in 1169 ms on 10.0.0.133 (executor driver) (138/208) +26/04/01 08:38:28 INFO Executor: Running task 145.0 in stage 10.0 (TID 162) +26/04/01 08:38:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:28 INFO Executor: Finished task 136.0 in stage 10.0 (TID 153). 2251 bytes result sent to driver +26/04/01 08:38:28 INFO TaskSetManager: Starting task 146.0 in stage 10.0 (TID 163) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:28 INFO TaskSetManager: Finished task 136.0 in stage 10.0 (TID 153) in 1219 ms on 10.0.0.133 (executor driver) (139/208) +26/04/01 08:38:28 INFO Executor: Running task 146.0 in stage 10.0 (TID 163) +26/04/01 08:38:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:28 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:28 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:28 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:28 INFO Executor: Finished task 139.0 in stage 10.0 (TID 156). 2251 bytes result sent to driver +26/04/01 08:38:28 INFO TaskSetManager: Starting task 147.0 in stage 10.0 (TID 164) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:28 INFO TaskSetManager: Finished task 139.0 in stage 10.0 (TID 156) in 1107 ms on 10.0.0.133 (executor driver) (140/208) +26/04/01 08:38:28 INFO Executor: Running task 147.0 in stage 10.0 (TID 164) +26/04/01 08:38:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:28 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:29 INFO Executor: Finished task 140.0 in stage 10.0 (TID 157). 2251 bytes result sent to driver +26/04/01 08:38:29 INFO TaskSetManager: Starting task 148.0 in stage 10.0 (TID 165) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:29 INFO TaskSetManager: Finished task 140.0 in stage 10.0 (TID 157) in 1133 ms on 10.0.0.133 (executor driver) (141/208) +26/04/01 08:38:29 INFO Executor: Running task 148.0 in stage 10.0 (TID 165) +26/04/01 08:38:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:29 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:29 INFO Executor: Finished task 141.0 in stage 10.0 (TID 158). 2251 bytes result sent to driver +26/04/01 08:38:29 INFO TaskSetManager: Starting task 149.0 in stage 10.0 (TID 166) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:29 INFO Executor: Running task 149.0 in stage 10.0 (TID 166) +26/04/01 08:38:29 INFO TaskSetManager: Finished task 141.0 in stage 10.0 (TID 158) in 1123 ms on 10.0.0.133 (executor driver) (142/208) +26/04/01 08:38:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:29 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:29 INFO Executor: Finished task 142.0 in stage 10.0 (TID 159). 2251 bytes result sent to driver +26/04/01 08:38:29 INFO TaskSetManager: Starting task 150.0 in stage 10.0 (TID 167) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:29 INFO Executor: Running task 150.0 in stage 10.0 (TID 167) +26/04/01 08:38:29 INFO TaskSetManager: Finished task 142.0 in stage 10.0 (TID 159) in 1109 ms on 10.0.0.133 (executor driver) (143/208) +26/04/01 08:38:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:29 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:29 INFO Executor: Finished task 143.0 in stage 10.0 (TID 160). 2251 bytes result sent to driver +26/04/01 08:38:29 INFO TaskSetManager: Starting task 151.0 in stage 10.0 (TID 168) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:29 INFO TaskSetManager: Finished task 143.0 in stage 10.0 (TID 160) in 1099 ms on 10.0.0.133 (executor driver) (144/208) +26/04/01 08:38:29 INFO Executor: Running task 151.0 in stage 10.0 (TID 168) +26/04/01 08:38:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:29 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:29 INFO Executor: Finished task 146.0 in stage 10.0 (TID 163). 2251 bytes result sent to driver +26/04/01 08:38:29 INFO TaskSetManager: Starting task 152.0 in stage 10.0 (TID 169) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:29 INFO TaskSetManager: Finished task 146.0 in stage 10.0 (TID 163) in 1159 ms on 10.0.0.133 (executor driver) (145/208) +26/04/01 08:38:29 INFO Executor: Running task 152.0 in stage 10.0 (TID 169) +26/04/01 08:38:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:29 INFO Executor: Finished task 145.0 in stage 10.0 (TID 162). 2251 bytes result sent to driver +26/04/01 08:38:29 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:29 INFO TaskSetManager: Starting task 153.0 in stage 10.0 (TID 170) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:29 INFO TaskSetManager: Finished task 145.0 in stage 10.0 (TID 162) in 1179 ms on 10.0.0.133 (executor driver) (146/208) +26/04/01 08:38:29 INFO Executor: Running task 153.0 in stage 10.0 (TID 170) +26/04/01 08:38:29 INFO Executor: Finished task 144.0 in stage 10.0 (TID 161). 2251 bytes result sent to driver +26/04/01 08:38:29 INFO TaskSetManager: Starting task 154.0 in stage 10.0 (TID 171) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:29 INFO TaskSetManager: Finished task 144.0 in stage 10.0 (TID 161) in 1190 ms on 10.0.0.133 (executor driver) (147/208) +26/04/01 08:38:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:29 INFO Executor: Running task 154.0 in stage 10.0 (TID 171) +26/04/01 08:38:29 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:29 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:30 INFO Executor: Finished task 147.0 in stage 10.0 (TID 164). 2251 bytes result sent to driver +26/04/01 08:38:30 INFO TaskSetManager: Starting task 155.0 in stage 10.0 (TID 172) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:30 INFO TaskSetManager: Finished task 147.0 in stage 10.0 (TID 164) in 1297 ms on 10.0.0.133 (executor driver) (148/208) +26/04/01 08:38:30 INFO Executor: Running task 155.0 in stage 10.0 (TID 172) +26/04/01 08:38:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:30 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:30 INFO Executor: Finished task 148.0 in stage 10.0 (TID 165). 2251 bytes result sent to driver +26/04/01 08:38:30 INFO TaskSetManager: Starting task 156.0 in stage 10.0 (TID 173) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:30 INFO TaskSetManager: Finished task 148.0 in stage 10.0 (TID 165) in 1225 ms on 10.0.0.133 (executor driver) (149/208) +26/04/01 08:38:30 INFO Executor: Running task 156.0 in stage 10.0 (TID 173) +26/04/01 08:38:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:30 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:30 INFO Executor: Finished task 149.0 in stage 10.0 (TID 166). 2251 bytes result sent to driver +26/04/01 08:38:30 INFO TaskSetManager: Starting task 157.0 in stage 10.0 (TID 174) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:30 INFO TaskSetManager: Finished task 149.0 in stage 10.0 (TID 166) in 1176 ms on 10.0.0.133 (executor driver) (150/208) +26/04/01 08:38:30 INFO Executor: Running task 157.0 in stage 10.0 (TID 174) +26/04/01 08:38:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:30 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:30 INFO Executor: Finished task 150.0 in stage 10.0 (TID 167). 2251 bytes result sent to driver +26/04/01 08:38:30 INFO TaskSetManager: Starting task 158.0 in stage 10.0 (TID 175) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:30 INFO Executor: Running task 158.0 in stage 10.0 (TID 175) +26/04/01 08:38:30 INFO TaskSetManager: Finished task 150.0 in stage 10.0 (TID 167) in 1184 ms on 10.0.0.133 (executor driver) (151/208) +26/04/01 08:38:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:30 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:30 INFO Executor: Finished task 151.0 in stage 10.0 (TID 168). 2251 bytes result sent to driver +26/04/01 08:38:30 INFO TaskSetManager: Starting task 159.0 in stage 10.0 (TID 176) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:30 INFO TaskSetManager: Finished task 151.0 in stage 10.0 (TID 168) in 1175 ms on 10.0.0.133 (executor driver) (152/208) +26/04/01 08:38:30 INFO Executor: Running task 159.0 in stage 10.0 (TID 176) +26/04/01 08:38:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:30 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:31 INFO Executor: Finished task 153.0 in stage 10.0 (TID 170). 2294 bytes result sent to driver +26/04/01 08:38:31 INFO TaskSetManager: Starting task 160.0 in stage 10.0 (TID 177) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:31 INFO TaskSetManager: Finished task 153.0 in stage 10.0 (TID 170) in 1364 ms on 10.0.0.133 (executor driver) (153/208) +26/04/01 08:38:31 INFO Executor: Running task 160.0 in stage 10.0 (TID 177) +26/04/01 08:38:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:31 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:31 INFO Executor: Finished task 154.0 in stage 10.0 (TID 171). 2294 bytes result sent to driver +26/04/01 08:38:31 INFO TaskSetManager: Starting task 161.0 in stage 10.0 (TID 178) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:31 INFO TaskSetManager: Finished task 154.0 in stage 10.0 (TID 171) in 1385 ms on 10.0.0.133 (executor driver) (154/208) +26/04/01 08:38:31 INFO Executor: Running task 161.0 in stage 10.0 (TID 178) +26/04/01 08:38:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:31 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:31 INFO Executor: Finished task 152.0 in stage 10.0 (TID 169). 2251 bytes result sent to driver +26/04/01 08:38:31 INFO TaskSetManager: Starting task 162.0 in stage 10.0 (TID 179) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:31 INFO Executor: Running task 162.0 in stage 10.0 (TID 179) +26/04/01 08:38:31 INFO TaskSetManager: Finished task 152.0 in stage 10.0 (TID 169) in 1429 ms on 10.0.0.133 (executor driver) (155/208) +26/04/01 08:38:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:31 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:31 INFO Executor: Finished task 155.0 in stage 10.0 (TID 172). 2251 bytes result sent to driver +26/04/01 08:38:31 INFO TaskSetManager: Starting task 163.0 in stage 10.0 (TID 180) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:31 INFO TaskSetManager: Finished task 155.0 in stage 10.0 (TID 172) in 1394 ms on 10.0.0.133 (executor driver) (156/208) +26/04/01 08:38:31 INFO Executor: Running task 163.0 in stage 10.0 (TID 180) +26/04/01 08:38:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:31 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:31 INFO Executor: Finished task 157.0 in stage 10.0 (TID 174). 2251 bytes result sent to driver +26/04/01 08:38:31 INFO TaskSetManager: Starting task 164.0 in stage 10.0 (TID 181) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:31 INFO Executor: Running task 164.0 in stage 10.0 (TID 181) +26/04/01 08:38:31 INFO TaskSetManager: Finished task 157.0 in stage 10.0 (TID 174) in 1456 ms on 10.0.0.133 (executor driver) (157/208) +26/04/01 08:38:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:31 INFO Executor: Finished task 156.0 in stage 10.0 (TID 173). 2251 bytes result sent to driver +26/04/01 08:38:31 INFO TaskSetManager: Starting task 165.0 in stage 10.0 (TID 182) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:31 INFO TaskSetManager: Finished task 156.0 in stage 10.0 (TID 173) in 1486 ms on 10.0.0.133 (executor driver) (158/208) +26/04/01 08:38:31 INFO Executor: Running task 165.0 in stage 10.0 (TID 182) +26/04/01 08:38:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:31 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:31 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:32 INFO Executor: Finished task 158.0 in stage 10.0 (TID 175). 2251 bytes result sent to driver +26/04/01 08:38:32 INFO TaskSetManager: Starting task 166.0 in stage 10.0 (TID 183) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:32 INFO TaskSetManager: Finished task 158.0 in stage 10.0 (TID 175) in 1505 ms on 10.0.0.133 (executor driver) (159/208) +26/04/01 08:38:32 INFO Executor: Running task 166.0 in stage 10.0 (TID 183) +26/04/01 08:38:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:32 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:32 INFO Executor: Finished task 159.0 in stage 10.0 (TID 176). 2251 bytes result sent to driver +26/04/01 08:38:32 INFO TaskSetManager: Starting task 167.0 in stage 10.0 (TID 184) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:32 INFO Executor: Running task 167.0 in stage 10.0 (TID 184) +26/04/01 08:38:32 INFO TaskSetManager: Finished task 159.0 in stage 10.0 (TID 176) in 1562 ms on 10.0.0.133 (executor driver) (160/208) +26/04/01 08:38:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:32 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:32 INFO Executor: Finished task 160.0 in stage 10.0 (TID 177). 2251 bytes result sent to driver +26/04/01 08:38:32 INFO TaskSetManager: Starting task 168.0 in stage 10.0 (TID 185) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:32 INFO TaskSetManager: Finished task 160.0 in stage 10.0 (TID 177) in 1662 ms on 10.0.0.133 (executor driver) (161/208) +26/04/01 08:38:32 INFO Executor: Running task 168.0 in stage 10.0 (TID 185) +26/04/01 08:38:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:32 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:32 INFO Executor: Finished task 161.0 in stage 10.0 (TID 178). 2251 bytes result sent to driver +26/04/01 08:38:32 INFO TaskSetManager: Starting task 169.0 in stage 10.0 (TID 186) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:32 INFO TaskSetManager: Finished task 161.0 in stage 10.0 (TID 178) in 1674 ms on 10.0.0.133 (executor driver) (162/208) +26/04/01 08:38:32 INFO Executor: Running task 169.0 in stage 10.0 (TID 186) +26/04/01 08:38:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:32 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:32 INFO Executor: Finished task 162.0 in stage 10.0 (TID 179). 2251 bytes result sent to driver +26/04/01 08:38:32 INFO TaskSetManager: Starting task 170.0 in stage 10.0 (TID 187) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:32 INFO TaskSetManager: Finished task 162.0 in stage 10.0 (TID 179) in 1668 ms on 10.0.0.133 (executor driver) (163/208) +26/04/01 08:38:32 INFO Executor: Running task 170.0 in stage 10.0 (TID 187) +26/04/01 08:38:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:32 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:33 INFO Executor: Finished task 163.0 in stage 10.0 (TID 180). 2251 bytes result sent to driver +26/04/01 08:38:33 INFO TaskSetManager: Starting task 171.0 in stage 10.0 (TID 188) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:33 INFO TaskSetManager: Finished task 163.0 in stage 10.0 (TID 180) in 1664 ms on 10.0.0.133 (executor driver) (164/208) +26/04/01 08:38:33 INFO Executor: Running task 171.0 in stage 10.0 (TID 188) +26/04/01 08:38:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:33 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:33 INFO Executor: Finished task 165.0 in stage 10.0 (TID 182). 2251 bytes result sent to driver +26/04/01 08:38:33 INFO TaskSetManager: Starting task 172.0 in stage 10.0 (TID 189) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:33 INFO TaskSetManager: Finished task 165.0 in stage 10.0 (TID 182) in 1737 ms on 10.0.0.133 (executor driver) (165/208) +26/04/01 08:38:33 INFO Executor: Running task 172.0 in stage 10.0 (TID 189) +26/04/01 08:38:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:33 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:33 INFO Executor: Finished task 164.0 in stage 10.0 (TID 181). 2251 bytes result sent to driver +26/04/01 08:38:33 INFO TaskSetManager: Starting task 173.0 in stage 10.0 (TID 190) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:33 INFO TaskSetManager: Finished task 164.0 in stage 10.0 (TID 181) in 1748 ms on 10.0.0.133 (executor driver) (166/208) +26/04/01 08:38:33 INFO Executor: Running task 173.0 in stage 10.0 (TID 190) +26/04/01 08:38:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:33 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:33 INFO Executor: Finished task 166.0 in stage 10.0 (TID 183). 2251 bytes result sent to driver +26/04/01 08:38:33 INFO TaskSetManager: Starting task 174.0 in stage 10.0 (TID 191) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:33 INFO TaskSetManager: Finished task 166.0 in stage 10.0 (TID 183) in 1673 ms on 10.0.0.133 (executor driver) (167/208) +26/04/01 08:38:33 INFO Executor: Running task 174.0 in stage 10.0 (TID 191) +26/04/01 08:38:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:33 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:33 INFO Executor: Finished task 167.0 in stage 10.0 (TID 184). 2251 bytes result sent to driver +26/04/01 08:38:33 INFO TaskSetManager: Starting task 175.0 in stage 10.0 (TID 192) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:33 INFO TaskSetManager: Finished task 167.0 in stage 10.0 (TID 184) in 1726 ms on 10.0.0.133 (executor driver) (168/208) +26/04/01 08:38:33 INFO Executor: Running task 175.0 in stage 10.0 (TID 192) +26/04/01 08:38:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:33 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:34 INFO Executor: Finished task 168.0 in stage 10.0 (TID 185). 2251 bytes result sent to driver +26/04/01 08:38:34 INFO TaskSetManager: Starting task 176.0 in stage 10.0 (TID 193) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:34 INFO TaskSetManager: Finished task 168.0 in stage 10.0 (TID 185) in 1775 ms on 10.0.0.133 (executor driver) (169/208) +26/04/01 08:38:34 INFO Executor: Running task 176.0 in stage 10.0 (TID 193) +26/04/01 08:38:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:34 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:34 INFO Executor: Finished task 170.0 in stage 10.0 (TID 187). 2251 bytes result sent to driver +26/04/01 08:38:34 INFO TaskSetManager: Starting task 177.0 in stage 10.0 (TID 194) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:34 INFO TaskSetManager: Finished task 170.0 in stage 10.0 (TID 187) in 1777 ms on 10.0.0.133 (executor driver) (170/208) +26/04/01 08:38:34 INFO Executor: Running task 177.0 in stage 10.0 (TID 194) +26/04/01 08:38:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:34 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:34 INFO Executor: Finished task 169.0 in stage 10.0 (TID 186). 2251 bytes result sent to driver +26/04/01 08:38:34 INFO TaskSetManager: Starting task 178.0 in stage 10.0 (TID 195) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:34 INFO Executor: Running task 178.0 in stage 10.0 (TID 195) +26/04/01 08:38:34 INFO TaskSetManager: Finished task 169.0 in stage 10.0 (TID 186) in 1826 ms on 10.0.0.133 (executor driver) (171/208) +26/04/01 08:38:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:34 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:34 INFO Executor: Finished task 171.0 in stage 10.0 (TID 188). 2251 bytes result sent to driver +26/04/01 08:38:34 INFO TaskSetManager: Starting task 179.0 in stage 10.0 (TID 196) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:34 INFO Executor: Running task 179.0 in stage 10.0 (TID 196) +26/04/01 08:38:34 INFO TaskSetManager: Finished task 171.0 in stage 10.0 (TID 188) in 1691 ms on 10.0.0.133 (executor driver) (172/208) +26/04/01 08:38:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:34 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:35 INFO Executor: Finished task 172.0 in stage 10.0 (TID 189). 2251 bytes result sent to driver +26/04/01 08:38:35 INFO TaskSetManager: Starting task 180.0 in stage 10.0 (TID 197) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:35 INFO TaskSetManager: Finished task 172.0 in stage 10.0 (TID 189) in 1634 ms on 10.0.0.133 (executor driver) (173/208) +26/04/01 08:38:35 INFO Executor: Running task 180.0 in stage 10.0 (TID 197) +26/04/01 08:38:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:35 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:35 INFO Executor: Finished task 173.0 in stage 10.0 (TID 190). 2294 bytes result sent to driver +26/04/01 08:38:35 INFO TaskSetManager: Starting task 181.0 in stage 10.0 (TID 198) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:35 INFO TaskSetManager: Finished task 173.0 in stage 10.0 (TID 190) in 1633 ms on 10.0.0.133 (executor driver) (174/208) +26/04/01 08:38:35 INFO Executor: Running task 181.0 in stage 10.0 (TID 198) +26/04/01 08:38:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:35 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:35 INFO Executor: Finished task 174.0 in stage 10.0 (TID 191). 2251 bytes result sent to driver +26/04/01 08:38:35 INFO TaskSetManager: Starting task 182.0 in stage 10.0 (TID 199) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:35 INFO Executor: Running task 182.0 in stage 10.0 (TID 199) +26/04/01 08:38:35 INFO TaskSetManager: Finished task 174.0 in stage 10.0 (TID 191) in 1604 ms on 10.0.0.133 (executor driver) (175/208) +26/04/01 08:38:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:35 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:35 INFO Executor: Finished task 175.0 in stage 10.0 (TID 192). 2251 bytes result sent to driver +26/04/01 08:38:35 INFO TaskSetManager: Starting task 183.0 in stage 10.0 (TID 200) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:35 INFO TaskSetManager: Finished task 175.0 in stage 10.0 (TID 192) in 1552 ms on 10.0.0.133 (executor driver) (176/208) +26/04/01 08:38:35 INFO Executor: Running task 183.0 in stage 10.0 (TID 200) +26/04/01 08:38:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:35 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:36 INFO Executor: Finished task 176.0 in stage 10.0 (TID 193). 2251 bytes result sent to driver +26/04/01 08:38:36 INFO TaskSetManager: Starting task 184.0 in stage 10.0 (TID 201) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:36 INFO TaskSetManager: Finished task 176.0 in stage 10.0 (TID 193) in 1582 ms on 10.0.0.133 (executor driver) (177/208) +26/04/01 08:38:36 INFO Executor: Running task 184.0 in stage 10.0 (TID 201) +26/04/01 08:38:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:36 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:36 INFO Executor: Finished task 177.0 in stage 10.0 (TID 194). 2251 bytes result sent to driver +26/04/01 08:38:36 INFO TaskSetManager: Starting task 185.0 in stage 10.0 (TID 202) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:36 INFO TaskSetManager: Finished task 177.0 in stage 10.0 (TID 194) in 1578 ms on 10.0.0.133 (executor driver) (178/208) +26/04/01 08:38:36 INFO Executor: Running task 185.0 in stage 10.0 (TID 202) +26/04/01 08:38:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:36 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:36 INFO Executor: Finished task 178.0 in stage 10.0 (TID 195). 2251 bytes result sent to driver +26/04/01 08:38:36 INFO TaskSetManager: Starting task 186.0 in stage 10.0 (TID 203) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:36 INFO TaskSetManager: Finished task 178.0 in stage 10.0 (TID 195) in 1566 ms on 10.0.0.133 (executor driver) (179/208) +26/04/01 08:38:36 INFO Executor: Running task 186.0 in stage 10.0 (TID 203) +26/04/01 08:38:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:36 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:36 INFO Executor: Finished task 179.0 in stage 10.0 (TID 196). 2251 bytes result sent to driver +26/04/01 08:38:36 INFO TaskSetManager: Starting task 187.0 in stage 10.0 (TID 204) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:36 INFO Executor: Running task 187.0 in stage 10.0 (TID 204) +26/04/01 08:38:36 INFO TaskSetManager: Finished task 179.0 in stage 10.0 (TID 196) in 1557 ms on 10.0.0.133 (executor driver) (180/208) +26/04/01 08:38:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 134217728-268435456, partition values: [empty row] +26/04/01 08:38:36 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:36 INFO Executor: Finished task 181.0 in stage 10.0 (TID 198). 2251 bytes result sent to driver +26/04/01 08:38:36 INFO TaskSetManager: Starting task 188.0 in stage 10.0 (TID 205) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:36 INFO TaskSetManager: Finished task 181.0 in stage 10.0 (TID 198) in 1664 ms on 10.0.0.133 (executor driver) (181/208) +26/04/01 08:38:36 INFO Executor: Running task 188.0 in stage 10.0 (TID 205) +26/04/01 08:38:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 268435456-402653184, partition values: [empty row] +26/04/01 08:38:36 INFO Executor: Finished task 180.0 in stage 10.0 (TID 197). 2251 bytes result sent to driver +26/04/01 08:38:36 INFO TaskSetManager: Starting task 189.0 in stage 10.0 (TID 206) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:36 INFO Executor: Running task 189.0 in stage 10.0 (TID 206) +26/04/01 08:38:36 INFO TaskSetManager: Finished task 180.0 in stage 10.0 (TID 197) in 1676 ms on 10.0.0.133 (executor driver) (182/208) +26/04/01 08:38:36 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 402653184-536870912, partition values: [empty row] +26/04/01 08:38:36 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:36 INFO Executor: Finished task 182.0 in stage 10.0 (TID 199). 2251 bytes result sent to driver +26/04/01 08:38:36 INFO TaskSetManager: Starting task 190.0 in stage 10.0 (TID 207) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:36 INFO TaskSetManager: Finished task 182.0 in stage 10.0 (TID 199) in 1599 ms on 10.0.0.133 (executor driver) (183/208) +26/04/01 08:38:36 INFO Executor: Running task 190.0 in stage 10.0 (TID 207) +26/04/01 08:38:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 536870912-671088640, partition values: [empty row] +26/04/01 08:38:36 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:37 INFO Executor: Finished task 183.0 in stage 10.0 (TID 200). 2251 bytes result sent to driver +26/04/01 08:38:37 INFO TaskSetManager: Starting task 191.0 in stage 10.0 (TID 208) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:37 INFO Executor: Running task 191.0 in stage 10.0 (TID 208) +26/04/01 08:38:37 INFO TaskSetManager: Finished task 183.0 in stage 10.0 (TID 200) in 1645 ms on 10.0.0.133 (executor driver) (184/208) +26/04/01 08:38:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 671088640-805306368, partition values: [empty row] +26/04/01 08:38:37 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:37 INFO Executor: Finished task 184.0 in stage 10.0 (TID 201). 2251 bytes result sent to driver +26/04/01 08:38:37 INFO TaskSetManager: Starting task 192.0 in stage 10.0 (TID 209) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9710 bytes) +26/04/01 08:38:37 INFO TaskSetManager: Finished task 184.0 in stage 10.0 (TID 201) in 1703 ms on 10.0.0.133 (executor driver) (185/208) +26/04/01 08:38:37 INFO Executor: Running task 192.0 in stage 10.0 (TID 209) +26/04/01 08:38:37 INFO Executor: Finished task 185.0 in stage 10.0 (TID 202). 2251 bytes result sent to driver +26/04/01 08:38:37 INFO TaskSetManager: Starting task 193.0 in stage 10.0 (TID 210) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9709 bytes) +26/04/01 08:38:37 INFO TaskSetManager: Finished task 185.0 in stage 10.0 (TID 202) in 1649 ms on 10.0.0.133 (executor driver) (186/208) +26/04/01 08:38:37 INFO Executor: Running task 193.0 in stage 10.0 (TID 210) +26/04/01 08:38:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 805306368-848649288, partition values: [empty row] +26/04/01 08:38:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 805306368-848563442, partition values: [empty row] +26/04/01 08:38:37 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:37 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:38 INFO Executor: Finished task 186.0 in stage 10.0 (TID 203). 2251 bytes result sent to driver +26/04/01 08:38:38 INFO TaskSetManager: Starting task 194.0 in stage 10.0 (TID 211) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9710 bytes) +26/04/01 08:38:38 INFO Executor: Running task 194.0 in stage 10.0 (TID 211) +26/04/01 08:38:38 INFO TaskSetManager: Finished task 186.0 in stage 10.0 (TID 203) in 1743 ms on 10.0.0.133 (executor driver) (187/208) +26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 805306368-848496433, partition values: [empty row] +26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:38 INFO Executor: Finished task 187.0 in stage 10.0 (TID 204). 2251 bytes result sent to driver +26/04/01 08:38:38 INFO TaskSetManager: Starting task 195.0 in stage 10.0 (TID 212) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9710 bytes) +26/04/01 08:38:38 INFO TaskSetManager: Finished task 187.0 in stage 10.0 (TID 204) in 1663 ms on 10.0.0.133 (executor driver) (188/208) +26/04/01 08:38:38 INFO Executor: Running task 195.0 in stage 10.0 (TID 212) +26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 805306368-848463796, partition values: [empty row] +26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 805306368-848521656, partition values: [empty row] +26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 805306368-848594494, partition values: [empty row] +26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 805306368-848442878, partition values: [empty row] +26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 805306368-848489219, partition values: [empty row] +26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:38 INFO Executor: Finished task 188.0 in stage 10.0 (TID 205). 2251 bytes result sent to driver +26/04/01 08:38:38 INFO TaskSetManager: Starting task 196.0 in stage 10.0 (TID 213) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9709 bytes) +26/04/01 08:38:38 INFO TaskSetManager: Finished task 188.0 in stage 10.0 (TID 205) in 1688 ms on 10.0.0.133 (executor driver) (189/208) +26/04/01 08:38:38 INFO Executor: Running task 196.0 in stage 10.0 (TID 213) +26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 805306368-848432523, partition values: [empty row] +26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:38 INFO Executor: Finished task 189.0 in stage 10.0 (TID 206). 2294 bytes result sent to driver +26/04/01 08:38:38 INFO TaskSetManager: Starting task 197.0 in stage 10.0 (TID 214) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9709 bytes) +26/04/01 08:38:38 INFO TaskSetManager: Finished task 189.0 in stage 10.0 (TID 206) in 1769 ms on 10.0.0.133 (executor driver) (190/208) +26/04/01 08:38:38 INFO Executor: Running task 197.0 in stage 10.0 (TID 214) +26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 805306368-848422348, partition values: [empty row] +26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:38 INFO Executor: Finished task 190.0 in stage 10.0 (TID 207). 2294 bytes result sent to driver +26/04/01 08:38:38 INFO TaskSetManager: Starting task 198.0 in stage 10.0 (TID 215) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9710 bytes) +26/04/01 08:38:38 INFO TaskSetManager: Finished task 190.0 in stage 10.0 (TID 207) in 1709 ms on 10.0.0.133 (executor driver) (191/208) +26/04/01 08:38:38 INFO Executor: Running task 198.0 in stage 10.0 (TID 215) +26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 805306368-848399844, partition values: [empty row] +26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:38 INFO Executor: Finished task 191.0 in stage 10.0 (TID 208). 2251 bytes result sent to driver +26/04/01 08:38:38 INFO TaskSetManager: Starting task 199.0 in stage 10.0 (TID 216) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9710 bytes) +26/04/01 08:38:38 INFO TaskSetManager: Finished task 191.0 in stage 10.0 (TID 208) in 1581 ms on 10.0.0.133 (executor driver) (192/208) +26/04/01 08:38:38 INFO Executor: Running task 199.0 in stage 10.0 (TID 216) +26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 805306368-848378208, partition values: [empty row] +26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 805306368-848423659, partition values: [empty row] +26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 805306368-848411809, partition values: [empty row] +26/04/01 08:38:39 INFO Executor: Finished task 192.0 in stage 10.0 (TID 209). 2251 bytes result sent to driver +26/04/01 08:38:39 INFO TaskSetManager: Starting task 200.0 in stage 10.0 (TID 217) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9708 bytes) +26/04/01 08:38:39 INFO TaskSetManager: Finished task 192.0 in stage 10.0 (TID 209) in 1126 ms on 10.0.0.133 (executor driver) (193/208) +26/04/01 08:38:39 INFO Executor: Running task 200.0 in stage 10.0 (TID 217) +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 805306368-848344114, partition values: [empty row] +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO Executor: Finished task 193.0 in stage 10.0 (TID 210). 2251 bytes result sent to driver +26/04/01 08:38:39 INFO TaskSetManager: Starting task 201.0 in stage 10.0 (TID 218) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9709 bytes) +26/04/01 08:38:39 INFO TaskSetManager: Finished task 193.0 in stage 10.0 (TID 210) in 1132 ms on 10.0.0.133 (executor driver) (194/208) +26/04/01 08:38:39 INFO Executor: Running task 201.0 in stage 10.0 (TID 218) +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 805306368-848379799, partition values: [empty row] +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 805306368-848329366, partition values: [empty row] +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 805306368-848348606, partition values: [empty row] +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO Executor: Finished task 194.0 in stage 10.0 (TID 211). 2251 bytes result sent to driver +26/04/01 08:38:39 INFO TaskSetManager: Starting task 202.0 in stage 10.0 (TID 219) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9709 bytes) +26/04/01 08:38:39 INFO Executor: Finished task 195.0 in stage 10.0 (TID 212). 2251 bytes result sent to driver +26/04/01 08:38:39 INFO Executor: Running task 202.0 in stage 10.0 (TID 219) +26/04/01 08:38:39 INFO TaskSetManager: Starting task 203.0 in stage 10.0 (TID 220) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9710 bytes) +26/04/01 08:38:39 INFO TaskSetManager: Finished task 194.0 in stage 10.0 (TID 211) in 1127 ms on 10.0.0.133 (executor driver) (195/208) +26/04/01 08:38:39 INFO Executor: Running task 203.0 in stage 10.0 (TID 220) +26/04/01 08:38:39 INFO TaskSetManager: Finished task 195.0 in stage 10.0 (TID 212) in 1120 ms on 10.0.0.133 (executor driver) (196/208) +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 805306368-848328570, partition values: [empty row] +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 805306368-848301737, partition values: [empty row] +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 805306368-848338051, partition values: [empty row] +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 805306368-848329327, partition values: [empty row] +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO Executor: Finished task 196.0 in stage 10.0 (TID 213). 2251 bytes result sent to driver +26/04/01 08:38:39 INFO TaskSetManager: Starting task 204.0 in stage 10.0 (TID 221) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9710 bytes) +26/04/01 08:38:39 INFO TaskSetManager: Finished task 196.0 in stage 10.0 (TID 213) in 1032 ms on 10.0.0.133 (executor driver) (197/208) +26/04/01 08:38:39 INFO Executor: Running task 204.0 in stage 10.0 (TID 221) +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 805306368-848263904, partition values: [empty row] +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 805306368-848306153, partition values: [empty row] +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 805306368-848278425, partition values: [empty row] +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO Executor: Finished task 197.0 in stage 10.0 (TID 214). 2251 bytes result sent to driver +26/04/01 08:38:39 INFO TaskSetManager: Starting task 205.0 in stage 10.0 (TID 222) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9710 bytes) +26/04/01 08:38:39 INFO TaskSetManager: Finished task 197.0 in stage 10.0 (TID 214) in 1086 ms on 10.0.0.133 (executor driver) (198/208) +26/04/01 08:38:39 INFO Executor: Running task 205.0 in stage 10.0 (TID 222) +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 805306368-848235021, partition values: [empty row] +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO Executor: Finished task 198.0 in stage 10.0 (TID 215). 2251 bytes result sent to driver +26/04/01 08:38:39 INFO TaskSetManager: Starting task 206.0 in stage 10.0 (TID 223) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9709 bytes) +26/04/01 08:38:39 INFO TaskSetManager: Finished task 198.0 in stage 10.0 (TID 215) in 1096 ms on 10.0.0.133 (executor driver) (199/208) +26/04/01 08:38:39 INFO Executor: Running task 206.0 in stage 10.0 (TID 223) +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 805306368-848107384, partition values: [empty row] +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO Executor: Finished task 199.0 in stage 10.0 (TID 216). 2251 bytes result sent to driver +26/04/01 08:38:39 INFO TaskSetManager: Starting task 207.0 in stage 10.0 (TID 224) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9709 bytes) +26/04/01 08:38:39 INFO TaskSetManager: Finished task 199.0 in stage 10.0 (TID 216) in 1050 ms on 10.0.0.133 (executor driver) (200/208) +26/04/01 08:38:39 INFO Executor: Running task 207.0 in stage 10.0 (TID 224) +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 805306368-848005421, partition values: [empty row] +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 805306368-848247851, partition values: [empty row] +26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 805306368-848234776, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 805306368-848050794, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:40 INFO Executor: Finished task 200.0 in stage 10.0 (TID 217). 2251 bytes result sent to driver +26/04/01 08:38:40 INFO TaskSetManager: Starting task 0.0 in stage 11.0 (TID 225) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9924 bytes) +26/04/01 08:38:40 INFO TaskSetManager: Finished task 200.0 in stage 10.0 (TID 217) in 1195 ms on 10.0.0.133 (executor driver) (201/208) +26/04/01 08:38:40 INFO Executor: Running task 0.0 in stage 11.0 (TID 225) +26/04/01 08:38:40 INFO CodeGenerator: Code generated in 3.62975 ms +26/04/01 08:38:40 INFO CodeGenerator: Code generated in 3.54975 ms +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.9.parquet, range: 0-2801305, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO Executor: Finished task 201.0 in stage 10.0 (TID 218). 2251 bytes result sent to driver +26/04/01 08:38:40 INFO TaskSetManager: Starting task 1.0 in stage 11.0 (TID 226) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9925 bytes) +26/04/01 08:38:40 INFO TaskSetManager: Finished task 201.0 in stage 10.0 (TID 218) in 1220 ms on 10.0.0.133 (executor driver) (202/208) +26/04/01 08:38:40 INFO Executor: Running task 1.0 in stage 11.0 (TID 226) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.31.parquet, range: 0-2799730, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.21.parquet, range: 0-2800748, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 805306368-847699715, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.30.parquet, range: 0-2799509, partition values: [empty row] +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.13.parquet, range: 0-2800604, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.25.parquet, range: 0-2798499, partition values: [empty row] +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.1.parquet, range: 0-2799999, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO Executor: Finished task 203.0 in stage 10.0 (TID 220). 2251 bytes result sent to driver +26/04/01 08:38:40 INFO TaskSetManager: Starting task 2.0 in stage 11.0 (TID 227) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9925 bytes) +26/04/01 08:38:40 INFO Executor: Running task 2.0 in stage 11.0 (TID 227) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.6.parquet, range: 0-2798414, partition values: [empty row] +26/04/01 08:38:40 INFO TaskSetManager: Finished task 203.0 in stage 10.0 (TID 220) in 1188 ms on 10.0.0.133 (executor driver) (203/208) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.11.parquet, range: 0-2798396, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO Executor: Finished task 202.0 in stage 10.0 (TID 219). 2251 bytes result sent to driver +26/04/01 08:38:40 INFO TaskSetManager: Starting task 3.0 in stage 11.0 (TID 228) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9926 bytes) +26/04/01 08:38:40 INFO TaskSetManager: Finished task 202.0 in stage 10.0 (TID 219) in 1203 ms on 10.0.0.133 (executor driver) (204/208) +26/04/01 08:38:40 INFO Executor: Running task 3.0 in stage 11.0 (TID 228) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.26.parquet, range: 0-2797291, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.18.parquet, range: 0-2797632, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.2.parquet, range: 0-2797502, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.12.parquet, range: 0-2797316, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.14.parquet, range: 0-2797103, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.22.parquet, range: 0-2797014, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.17.parquet, range: 0-2795731, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO Executor: Finished task 0.0 in stage 11.0 (TID 225). 2208 bytes result sent to driver +26/04/01 08:38:40 INFO TaskSetManager: Starting task 4.0 in stage 11.0 (TID 229) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9926 bytes) +26/04/01 08:38:40 INFO Executor: Running task 4.0 in stage 11.0 (TID 229) +26/04/01 08:38:40 INFO TaskSetManager: Finished task 0.0 in stage 11.0 (TID 225) in 184 ms on 10.0.0.133 (executor driver) (1/8) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.27.parquet, range: 0-2795177, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO Executor: Finished task 1.0 in stage 11.0 (TID 226). 2251 bytes result sent to driver +26/04/01 08:38:40 INFO TaskSetManager: Starting task 5.0 in stage 11.0 (TID 230) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9925 bytes) +26/04/01 08:38:40 INFO TaskSetManager: Finished task 1.0 in stage 11.0 (TID 226) in 166 ms on 10.0.0.133 (executor driver) (2/8) +26/04/01 08:38:40 INFO Executor: Running task 5.0 in stage 11.0 (TID 230) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.3.parquet, range: 0-2794406, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.10.parquet, range: 0-2795129, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO Executor: Finished task 2.0 in stage 11.0 (TID 227). 2208 bytes result sent to driver +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.19.parquet, range: 0-2794808, partition values: [empty row] +26/04/01 08:38:40 INFO TaskSetManager: Starting task 6.0 in stage 11.0 (TID 231) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9924 bytes) +26/04/01 08:38:40 INFO Executor: Running task 6.0 in stage 11.0 (TID 231) +26/04/01 08:38:40 INFO TaskSetManager: Finished task 2.0 in stage 11.0 (TID 227) in 125 ms on 10.0.0.133 (executor driver) (3/8) +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.32.parquet, range: 0-2792572, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.15.parquet, range: 0-2794714, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.16.parquet, range: 0-2794298, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.24.parquet, range: 0-2793929, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO Executor: Finished task 3.0 in stage 11.0 (TID 228). 2251 bytes result sent to driver +26/04/01 08:38:40 INFO TaskSetManager: Starting task 7.0 in stage 11.0 (TID 232) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9924 bytes) +26/04/01 08:38:40 INFO Executor: Running task 7.0 in stage 11.0 (TID 232) +26/04/01 08:38:40 INFO TaskSetManager: Finished task 3.0 in stage 11.0 (TID 228) in 148 ms on 10.0.0.133 (executor driver) (4/8) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.20.parquet, range: 0-2791117, partition values: [empty row] +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.23.parquet, range: 0-2792937, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.5.parquet, range: 0-2792421, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.7.parquet, range: 0-2791953, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.29.parquet, range: 0-2791555, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.28.parquet, range: 0-2790847, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.8.parquet, range: 0-2790608, partition values: [empty row] +26/04/01 08:38:40 INFO Executor: Finished task 4.0 in stage 11.0 (TID 229). 2208 bytes result sent to driver +26/04/01 08:38:40 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 233) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:40 INFO Executor: Running task 0.0 in stage 12.0 (TID 233) +26/04/01 08:38:40 INFO TaskSetManager: Finished task 4.0 in stage 11.0 (TID 229) in 164 ms on 10.0.0.133 (executor driver) (5/8) +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO CodeGenerator: Code generated in 3.386042 ms +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.4.parquet, range: 0-2789514, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) +26/04/01 08:38:40 INFO CodeGenerator: Code generated in 3.224208 ms +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.15.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:40 INFO Executor: Finished task 5.0 in stage 11.0 (TID 230). 2208 bytes result sent to driver +26/04/01 08:38:40 INFO TaskSetManager: Starting task 1.0 in stage 12.0 (TID 234) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:40 INFO TaskSetManager: Finished task 5.0 in stage 11.0 (TID 230) in 207 ms on 10.0.0.133 (executor driver) (6/8) +26/04/01 08:38:40 INFO Executor: Running task 1.0 in stage 12.0 (TID 234) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.23.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:40 INFO Executor: Finished task 6.0 in stage 11.0 (TID 231). 2208 bytes result sent to driver +26/04/01 08:38:40 INFO TaskSetManager: Starting task 2.0 in stage 12.0 (TID 235) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:40 INFO TaskSetManager: Finished task 6.0 in stage 11.0 (TID 231) in 203 ms on 10.0.0.133 (executor driver) (7/8) +26/04/01 08:38:40 INFO Executor: Running task 2.0 in stage 12.0 (TID 235) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.9.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:40 INFO Executor: Finished task 7.0 in stage 11.0 (TID 232). 2208 bytes result sent to driver +26/04/01 08:38:40 INFO TaskSetManager: Starting task 3.0 in stage 12.0 (TID 236) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:40 INFO Executor: Running task 3.0 in stage 12.0 (TID 236) +26/04/01 08:38:40 INFO TaskSetManager: Finished task 7.0 in stage 11.0 (TID 232) in 250 ms on 10.0.0.133 (executor driver) (8/8) +26/04/01 08:38:40 INFO TaskSchedulerImpl: Removed TaskSet 11.0, whose tasks have all completed, from pool +26/04/01 08:38:40 INFO DAGScheduler: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 33.513 s +26/04/01 08:38:40 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:38:40 INFO DAGScheduler: running: Set(ShuffleMapStage 12, ShuffleMapStage 13, ShuffleMapStage 10) +26/04/01 08:38:40 INFO DAGScheduler: waiting: Set() +26/04/01 08:38:40 INFO DAGScheduler: failed: Set() +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.22.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:40 INFO Executor: Finished task 204.0 in stage 10.0 (TID 221). 2251 bytes result sent to driver +26/04/01 08:38:40 INFO TaskSetManager: Starting task 4.0 in stage 12.0 (TID 237) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:40 INFO Executor: Running task 4.0 in stage 12.0 (TID 237) +26/04/01 08:38:40 INFO TaskSetManager: Finished task 204.0 in stage 10.0 (TID 221) in 1223 ms on 10.0.0.133 (executor driver) (205/208) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.32.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:40 INFO Executor: Finished task 205.0 in stage 10.0 (TID 222). 2251 bytes result sent to driver +26/04/01 08:38:40 INFO TaskSetManager: Starting task 5.0 in stage 12.0 (TID 238) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:40 INFO TaskSetManager: Finished task 205.0 in stage 10.0 (TID 222) in 1232 ms on 10.0.0.133 (executor driver) (206/208) +26/04/01 08:38:40 INFO Executor: Running task 5.0 in stage 12.0 (TID 238) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.8.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:40 INFO Executor: Finished task 206.0 in stage 10.0 (TID 223). 2251 bytes result sent to driver +26/04/01 08:38:40 INFO TaskSetManager: Starting task 6.0 in stage 12.0 (TID 239) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:40 INFO Executor: Running task 6.0 in stage 12.0 (TID 239) +26/04/01 08:38:40 INFO TaskSetManager: Finished task 206.0 in stage 10.0 (TID 223) in 1246 ms on 10.0.0.133 (executor driver) (207/208) +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.1.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:40 INFO Executor: Finished task 207.0 in stage 10.0 (TID 224). 2251 bytes result sent to driver +26/04/01 08:38:40 INFO TaskSetManager: Starting task 7.0 in stage 12.0 (TID 240) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:40 INFO TaskSetManager: Finished task 207.0 in stage 10.0 (TID 224) in 1193 ms on 10.0.0.133 (executor driver) (208/208) +26/04/01 08:38:40 INFO Executor: Running task 7.0 in stage 12.0 (TID 240) +26/04/01 08:38:40 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool +26/04/01 08:38:40 INFO DAGScheduler: ShuffleMapStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 33.763 s +26/04/01 08:38:40 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:38:40 INFO DAGScheduler: running: Set(ShuffleMapStage 12, ShuffleMapStage 13) +26/04/01 08:38:40 INFO DAGScheduler: waiting: Set() +26/04/01 08:38:40 INFO DAGScheduler: failed: Set() +26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.14.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:40 INFO ShufflePartitionsUtil: For shuffle(0, 1), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 08:38:41 INFO CodeGenerator: Code generated in 10.615 ms +26/04/01 08:38:41 INFO CodeGenerator: Code generated in 4.236625 ms +26/04/01 08:38:41 INFO CodeGenerator: Code generated in 2.943458 ms +26/04/01 08:38:41 INFO DAGScheduler: Registering RDD 46 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 5 +26/04/01 08:38:41 INFO DAGScheduler: Got map stage job 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions +26/04/01 08:38:41 INFO DAGScheduler: Final stage: ShuffleMapStage 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:38:41 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 15, ShuffleMapStage 14) +26/04/01 08:38:41 INFO DAGScheduler: Missing parents: List() +26/04/01 08:38:41 INFO DAGScheduler: Submitting ShuffleMapStage 16 (MapPartitionsRDD[46] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:38:41 INFO MemoryStore: Block broadcast_21 stored as values in memory (estimated size 55.2 KiB, free 8.6 GiB) +26/04/01 08:38:41 INFO MemoryStore: Block broadcast_21_piece0 stored as bytes in memory (estimated size 26.3 KiB, free 8.6 GiB) +26/04/01 08:38:41 INFO BlockManagerInfo: Added broadcast_21_piece0 in memory on 10.0.0.133:59288 (size: 26.3 KiB, free: 8.6 GiB) +26/04/01 08:38:41 INFO SparkContext: Created broadcast 21 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:38:41 INFO DAGScheduler: Submitting 200 missing tasks from ShuffleMapStage 16 (MapPartitionsRDD[46] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:38:41 INFO TaskSchedulerImpl: Adding task set 16.0 with 200 tasks resource profile 0 +26/04/01 08:38:41 INFO Executor: Finished task 0.0 in stage 12.0 (TID 233). 2251 bytes result sent to driver +26/04/01 08:38:41 INFO TaskSetManager: Starting task 8.0 in stage 12.0 (TID 241) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:41 INFO Executor: Running task 8.0 in stage 12.0 (TID 241) +26/04/01 08:38:41 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 233) in 1007 ms on 10.0.0.133 (executor driver) (1/37) +26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.30.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:41 INFO Executor: Finished task 2.0 in stage 12.0 (TID 235). 2251 bytes result sent to driver +26/04/01 08:38:41 INFO TaskSetManager: Starting task 9.0 in stage 12.0 (TID 242) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:41 INFO Executor: Running task 9.0 in stage 12.0 (TID 242) +26/04/01 08:38:41 INFO TaskSetManager: Finished task 2.0 in stage 12.0 (TID 235) in 1002 ms on 10.0.0.133 (executor driver) (2/37) +26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.20.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:41 INFO Executor: Finished task 1.0 in stage 12.0 (TID 234). 2251 bytes result sent to driver +26/04/01 08:38:41 INFO TaskSetManager: Starting task 10.0 in stage 12.0 (TID 243) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:41 INFO TaskSetManager: Finished task 1.0 in stage 12.0 (TID 234) in 1041 ms on 10.0.0.133 (executor driver) (3/37) +26/04/01 08:38:41 INFO Executor: Running task 10.0 in stage 12.0 (TID 243) +26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.29.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:41 INFO Executor: Finished task 4.0 in stage 12.0 (TID 237). 2251 bytes result sent to driver +26/04/01 08:38:41 INFO TaskSetManager: Starting task 11.0 in stage 12.0 (TID 244) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:41 INFO Executor: Running task 11.0 in stage 12.0 (TID 244) +26/04/01 08:38:41 INFO TaskSetManager: Finished task 4.0 in stage 12.0 (TID 237) in 941 ms on 10.0.0.133 (executor driver) (4/37) +26/04/01 08:38:41 INFO Executor: Finished task 3.0 in stage 12.0 (TID 236). 2251 bytes result sent to driver +26/04/01 08:38:41 INFO TaskSetManager: Starting task 12.0 in stage 12.0 (TID 245) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:41 INFO TaskSetManager: Finished task 3.0 in stage 12.0 (TID 236) in 960 ms on 10.0.0.133 (executor driver) (5/37) +26/04/01 08:38:41 INFO Executor: Running task 12.0 in stage 12.0 (TID 245) +26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.3.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.16.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:41 INFO Executor: Finished task 5.0 in stage 12.0 (TID 238). 2251 bytes result sent to driver +26/04/01 08:38:41 INFO TaskSetManager: Starting task 13.0 in stage 12.0 (TID 246) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:41 INFO TaskSetManager: Finished task 5.0 in stage 12.0 (TID 238) in 985 ms on 10.0.0.133 (executor driver) (6/37) +26/04/01 08:38:41 INFO Executor: Running task 13.0 in stage 12.0 (TID 246) +26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.28.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:41 INFO Executor: Finished task 6.0 in stage 12.0 (TID 239). 2251 bytes result sent to driver +26/04/01 08:38:41 INFO TaskSetManager: Starting task 14.0 in stage 12.0 (TID 247) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:41 INFO Executor: Running task 14.0 in stage 12.0 (TID 247) +26/04/01 08:38:41 INFO TaskSetManager: Finished task 6.0 in stage 12.0 (TID 239) in 1012 ms on 10.0.0.133 (executor driver) (7/37) +26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.2.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:41 INFO Executor: Finished task 7.0 in stage 12.0 (TID 240). 2251 bytes result sent to driver +26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:41 INFO TaskSetManager: Starting task 15.0 in stage 12.0 (TID 248) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:41 INFO TaskSetManager: Finished task 7.0 in stage 12.0 (TID 240) in 989 ms on 10.0.0.133 (executor driver) (8/37) +26/04/01 08:38:41 INFO Executor: Running task 15.0 in stage 12.0 (TID 248) +26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.17.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:42 INFO Executor: Finished task 8.0 in stage 12.0 (TID 241). 2251 bytes result sent to driver +26/04/01 08:38:42 INFO TaskSetManager: Starting task 16.0 in stage 12.0 (TID 249) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:42 INFO TaskSetManager: Finished task 8.0 in stage 12.0 (TID 241) in 758 ms on 10.0.0.133 (executor driver) (9/37) +26/04/01 08:38:42 INFO Executor: Running task 16.0 in stage 12.0 (TID 249) +26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.21.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:42 INFO Executor: Finished task 9.0 in stage 12.0 (TID 242). 2251 bytes result sent to driver +26/04/01 08:38:42 INFO TaskSetManager: Starting task 17.0 in stage 12.0 (TID 250) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:42 INFO TaskSetManager: Finished task 9.0 in stage 12.0 (TID 242) in 925 ms on 10.0.0.133 (executor driver) (10/37) +26/04/01 08:38:42 INFO Executor: Running task 17.0 in stage 12.0 (TID 250) +26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.31.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:42 INFO Executor: Finished task 10.0 in stage 12.0 (TID 243). 2251 bytes result sent to driver +26/04/01 08:38:42 INFO TaskSetManager: Starting task 18.0 in stage 12.0 (TID 251) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:42 INFO Executor: Running task 18.0 in stage 12.0 (TID 251) +26/04/01 08:38:42 INFO TaskSetManager: Finished task 10.0 in stage 12.0 (TID 243) in 990 ms on 10.0.0.133 (executor driver) (11/37) +26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.24.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:42 INFO Executor: Finished task 11.0 in stage 12.0 (TID 244). 2251 bytes result sent to driver +26/04/01 08:38:42 INFO TaskSetManager: Starting task 19.0 in stage 12.0 (TID 252) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:42 INFO Executor: Running task 19.0 in stage 12.0 (TID 252) +26/04/01 08:38:42 INFO TaskSetManager: Finished task 11.0 in stage 12.0 (TID 244) in 973 ms on 10.0.0.133 (executor driver) (12/37) +26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.12.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:42 INFO Executor: Finished task 12.0 in stage 12.0 (TID 245). 2251 bytes result sent to driver +26/04/01 08:38:42 INFO TaskSetManager: Starting task 20.0 in stage 12.0 (TID 253) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:42 INFO TaskSetManager: Finished task 12.0 in stage 12.0 (TID 245) in 987 ms on 10.0.0.133 (executor driver) (13/37) +26/04/01 08:38:42 INFO Executor: Running task 20.0 in stage 12.0 (TID 253) +26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.7.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:42 INFO Executor: Finished task 13.0 in stage 12.0 (TID 246). 2251 bytes result sent to driver +26/04/01 08:38:42 INFO TaskSetManager: Starting task 21.0 in stage 12.0 (TID 254) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:42 INFO TaskSetManager: Finished task 13.0 in stage 12.0 (TID 246) in 856 ms on 10.0.0.133 (executor driver) (14/37) +26/04/01 08:38:42 INFO Executor: Running task 21.0 in stage 12.0 (TID 254) +26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.13.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:42 INFO Executor: Finished task 15.0 in stage 12.0 (TID 248). 2251 bytes result sent to driver +26/04/01 08:38:42 INFO TaskSetManager: Starting task 22.0 in stage 12.0 (TID 255) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:42 INFO TaskSetManager: Finished task 15.0 in stage 12.0 (TID 248) in 868 ms on 10.0.0.133 (executor driver) (15/37) +26/04/01 08:38:42 INFO Executor: Running task 22.0 in stage 12.0 (TID 255) +26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.6.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:42 INFO Executor: Finished task 14.0 in stage 12.0 (TID 247). 2251 bytes result sent to driver +26/04/01 08:38:42 INFO TaskSetManager: Starting task 23.0 in stage 12.0 (TID 256) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:42 INFO Executor: Running task 23.0 in stage 12.0 (TID 256) +26/04/01 08:38:42 INFO TaskSetManager: Finished task 14.0 in stage 12.0 (TID 247) in 882 ms on 10.0.0.133 (executor driver) (16/37) +26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.25.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:43 INFO Executor: Finished task 16.0 in stage 12.0 (TID 249). 2251 bytes result sent to driver +26/04/01 08:38:43 INFO TaskSetManager: Starting task 24.0 in stage 12.0 (TID 257) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:43 INFO TaskSetManager: Finished task 16.0 in stage 12.0 (TID 249) in 792 ms on 10.0.0.133 (executor driver) (17/37) +26/04/01 08:38:43 INFO Executor: Running task 24.0 in stage 12.0 (TID 257) +26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.11.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:43 INFO Executor: Finished task 17.0 in stage 12.0 (TID 250). 2294 bytes result sent to driver +26/04/01 08:38:43 INFO TaskSetManager: Starting task 25.0 in stage 12.0 (TID 258) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:43 INFO TaskSetManager: Finished task 17.0 in stage 12.0 (TID 250) in 856 ms on 10.0.0.133 (executor driver) (18/37) +26/04/01 08:38:43 INFO Executor: Running task 25.0 in stage 12.0 (TID 258) +26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.4.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:43 INFO Executor: Finished task 18.0 in stage 12.0 (TID 251). 2251 bytes result sent to driver +26/04/01 08:38:43 INFO TaskSetManager: Starting task 26.0 in stage 12.0 (TID 259) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:43 INFO TaskSetManager: Finished task 18.0 in stage 12.0 (TID 251) in 810 ms on 10.0.0.133 (executor driver) (19/37) +26/04/01 08:38:43 INFO Executor: Running task 26.0 in stage 12.0 (TID 259) +26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.18.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:43 INFO Executor: Finished task 19.0 in stage 12.0 (TID 252). 2251 bytes result sent to driver +26/04/01 08:38:43 INFO TaskSetManager: Starting task 27.0 in stage 12.0 (TID 260) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:43 INFO Executor: Running task 27.0 in stage 12.0 (TID 260) +26/04/01 08:38:43 INFO TaskSetManager: Finished task 19.0 in stage 12.0 (TID 252) in 827 ms on 10.0.0.133 (executor driver) (20/37) +26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.27.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:43 INFO Executor: Finished task 20.0 in stage 12.0 (TID 253). 2251 bytes result sent to driver +26/04/01 08:38:43 INFO TaskSetManager: Starting task 28.0 in stage 12.0 (TID 261) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:43 INFO Executor: Running task 28.0 in stage 12.0 (TID 261) +26/04/01 08:38:43 INFO TaskSetManager: Finished task 20.0 in stage 12.0 (TID 253) in 910 ms on 10.0.0.133 (executor driver) (21/37) +26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.19.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:43 INFO Executor: Finished task 21.0 in stage 12.0 (TID 254). 2251 bytes result sent to driver +26/04/01 08:38:43 INFO TaskSetManager: Starting task 29.0 in stage 12.0 (TID 262) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:43 INFO TaskSetManager: Finished task 21.0 in stage 12.0 (TID 254) in 854 ms on 10.0.0.133 (executor driver) (22/37) +26/04/01 08:38:43 INFO Executor: Running task 29.0 in stage 12.0 (TID 262) +26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.26.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:43 INFO Executor: Finished task 22.0 in stage 12.0 (TID 255). 2251 bytes result sent to driver +26/04/01 08:38:43 INFO TaskSetManager: Starting task 30.0 in stage 12.0 (TID 263) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9602 bytes) +26/04/01 08:38:43 INFO Executor: Running task 30.0 in stage 12.0 (TID 263) +26/04/01 08:38:43 INFO TaskSetManager: Finished task 22.0 in stage 12.0 (TID 255) in 837 ms on 10.0.0.133 (executor driver) (23/37) +26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.10.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:43 INFO Executor: Finished task 23.0 in stage 12.0 (TID 256). 2251 bytes result sent to driver +26/04/01 08:38:43 INFO TaskSetManager: Starting task 31.0 in stage 12.0 (TID 264) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9601 bytes) +26/04/01 08:38:43 INFO TaskSetManager: Finished task 23.0 in stage 12.0 (TID 256) in 830 ms on 10.0.0.133 (executor driver) (24/37) +26/04/01 08:38:43 INFO Executor: Running task 31.0 in stage 12.0 (TID 264) +26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.5.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:43 INFO Executor: Finished task 24.0 in stage 12.0 (TID 257). 2251 bytes result sent to driver +26/04/01 08:38:43 INFO TaskSetManager: Starting task 32.0 in stage 12.0 (TID 265) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 10249 bytes) +26/04/01 08:38:43 INFO TaskSetManager: Finished task 24.0 in stage 12.0 (TID 257) in 766 ms on 10.0.0.133 (executor driver) (25/37) +26/04/01 08:38:43 INFO Executor: Running task 32.0 in stage 12.0 (TID 265) +26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.30.parquet, range: 134217728-148587359, partition values: [empty row] +26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.20.parquet, range: 134217728-148586863, partition values: [empty row] +26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.12.parquet, range: 134217728-148577687, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.13.parquet, range: 134217728-148575763, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.7.parquet, range: 134217728-148573022, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.21.parquet, range: 134217728-148568504, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO Executor: Finished task 25.0 in stage 12.0 (TID 258). 2294 bytes result sent to driver +26/04/01 08:38:44 INFO TaskSetManager: Starting task 33.0 in stage 12.0 (TID 266) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 10250 bytes) +26/04/01 08:38:44 INFO Executor: Running task 33.0 in stage 12.0 (TID 266) +26/04/01 08:38:44 INFO TaskSetManager: Finished task 25.0 in stage 12.0 (TID 258) in 833 ms on 10.0.0.133 (executor driver) (26/37) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.19.parquet, range: 134217728-148561478, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO Executor: Finished task 26.0 in stage 12.0 (TID 259). 2251 bytes result sent to driver +26/04/01 08:38:44 INFO TaskSetManager: Starting task 34.0 in stage 12.0 (TID 267) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 10246 bytes) +26/04/01 08:38:44 INFO TaskSetManager: Finished task 26.0 in stage 12.0 (TID 259) in 869 ms on 10.0.0.133 (executor driver) (27/37) +26/04/01 08:38:44 INFO Executor: Running task 34.0 in stage 12.0 (TID 267) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.1.parquet, range: 134217728-148552923, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.32.parquet, range: 134217728-148564089, partition values: [empty row] +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.26.parquet, range: 134217728-148560884, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.27.parquet, range: 134217728-148559546, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.2.parquet, range: 134217728-148551531, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO Executor: Finished task 27.0 in stage 12.0 (TID 260). 2251 bytes result sent to driver +26/04/01 08:38:44 INFO TaskSetManager: Starting task 35.0 in stage 12.0 (TID 268) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 10249 bytes) +26/04/01 08:38:44 INFO TaskSetManager: Finished task 27.0 in stage 12.0 (TID 260) in 944 ms on 10.0.0.133 (executor driver) (28/37) +26/04/01 08:38:44 INFO Executor: Running task 35.0 in stage 12.0 (TID 268) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.14.parquet, range: 134217728-148540833, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.28.parquet, range: 134217728-148544471, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.15.parquet, range: 134217728-148558797, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.31.parquet, range: 134217728-148540689, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.10.parquet, range: 134217728-148543426, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.29.parquet, range: 134217728-148539628, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO Executor: Finished task 28.0 in stage 12.0 (TID 261). 2251 bytes result sent to driver +26/04/01 08:38:44 INFO TaskSetManager: Starting task 36.0 in stage 12.0 (TID 269) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9923 bytes) +26/04/01 08:38:44 INFO Executor: Running task 36.0 in stage 12.0 (TID 269) +26/04/01 08:38:44 INFO TaskSetManager: Finished task 28.0 in stage 12.0 (TID 261) in 976 ms on 10.0.0.133 (executor driver) (29/37) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.6.parquet, range: 134217728-148521236, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO Executor: Finished task 29.0 in stage 12.0 (TID 262). 2251 bytes result sent to driver +26/04/01 08:38:44 INFO TaskSetManager: Starting task 0.0 in stage 13.0 (TID 270) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:44 INFO Executor: Running task 0.0 in stage 13.0 (TID 270) +26/04/01 08:38:44 INFO TaskSetManager: Finished task 29.0 in stage 12.0 (TID 262) in 993 ms on 10.0.0.133 (executor driver) (30/37) +26/04/01 08:38:44 INFO CodeGenerator: Code generated in 3.345584 ms +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.23.parquet, range: 134217728-148558546, partition values: [empty row] +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.13.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:44 INFO Executor: Finished task 30.0 in stage 12.0 (TID 263). 2251 bytes result sent to driver +26/04/01 08:38:44 INFO TaskSetManager: Starting task 1.0 in stage 13.0 (TID 271) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:44 INFO Executor: Running task 1.0 in stage 13.0 (TID 271) +26/04/01 08:38:44 INFO TaskSetManager: Finished task 30.0 in stage 12.0 (TID 263) in 954 ms on 10.0.0.133 (executor driver) (31/37) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.7.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:44 INFO Executor: Finished task 31.0 in stage 12.0 (TID 264). 2251 bytes result sent to driver +26/04/01 08:38:44 INFO TaskSetManager: Starting task 2.0 in stage 13.0 (TID 272) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:44 INFO Executor: Running task 2.0 in stage 13.0 (TID 272) +26/04/01 08:38:44 INFO TaskSetManager: Finished task 31.0 in stage 12.0 (TID 264) in 957 ms on 10.0.0.133 (executor driver) (32/37) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.25.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.8.parquet, range: 134217728-148542255, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO Executor: Finished task 32.0 in stage 12.0 (TID 265). 2251 bytes result sent to driver +26/04/01 08:38:44 INFO TaskSetManager: Starting task 3.0 in stage 13.0 (TID 273) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:44 INFO TaskSetManager: Finished task 32.0 in stage 12.0 (TID 265) in 748 ms on 10.0.0.133 (executor driver) (33/37) +26/04/01 08:38:44 INFO Executor: Running task 3.0 in stage 13.0 (TID 273) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.24.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.25.parquet, range: 134217728-148557400, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.16.parquet, range: 134217728-148538785, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.4.parquet, range: 134217728-148516007, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.24.parquet, range: 134217728-148542182, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.17.parquet, range: 134217728-148515342, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.11.parquet, range: 134217728-148554765, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.9.parquet, range: 134217728-148538414, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.5.parquet, range: 134217728-148541392, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.3.parquet, range: 134217728-148499293, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.22.parquet, range: 134217728-148525436, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.18.parquet, range: 134217728-148522266, partition values: [empty row] +26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) +26/04/01 08:38:45 INFO Executor: Finished task 33.0 in stage 12.0 (TID 266). 2251 bytes result sent to driver +26/04/01 08:38:45 INFO TaskSetManager: Starting task 4.0 in stage 13.0 (TID 274) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:45 INFO TaskSetManager: Finished task 33.0 in stage 12.0 (TID 266) in 767 ms on 10.0.0.133 (executor driver) (34/37) +26/04/01 08:38:45 INFO Executor: Running task 4.0 in stage 13.0 (TID 274) +26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.12.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:45 INFO Executor: Finished task 36.0 in stage 12.0 (TID 269). 2208 bytes result sent to driver +26/04/01 08:38:45 INFO TaskSetManager: Starting task 5.0 in stage 13.0 (TID 275) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:45 INFO TaskSetManager: Finished task 36.0 in stage 12.0 (TID 269) in 482 ms on 10.0.0.133 (executor driver) (35/37) +26/04/01 08:38:45 INFO Executor: Running task 5.0 in stage 13.0 (TID 275) +26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.6.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:45 INFO Executor: Finished task 34.0 in stage 12.0 (TID 267). 2251 bytes result sent to driver +26/04/01 08:38:45 INFO TaskSetManager: Starting task 6.0 in stage 13.0 (TID 276) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:45 INFO TaskSetManager: Finished task 34.0 in stage 12.0 (TID 267) in 764 ms on 10.0.0.133 (executor driver) (36/37) +26/04/01 08:38:45 INFO Executor: Running task 6.0 in stage 13.0 (TID 276) +26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.19.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:45 INFO Executor: Finished task 35.0 in stage 12.0 (TID 268). 2208 bytes result sent to driver +26/04/01 08:38:45 INFO TaskSetManager: Starting task 7.0 in stage 13.0 (TID 277) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:45 INFO TaskSetManager: Finished task 35.0 in stage 12.0 (TID 268) in 690 ms on 10.0.0.133 (executor driver) (37/37) +26/04/01 08:38:45 INFO Executor: Running task 7.0 in stage 13.0 (TID 277) +26/04/01 08:38:45 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool +26/04/01 08:38:45 INFO DAGScheduler: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 37.888 s +26/04/01 08:38:45 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:38:45 INFO DAGScheduler: running: Set(ShuffleMapStage 16, ShuffleMapStage 13) +26/04/01 08:38:45 INFO DAGScheduler: waiting: Set() +26/04/01 08:38:45 INFO DAGScheduler: failed: Set() +26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.26.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:45 INFO Executor: Finished task 2.0 in stage 13.0 (TID 272). 2208 bytes result sent to driver +26/04/01 08:38:45 INFO TaskSetManager: Starting task 8.0 in stage 13.0 (TID 278) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:45 INFO Executor: Running task 8.0 in stage 13.0 (TID 278) +26/04/01 08:38:45 INFO TaskSetManager: Finished task 2.0 in stage 13.0 (TID 272) in 934 ms on 10.0.0.133 (executor driver) (1/64) +26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.4.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:45 INFO Executor: Finished task 0.0 in stage 13.0 (TID 270). 2251 bytes result sent to driver +26/04/01 08:38:45 INFO TaskSetManager: Starting task 9.0 in stage 13.0 (TID 279) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:45 INFO Executor: Running task 9.0 in stage 13.0 (TID 279) +26/04/01 08:38:45 INFO TaskSetManager: Finished task 0.0 in stage 13.0 (TID 270) in 993 ms on 10.0.0.133 (executor driver) (2/64) +26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.10.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:45 INFO Executor: Finished task 3.0 in stage 13.0 (TID 273). 2251 bytes result sent to driver +26/04/01 08:38:45 INFO TaskSetManager: Starting task 10.0 in stage 13.0 (TID 280) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:45 INFO TaskSetManager: Finished task 3.0 in stage 13.0 (TID 273) in 958 ms on 10.0.0.133 (executor driver) (3/64) +26/04/01 08:38:45 INFO Executor: Running task 10.0 in stage 13.0 (TID 280) +26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.5.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:45 INFO Executor: Finished task 1.0 in stage 13.0 (TID 271). 2251 bytes result sent to driver +26/04/01 08:38:45 INFO TaskSetManager: Starting task 11.0 in stage 13.0 (TID 281) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:45 INFO TaskSetManager: Finished task 1.0 in stage 13.0 (TID 271) in 979 ms on 10.0.0.133 (executor driver) (4/64) +26/04/01 08:38:45 INFO Executor: Running task 11.0 in stage 13.0 (TID 281) +26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.11.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:45 INFO Executor: Finished task 4.0 in stage 13.0 (TID 274). 2251 bytes result sent to driver +26/04/01 08:38:45 INFO TaskSetManager: Starting task 12.0 in stage 13.0 (TID 282) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:45 INFO Executor: Running task 12.0 in stage 13.0 (TID 282) +26/04/01 08:38:45 INFO TaskSetManager: Finished task 4.0 in stage 13.0 (TID 274) in 765 ms on 10.0.0.133 (executor driver) (5/64) +26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.18.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:45 INFO Executor: Finished task 5.0 in stage 13.0 (TID 275). 2251 bytes result sent to driver +26/04/01 08:38:45 INFO TaskSetManager: Starting task 13.0 in stage 13.0 (TID 283) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:45 INFO Executor: Running task 13.0 in stage 13.0 (TID 283) +26/04/01 08:38:45 INFO TaskSetManager: Finished task 5.0 in stage 13.0 (TID 275) in 805 ms on 10.0.0.133 (executor driver) (6/64) +26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.27.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:45 INFO Executor: Finished task 6.0 in stage 13.0 (TID 276). 2251 bytes result sent to driver +26/04/01 08:38:45 INFO TaskSetManager: Starting task 14.0 in stage 13.0 (TID 284) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:45 INFO Executor: Running task 14.0 in stage 13.0 (TID 284) +26/04/01 08:38:45 INFO TaskSetManager: Finished task 6.0 in stage 13.0 (TID 276) in 790 ms on 10.0.0.133 (executor driver) (7/64) +26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.32.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:45 INFO Executor: Finished task 7.0 in stage 13.0 (TID 277). 2251 bytes result sent to driver +26/04/01 08:38:45 INFO TaskSetManager: Starting task 15.0 in stage 13.0 (TID 285) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:45 INFO TaskSetManager: Finished task 7.0 in stage 13.0 (TID 277) in 810 ms on 10.0.0.133 (executor driver) (8/64) +26/04/01 08:38:45 INFO Executor: Running task 15.0 in stage 13.0 (TID 285) +26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.22.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:46 INFO Executor: Finished task 8.0 in stage 13.0 (TID 278). 2251 bytes result sent to driver +26/04/01 08:38:46 INFO TaskSetManager: Starting task 16.0 in stage 13.0 (TID 286) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:46 INFO TaskSetManager: Finished task 8.0 in stage 13.0 (TID 278) in 874 ms on 10.0.0.133 (executor driver) (9/64) +26/04/01 08:38:46 INFO Executor: Running task 16.0 in stage 13.0 (TID 286) +26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.9.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:46 INFO Executor: Finished task 10.0 in stage 13.0 (TID 280). 2208 bytes result sent to driver +26/04/01 08:38:46 INFO TaskSetManager: Starting task 17.0 in stage 13.0 (TID 287) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:46 INFO Executor: Finished task 11.0 in stage 13.0 (TID 281). 2208 bytes result sent to driver +26/04/01 08:38:46 INFO Executor: Running task 17.0 in stage 13.0 (TID 287) +26/04/01 08:38:46 INFO TaskSetManager: Starting task 18.0 in stage 13.0 (TID 288) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:46 INFO Executor: Running task 18.0 in stage 13.0 (TID 288) +26/04/01 08:38:46 INFO TaskSetManager: Finished task 10.0 in stage 13.0 (TID 280) in 881 ms on 10.0.0.133 (executor driver) (10/64) +26/04/01 08:38:46 INFO TaskSetManager: Finished task 11.0 in stage 13.0 (TID 281) in 878 ms on 10.0.0.133 (executor driver) (11/64) +26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.14.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.1.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:46 INFO Executor: Finished task 9.0 in stage 13.0 (TID 279). 2208 bytes result sent to driver +26/04/01 08:38:46 INFO TaskSetManager: Starting task 19.0 in stage 13.0 (TID 289) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:46 INFO Executor: Running task 19.0 in stage 13.0 (TID 289) +26/04/01 08:38:46 INFO TaskSetManager: Finished task 9.0 in stage 13.0 (TID 279) in 892 ms on 10.0.0.133 (executor driver) (12/64) +26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.15.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:46 INFO Executor: Finished task 12.0 in stage 13.0 (TID 282). 2208 bytes result sent to driver +26/04/01 08:38:46 INFO TaskSetManager: Starting task 20.0 in stage 13.0 (TID 290) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:46 INFO TaskSetManager: Finished task 12.0 in stage 13.0 (TID 282) in 796 ms on 10.0.0.133 (executor driver) (13/64) +26/04/01 08:38:46 INFO Executor: Running task 20.0 in stage 13.0 (TID 290) +26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.23.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:46 INFO Executor: Finished task 13.0 in stage 13.0 (TID 283). 2208 bytes result sent to driver +26/04/01 08:38:46 INFO TaskSetManager: Starting task 21.0 in stage 13.0 (TID 291) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:46 INFO TaskSetManager: Finished task 13.0 in stage 13.0 (TID 283) in 807 ms on 10.0.0.133 (executor driver) (14/64) +26/04/01 08:38:46 INFO Executor: Running task 21.0 in stage 13.0 (TID 291) +26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.8.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:46 INFO Executor: Finished task 14.0 in stage 13.0 (TID 284). 2208 bytes result sent to driver +26/04/01 08:38:46 INFO TaskSetManager: Starting task 22.0 in stage 13.0 (TID 292) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:46 INFO TaskSetManager: Finished task 14.0 in stage 13.0 (TID 284) in 791 ms on 10.0.0.133 (executor driver) (15/64) +26/04/01 08:38:46 INFO Executor: Running task 22.0 in stage 13.0 (TID 292) +26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.28.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:46 INFO Executor: Finished task 15.0 in stage 13.0 (TID 285). 2208 bytes result sent to driver +26/04/01 08:38:46 INFO TaskSetManager: Starting task 23.0 in stage 13.0 (TID 293) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:46 INFO TaskSetManager: Finished task 15.0 in stage 13.0 (TID 285) in 763 ms on 10.0.0.133 (executor driver) (16/64) +26/04/01 08:38:46 INFO Executor: Running task 23.0 in stage 13.0 (TID 293) +26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.17.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:47 INFO Executor: Finished task 16.0 in stage 13.0 (TID 286). 2251 bytes result sent to driver +26/04/01 08:38:47 INFO TaskSetManager: Starting task 24.0 in stage 13.0 (TID 294) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:47 INFO TaskSetManager: Finished task 16.0 in stage 13.0 (TID 286) in 634 ms on 10.0.0.133 (executor driver) (17/64) +26/04/01 08:38:47 INFO Executor: Running task 24.0 in stage 13.0 (TID 294) +26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.3.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:47 INFO Executor: Finished task 18.0 in stage 13.0 (TID 288). 2251 bytes result sent to driver +26/04/01 08:38:47 INFO TaskSetManager: Starting task 25.0 in stage 13.0 (TID 295) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:47 INFO TaskSetManager: Finished task 18.0 in stage 13.0 (TID 288) in 873 ms on 10.0.0.133 (executor driver) (18/64) +26/04/01 08:38:47 INFO Executor: Running task 25.0 in stage 13.0 (TID 295) +26/04/01 08:38:47 INFO Executor: Finished task 19.0 in stage 13.0 (TID 289). 2251 bytes result sent to driver +26/04/01 08:38:47 INFO TaskSetManager: Starting task 26.0 in stage 13.0 (TID 296) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:47 INFO TaskSetManager: Finished task 19.0 in stage 13.0 (TID 289) in 872 ms on 10.0.0.133 (executor driver) (19/64) +26/04/01 08:38:47 INFO Executor: Running task 26.0 in stage 13.0 (TID 296) +26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.31.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.21.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:47 INFO Executor: Finished task 17.0 in stage 13.0 (TID 287). 2251 bytes result sent to driver +26/04/01 08:38:47 INFO TaskSetManager: Starting task 27.0 in stage 13.0 (TID 297) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:47 INFO Executor: Running task 27.0 in stage 13.0 (TID 297) +26/04/01 08:38:47 INFO TaskSetManager: Finished task 17.0 in stage 13.0 (TID 287) in 886 ms on 10.0.0.133 (executor driver) (20/64) +26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.20.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:47 INFO Executor: Finished task 20.0 in stage 13.0 (TID 290). 2294 bytes result sent to driver +26/04/01 08:38:47 INFO TaskSetManager: Starting task 28.0 in stage 13.0 (TID 298) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:47 INFO TaskSetManager: Finished task 20.0 in stage 13.0 (TID 290) in 913 ms on 10.0.0.133 (executor driver) (21/64) +26/04/01 08:38:47 INFO Executor: Running task 28.0 in stage 13.0 (TID 298) +26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.30.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:47 INFO Executor: Finished task 23.0 in stage 13.0 (TID 293). 2251 bytes result sent to driver +26/04/01 08:38:47 INFO TaskSetManager: Starting task 29.0 in stage 13.0 (TID 299) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:47 INFO TaskSetManager: Finished task 23.0 in stage 13.0 (TID 293) in 834 ms on 10.0.0.133 (executor driver) (22/64) +26/04/01 08:38:47 INFO Executor: Running task 29.0 in stage 13.0 (TID 299) +26/04/01 08:38:47 INFO Executor: Finished task 21.0 in stage 13.0 (TID 291). 2251 bytes result sent to driver +26/04/01 08:38:47 INFO TaskSetManager: Starting task 30.0 in stage 13.0 (TID 300) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:47 INFO Executor: Running task 30.0 in stage 13.0 (TID 300) +26/04/01 08:38:47 INFO TaskSetManager: Finished task 21.0 in stage 13.0 (TID 291) in 877 ms on 10.0.0.133 (executor driver) (23/64) +26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.29.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.16.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:47 INFO Executor: Finished task 22.0 in stage 13.0 (TID 292). 2251 bytes result sent to driver +26/04/01 08:38:47 INFO TaskSetManager: Starting task 31.0 in stage 13.0 (TID 301) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:47 INFO TaskSetManager: Finished task 22.0 in stage 13.0 (TID 292) in 870 ms on 10.0.0.133 (executor driver) (24/64) +26/04/01 08:38:47 INFO Executor: Running task 31.0 in stage 13.0 (TID 301) +26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.2.parquet, range: 0-134217728, partition values: [empty row] +26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:47 INFO Executor: Finished task 24.0 in stage 13.0 (TID 294). 2251 bytes result sent to driver +26/04/01 08:38:47 INFO TaskSetManager: Starting task 32.0 in stage 13.0 (TID 302) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:47 INFO TaskSetManager: Finished task 24.0 in stage 13.0 (TID 294) in 746 ms on 10.0.0.133 (executor driver) (25/64) +26/04/01 08:38:47 INFO Executor: Running task 32.0 in stage 13.0 (TID 302) +26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.11.parquet, range: 134217728-233581632, partition values: [empty row] +26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:47 INFO Executor: Finished task 26.0 in stage 13.0 (TID 296). 2251 bytes result sent to driver +26/04/01 08:38:47 INFO TaskSetManager: Starting task 33.0 in stage 13.0 (TID 303) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:47 INFO TaskSetManager: Finished task 26.0 in stage 13.0 (TID 296) in 645 ms on 10.0.0.133 (executor driver) (26/64) +26/04/01 08:38:47 INFO Executor: Running task 33.0 in stage 13.0 (TID 303) +26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.23.parquet, range: 134217728-233580600, partition values: [empty row] +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:48 INFO Executor: Finished task 25.0 in stage 13.0 (TID 295). 2251 bytes result sent to driver +26/04/01 08:38:48 INFO TaskSetManager: Starting task 34.0 in stage 13.0 (TID 304) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:48 INFO TaskSetManager: Finished task 25.0 in stage 13.0 (TID 295) in 682 ms on 10.0.0.133 (executor driver) (27/64) +26/04/01 08:38:48 INFO Executor: Running task 34.0 in stage 13.0 (TID 304) +26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.30.parquet, range: 134217728-233577812, partition values: [empty row] +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:48 INFO Executor: Finished task 27.0 in stage 13.0 (TID 297). 2251 bytes result sent to driver +26/04/01 08:38:48 INFO TaskSetManager: Starting task 35.0 in stage 13.0 (TID 305) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:48 INFO TaskSetManager: Finished task 27.0 in stage 13.0 (TID 297) in 696 ms on 10.0.0.133 (executor driver) (28/64) +26/04/01 08:38:48 INFO Executor: Running task 35.0 in stage 13.0 (TID 305) +26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.7.parquet, range: 134217728-233566955, partition values: [empty row] +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:48 INFO Executor: Finished task 28.0 in stage 13.0 (TID 298). 2251 bytes result sent to driver +26/04/01 08:38:48 INFO TaskSetManager: Starting task 36.0 in stage 13.0 (TID 306) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:48 INFO TaskSetManager: Finished task 28.0 in stage 13.0 (TID 298) in 699 ms on 10.0.0.133 (executor driver) (29/64) +26/04/01 08:38:48 INFO Executor: Running task 36.0 in stage 13.0 (TID 306) +26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.12.parquet, range: 134217728-233565571, partition values: [empty row] +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:48 INFO Executor: Finished task 30.0 in stage 13.0 (TID 300). 2251 bytes result sent to driver +26/04/01 08:38:48 INFO TaskSetManager: Starting task 37.0 in stage 13.0 (TID 307) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:48 INFO TaskSetManager: Finished task 30.0 in stage 13.0 (TID 300) in 759 ms on 10.0.0.133 (executor driver) (30/64) +26/04/01 08:38:48 INFO Executor: Running task 37.0 in stage 13.0 (TID 307) +26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.16.parquet, range: 134217728-233564116, partition values: [empty row] +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:48 INFO Executor: Finished task 31.0 in stage 13.0 (TID 301). 2251 bytes result sent to driver +26/04/01 08:38:48 INFO TaskSetManager: Starting task 38.0 in stage 13.0 (TID 308) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:48 INFO Executor: Running task 38.0 in stage 13.0 (TID 308) +26/04/01 08:38:48 INFO TaskSetManager: Finished task 31.0 in stage 13.0 (TID 301) in 771 ms on 10.0.0.133 (executor driver) (31/64) +26/04/01 08:38:48 INFO Executor: Finished task 29.0 in stage 13.0 (TID 299). 2251 bytes result sent to driver +26/04/01 08:38:48 INFO TaskSetManager: Starting task 39.0 in stage 13.0 (TID 309) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:48 INFO TaskSetManager: Finished task 29.0 in stage 13.0 (TID 299) in 773 ms on 10.0.0.133 (executor driver) (32/64) +26/04/01 08:38:48 INFO Executor: Running task 39.0 in stage 13.0 (TID 309) +26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.32.parquet, range: 134217728-233563384, partition values: [empty row] +26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.14.parquet, range: 134217728-233562716, partition values: [empty row] +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:48 INFO Executor: Finished task 32.0 in stage 13.0 (TID 302). 2208 bytes result sent to driver +26/04/01 08:38:48 INFO TaskSetManager: Starting task 40.0 in stage 13.0 (TID 310) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:48 INFO TaskSetManager: Finished task 32.0 in stage 13.0 (TID 302) in 517 ms on 10.0.0.133 (executor driver) (33/64) +26/04/01 08:38:48 INFO Executor: Running task 40.0 in stage 13.0 (TID 310) +26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.5.parquet, range: 134217728-233556822, partition values: [empty row] +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:48 INFO Executor: Finished task 33.0 in stage 13.0 (TID 303). 2208 bytes result sent to driver +26/04/01 08:38:48 INFO TaskSetManager: Starting task 41.0 in stage 13.0 (TID 311) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:48 INFO TaskSetManager: Finished task 33.0 in stage 13.0 (TID 303) in 525 ms on 10.0.0.133 (executor driver) (34/64) +26/04/01 08:38:48 INFO Executor: Running task 41.0 in stage 13.0 (TID 311) +26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.27.parquet, range: 134217728-233556807, partition values: [empty row] +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:48 INFO Executor: Finished task 34.0 in stage 13.0 (TID 304). 2208 bytes result sent to driver +26/04/01 08:38:48 INFO TaskSetManager: Starting task 42.0 in stage 13.0 (TID 312) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:48 INFO TaskSetManager: Finished task 34.0 in stage 13.0 (TID 304) in 537 ms on 10.0.0.133 (executor driver) (35/64) +26/04/01 08:38:48 INFO Executor: Running task 42.0 in stage 13.0 (TID 312) +26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.19.parquet, range: 134217728-233556337, partition values: [empty row] +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:48 INFO Executor: Finished task 35.0 in stage 13.0 (TID 305). 2208 bytes result sent to driver +26/04/01 08:38:48 INFO TaskSetManager: Starting task 43.0 in stage 13.0 (TID 313) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:48 INFO TaskSetManager: Finished task 35.0 in stage 13.0 (TID 305) in 520 ms on 10.0.0.133 (executor driver) (36/64) +26/04/01 08:38:48 INFO Executor: Running task 43.0 in stage 13.0 (TID 313) +26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.1.parquet, range: 134217728-233553544, partition values: [empty row] +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:48 INFO Executor: Finished task 36.0 in stage 13.0 (TID 306). 2208 bytes result sent to driver +26/04/01 08:38:48 INFO TaskSetManager: Starting task 44.0 in stage 13.0 (TID 314) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:48 INFO TaskSetManager: Finished task 36.0 in stage 13.0 (TID 306) in 533 ms on 10.0.0.133 (executor driver) (37/64) +26/04/01 08:38:48 INFO Executor: Running task 44.0 in stage 13.0 (TID 314) +26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.6.parquet, range: 134217728-233549987, partition values: [empty row] +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:48 INFO Executor: Finished task 37.0 in stage 13.0 (TID 307). 2251 bytes result sent to driver +26/04/01 08:38:48 INFO TaskSetManager: Starting task 45.0 in stage 13.0 (TID 315) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:48 INFO TaskSetManager: Finished task 37.0 in stage 13.0 (TID 307) in 536 ms on 10.0.0.133 (executor driver) (38/64) +26/04/01 08:38:48 INFO Executor: Running task 45.0 in stage 13.0 (TID 315) +26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.21.parquet, range: 134217728-233548593, partition values: [empty row] +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:48 INFO Executor: Finished task 39.0 in stage 13.0 (TID 309). 2251 bytes result sent to driver +26/04/01 08:38:48 INFO TaskSetManager: Starting task 46.0 in stage 13.0 (TID 316) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:48 INFO Executor: Running task 46.0 in stage 13.0 (TID 316) +26/04/01 08:38:48 INFO TaskSetManager: Finished task 39.0 in stage 13.0 (TID 309) in 563 ms on 10.0.0.133 (executor driver) (39/64) +26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.28.parquet, range: 134217728-233545450, partition values: [empty row] +26/04/01 08:38:48 INFO Executor: Finished task 40.0 in stage 13.0 (TID 310). 2251 bytes result sent to driver +26/04/01 08:38:48 INFO TaskSetManager: Starting task 47.0 in stage 13.0 (TID 317) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:48 INFO TaskSetManager: Finished task 40.0 in stage 13.0 (TID 310) in 549 ms on 10.0.0.133 (executor driver) (40/64) +26/04/01 08:38:48 INFO Executor: Running task 47.0 in stage 13.0 (TID 317) +26/04/01 08:38:48 INFO Executor: Finished task 38.0 in stage 13.0 (TID 308). 2251 bytes result sent to driver +26/04/01 08:38:48 INFO TaskSetManager: Starting task 48.0 in stage 13.0 (TID 318) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:48 INFO TaskSetManager: Finished task 38.0 in stage 13.0 (TID 308) in 565 ms on 10.0.0.133 (executor driver) (41/64) +26/04/01 08:38:48 INFO Executor: Running task 48.0 in stage 13.0 (TID 318) +26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.4.parquet, range: 134217728-233545407, partition values: [empty row] +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.29.parquet, range: 134217728-233542751, partition values: [empty row] +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO Executor: Finished task 41.0 in stage 13.0 (TID 311). 2251 bytes result sent to driver +26/04/01 08:38:49 INFO TaskSetManager: Starting task 49.0 in stage 13.0 (TID 319) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 41.0 in stage 13.0 (TID 311) in 482 ms on 10.0.0.133 (executor driver) (42/64) +26/04/01 08:38:49 INFO Executor: Running task 49.0 in stage 13.0 (TID 319) +26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.24.parquet, range: 134217728-233541467, partition values: [empty row] +26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO Executor: Finished task 43.0 in stage 13.0 (TID 313). 2251 bytes result sent to driver +26/04/01 08:38:49 INFO Executor: Finished task 42.0 in stage 13.0 (TID 312). 2251 bytes result sent to driver +26/04/01 08:38:49 INFO TaskSetManager: Starting task 50.0 in stage 13.0 (TID 320) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:49 INFO TaskSetManager: Starting task 51.0 in stage 13.0 (TID 321) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:49 INFO Executor: Running task 50.0 in stage 13.0 (TID 320) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 43.0 in stage 13.0 (TID 313) in 477 ms on 10.0.0.133 (executor driver) (43/64) +26/04/01 08:38:49 INFO Executor: Running task 51.0 in stage 13.0 (TID 321) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 42.0 in stage 13.0 (TID 312) in 488 ms on 10.0.0.133 (executor driver) (44/64) +26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.20.parquet, range: 134217728-233539473, partition values: [empty row] +26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.26.parquet, range: 134217728-233538804, partition values: [empty row] +26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO Executor: Finished task 44.0 in stage 13.0 (TID 314). 2251 bytes result sent to driver +26/04/01 08:38:49 INFO TaskSetManager: Starting task 52.0 in stage 13.0 (TID 322) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:49 INFO Executor: Running task 52.0 in stage 13.0 (TID 322) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 44.0 in stage 13.0 (TID 314) in 457 ms on 10.0.0.133 (executor driver) (45/64) +26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.10.parquet, range: 134217728-233538598, partition values: [empty row] +26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO Executor: Finished task 45.0 in stage 13.0 (TID 315). 2208 bytes result sent to driver +26/04/01 08:38:49 INFO TaskSetManager: Starting task 53.0 in stage 13.0 (TID 323) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 45.0 in stage 13.0 (TID 315) in 448 ms on 10.0.0.133 (executor driver) (46/64) +26/04/01 08:38:49 INFO Executor: Running task 53.0 in stage 13.0 (TID 323) +26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.13.parquet, range: 134217728-233537986, partition values: [empty row] +26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO Executor: Finished task 47.0 in stage 13.0 (TID 317). 2208 bytes result sent to driver +26/04/01 08:38:49 INFO TaskSetManager: Starting task 54.0 in stage 13.0 (TID 324) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 47.0 in stage 13.0 (TID 317) in 505 ms on 10.0.0.133 (executor driver) (47/64) +26/04/01 08:38:49 INFO Executor: Running task 54.0 in stage 13.0 (TID 324) +26/04/01 08:38:49 INFO Executor: Finished task 46.0 in stage 13.0 (TID 316). 2208 bytes result sent to driver +26/04/01 08:38:49 INFO TaskSetManager: Starting task 55.0 in stage 13.0 (TID 325) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 46.0 in stage 13.0 (TID 316) in 508 ms on 10.0.0.133 (executor driver) (48/64) +26/04/01 08:38:49 INFO Executor: Running task 55.0 in stage 13.0 (TID 325) +26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.8.parquet, range: 134217728-233536463, partition values: [empty row] +26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.22.parquet, range: 134217728-233536356, partition values: [empty row] +26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO Executor: Finished task 48.0 in stage 13.0 (TID 318). 2208 bytes result sent to driver +26/04/01 08:38:49 INFO TaskSetManager: Starting task 56.0 in stage 13.0 (TID 326) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:49 INFO Executor: Running task 56.0 in stage 13.0 (TID 326) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 48.0 in stage 13.0 (TID 318) in 516 ms on 10.0.0.133 (executor driver) (49/64) +26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.9.parquet, range: 134217728-233535871, partition values: [empty row] +26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO Executor: Finished task 49.0 in stage 13.0 (TID 319). 2208 bytes result sent to driver +26/04/01 08:38:49 INFO TaskSetManager: Starting task 57.0 in stage 13.0 (TID 327) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:49 INFO Executor: Running task 57.0 in stage 13.0 (TID 327) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 49.0 in stage 13.0 (TID 319) in 499 ms on 10.0.0.133 (executor driver) (50/64) +26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.31.parquet, range: 134217728-233535690, partition values: [empty row] +26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO Executor: Finished task 50.0 in stage 13.0 (TID 320). 2208 bytes result sent to driver +26/04/01 08:38:49 INFO Executor: Finished task 51.0 in stage 13.0 (TID 321). 2208 bytes result sent to driver +26/04/01 08:38:49 INFO TaskSetManager: Starting task 58.0 in stage 13.0 (TID 328) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 50.0 in stage 13.0 (TID 320) in 516 ms on 10.0.0.133 (executor driver) (51/64) +26/04/01 08:38:49 INFO Executor: Running task 58.0 in stage 13.0 (TID 328) +26/04/01 08:38:49 INFO TaskSetManager: Starting task 59.0 in stage 13.0 (TID 329) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:49 INFO Executor: Running task 59.0 in stage 13.0 (TID 329) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 51.0 in stage 13.0 (TID 321) in 516 ms on 10.0.0.133 (executor driver) (52/64) +26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.17.parquet, range: 134217728-233534488, partition values: [empty row] +26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.18.parquet, range: 134217728-233534883, partition values: [empty row] +26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO Executor: Finished task 52.0 in stage 13.0 (TID 322). 2251 bytes result sent to driver +26/04/01 08:38:49 INFO TaskSetManager: Starting task 60.0 in stage 13.0 (TID 330) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:49 INFO Executor: Running task 60.0 in stage 13.0 (TID 330) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 52.0 in stage 13.0 (TID 322) in 500 ms on 10.0.0.133 (executor driver) (53/64) +26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.15.parquet, range: 134217728-233533044, partition values: [empty row] +26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO Executor: Finished task 53.0 in stage 13.0 (TID 323). 2251 bytes result sent to driver +26/04/01 08:38:49 INFO TaskSetManager: Starting task 61.0 in stage 13.0 (TID 331) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9598 bytes) +26/04/01 08:38:49 INFO Executor: Running task 61.0 in stage 13.0 (TID 331) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 53.0 in stage 13.0 (TID 323) in 497 ms on 10.0.0.133 (executor driver) (54/64) +26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.25.parquet, range: 134217728-233532173, partition values: [empty row] +26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO Executor: Finished task 54.0 in stage 13.0 (TID 324). 2251 bytes result sent to driver +26/04/01 08:38:49 INFO TaskSetManager: Starting task 62.0 in stage 13.0 (TID 332) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 54.0 in stage 13.0 (TID 324) in 579 ms on 10.0.0.133 (executor driver) (55/64) +26/04/01 08:38:49 INFO Executor: Running task 62.0 in stage 13.0 (TID 332) +26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.3.parquet, range: 134217728-233524702, partition values: [empty row] +26/04/01 08:38:49 INFO Executor: Finished task 55.0 in stage 13.0 (TID 325). 2251 bytes result sent to driver +26/04/01 08:38:49 INFO TaskSetManager: Starting task 63.0 in stage 13.0 (TID 333) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9597 bytes) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 55.0 in stage 13.0 (TID 325) in 580 ms on 10.0.0.133 (executor driver) (56/64) +26/04/01 08:38:49 INFO Executor: Running task 63.0 in stage 13.0 (TID 333) +26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.2.parquet, range: 134217728-233511585, partition values: [empty row] +26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) +26/04/01 08:38:49 INFO Executor: Finished task 56.0 in stage 13.0 (TID 326). 2251 bytes result sent to driver +26/04/01 08:38:49 INFO TaskSetManager: Starting task 0.0 in stage 16.0 (TID 334) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:49 INFO Executor: Running task 0.0 in stage 16.0 (TID 334) +26/04/01 08:38:49 INFO TaskSetManager: Finished task 56.0 in stage 13.0 (TID 326) in 600 ms on 10.0.0.133 (executor driver) (57/64) +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 8 ms +26/04/01 08:38:50 INFO CodeGenerator: Code generated in 3.042541 ms +26/04/01 08:38:50 INFO CodeGenerator: Code generated in 3.434042 ms +26/04/01 08:38:50 INFO CodeGenerator: Code generated in 1.618833 ms +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:50 INFO Executor: Finished task 57.0 in stage 13.0 (TID 327). 2251 bytes result sent to driver +26/04/01 08:38:50 INFO TaskSetManager: Starting task 1.0 in stage 16.0 (TID 335) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:50 INFO TaskSetManager: Finished task 57.0 in stage 13.0 (TID 327) in 577 ms on 10.0.0.133 (executor driver) (58/64) +26/04/01 08:38:50 INFO Executor: Running task 1.0 in stage 16.0 (TID 335) +26/04/01 08:38:50 INFO Executor: Finished task 58.0 in stage 13.0 (TID 328). 2251 bytes result sent to driver +26/04/01 08:38:50 INFO TaskSetManager: Starting task 2.0 in stage 16.0 (TID 336) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:50 INFO Executor: Running task 2.0 in stage 16.0 (TID 336) +26/04/01 08:38:50 INFO TaskSetManager: Finished task 58.0 in stage 13.0 (TID 328) in 507 ms on 10.0.0.133 (executor driver) (59/64) +26/04/01 08:38:50 INFO Executor: Finished task 59.0 in stage 13.0 (TID 329). 2251 bytes result sent to driver +26/04/01 08:38:50 INFO TaskSetManager: Starting task 3.0 in stage 16.0 (TID 337) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:50 INFO TaskSetManager: Finished task 59.0 in stage 13.0 (TID 329) in 508 ms on 10.0.0.133 (executor driver) (60/64) +26/04/01 08:38:50 INFO Executor: Running task 3.0 in stage 16.0 (TID 337) +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:50 INFO CodeGenerator: Code generated in 3.219 ms +26/04/01 08:38:50 INFO CodeGenerator: Code generated in 1.6445 ms +26/04/01 08:38:50 INFO CodeGenerator: Code generated in 1.843458 ms +26/04/01 08:38:50 INFO CodeGenerator: Code generated in 4.8145 ms +26/04/01 08:38:50 INFO CodeGenerator: Code generated in 2.479792 ms +26/04/01 08:38:50 INFO Executor: Finished task 60.0 in stage 13.0 (TID 330). 2251 bytes result sent to driver +26/04/01 08:38:50 INFO TaskSetManager: Starting task 4.0 in stage 16.0 (TID 338) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:50 INFO Executor: Running task 4.0 in stage 16.0 (TID 338) +26/04/01 08:38:50 INFO TaskSetManager: Finished task 60.0 in stage 13.0 (TID 330) in 515 ms on 10.0.0.133 (executor driver) (61/64) +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:50 INFO Executor: Finished task 61.0 in stage 13.0 (TID 331). 2208 bytes result sent to driver +26/04/01 08:38:50 INFO TaskSetManager: Starting task 5.0 in stage 16.0 (TID 339) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:50 INFO Executor: Running task 5.0 in stage 16.0 (TID 339) +26/04/01 08:38:50 INFO TaskSetManager: Finished task 61.0 in stage 13.0 (TID 331) in 465 ms on 10.0.0.133 (executor driver) (62/64) +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:50 INFO Executor: Finished task 62.0 in stage 13.0 (TID 332). 2208 bytes result sent to driver +26/04/01 08:38:50 INFO TaskSetManager: Starting task 6.0 in stage 16.0 (TID 340) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:50 INFO Executor: Running task 6.0 in stage 16.0 (TID 340) +26/04/01 08:38:50 INFO TaskSetManager: Finished task 62.0 in stage 13.0 (TID 332) in 462 ms on 10.0.0.133 (executor driver) (63/64) +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:50 INFO Executor: Finished task 63.0 in stage 13.0 (TID 333). 2208 bytes result sent to driver +26/04/01 08:38:50 INFO TaskSetManager: Starting task 7.0 in stage 16.0 (TID 341) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:50 INFO TaskSetManager: Finished task 63.0 in stage 13.0 (TID 333) in 474 ms on 10.0.0.133 (executor driver) (64/64) +26/04/01 08:38:50 INFO Executor: Running task 7.0 in stage 16.0 (TID 341) +26/04/01 08:38:50 INFO TaskSchedulerImpl: Removed TaskSet 13.0, whose tasks have all completed, from pool +26/04/01 08:38:50 INFO DAGScheduler: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 43.175 s +26/04/01 08:38:50 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:38:50 INFO DAGScheduler: running: Set(ShuffleMapStage 16) +26/04/01 08:38:50 INFO DAGScheduler: waiting: Set() +26/04/01 08:38:50 INFO DAGScheduler: failed: Set() +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO Executor: Finished task 3.0 in stage 16.0 (TID 337). 7266 bytes result sent to driver +26/04/01 08:38:51 INFO TaskSetManager: Starting task 8.0 in stage 16.0 (TID 342) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:51 INFO TaskSetManager: Finished task 3.0 in stage 16.0 (TID 337) in 1102 ms on 10.0.0.133 (executor driver) (1/200) +26/04/01 08:38:51 INFO Executor: Running task 8.0 in stage 16.0 (TID 342) +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO Executor: Finished task 5.0 in stage 16.0 (TID 339). 7266 bytes result sent to driver +26/04/01 08:38:51 INFO TaskSetManager: Starting task 9.0 in stage 16.0 (TID 343) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:51 INFO Executor: Running task 9.0 in stage 16.0 (TID 343) +26/04/01 08:38:51 INFO TaskSetManager: Finished task 5.0 in stage 16.0 (TID 339) in 1055 ms on 10.0.0.133 (executor driver) (2/200) +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO Executor: Finished task 4.0 in stage 16.0 (TID 338). 7266 bytes result sent to driver +26/04/01 08:38:51 INFO TaskSetManager: Starting task 10.0 in stage 16.0 (TID 344) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:51 INFO TaskSetManager: Finished task 4.0 in stage 16.0 (TID 338) in 1101 ms on 10.0.0.133 (executor driver) (3/200) +26/04/01 08:38:51 INFO Executor: Running task 10.0 in stage 16.0 (TID 344) +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO Executor: Finished task 2.0 in stage 16.0 (TID 336). 7266 bytes result sent to driver +26/04/01 08:38:51 INFO TaskSetManager: Starting task 11.0 in stage 16.0 (TID 345) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:51 INFO TaskSetManager: Finished task 2.0 in stage 16.0 (TID 336) in 1277 ms on 10.0.0.133 (executor driver) (4/200) +26/04/01 08:38:51 INFO Executor: Running task 11.0 in stage 16.0 (TID 345) +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (28.1 KiB) non-empty blocks including 8 (28.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO Executor: Finished task 6.0 in stage 16.0 (TID 340). 7266 bytes result sent to driver +26/04/01 08:38:51 INFO TaskSetManager: Starting task 12.0 in stage 16.0 (TID 346) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:51 INFO TaskSetManager: Finished task 6.0 in stage 16.0 (TID 340) in 948 ms on 10.0.0.133 (executor driver) (5/200) +26/04/01 08:38:51 INFO Executor: Running task 12.0 in stage 16.0 (TID 346) +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO Executor: Finished task 1.0 in stage 16.0 (TID 335). 7266 bytes result sent to driver +26/04/01 08:38:51 INFO TaskSetManager: Starting task 13.0 in stage 16.0 (TID 347) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:51 INFO Executor: Running task 13.0 in stage 16.0 (TID 347) +26/04/01 08:38:51 INFO TaskSetManager: Finished task 1.0 in stage 16.0 (TID 335) in 1297 ms on 10.0.0.133 (executor driver) (6/200) +26/04/01 08:38:51 INFO Executor: Finished task 0.0 in stage 16.0 (TID 334). 7266 bytes result sent to driver +26/04/01 08:38:51 INFO TaskSetManager: Starting task 14.0 in stage 16.0 (TID 348) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:51 INFO Executor: Running task 14.0 in stage 16.0 (TID 348) +26/04/01 08:38:51 INFO TaskSetManager: Finished task 0.0 in stage 16.0 (TID 334) in 1382 ms on 10.0.0.133 (executor driver) (7/200) +26/04/01 08:38:51 INFO Executor: Finished task 7.0 in stage 16.0 (TID 341). 7223 bytes result sent to driver +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO TaskSetManager: Starting task 15.0 in stage 16.0 (TID 349) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:51 INFO TaskSetManager: Finished task 7.0 in stage 16.0 (TID 341) in 938 ms on 10.0.0.133 (executor driver) (8/200) +26/04/01 08:38:51 INFO Executor: Running task 15.0 in stage 16.0 (TID 349) +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO Executor: Finished task 8.0 in stage 16.0 (TID 342). 7309 bytes result sent to driver +26/04/01 08:38:51 INFO TaskSetManager: Starting task 16.0 in stage 16.0 (TID 350) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:51 INFO TaskSetManager: Finished task 8.0 in stage 16.0 (TID 342) in 722 ms on 10.0.0.133 (executor driver) (9/200) +26/04/01 08:38:51 INFO Executor: Running task 16.0 in stage 16.0 (TID 350) +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO Executor: Finished task 10.0 in stage 16.0 (TID 344). 7309 bytes result sent to driver +26/04/01 08:38:52 INFO TaskSetManager: Starting task 17.0 in stage 16.0 (TID 351) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:52 INFO TaskSetManager: Finished task 10.0 in stage 16.0 (TID 344) in 795 ms on 10.0.0.133 (executor driver) (10/200) +26/04/01 08:38:52 INFO Executor: Running task 17.0 in stage 16.0 (TID 351) +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO Executor: Finished task 9.0 in stage 16.0 (TID 343). 7309 bytes result sent to driver +26/04/01 08:38:52 INFO TaskSetManager: Starting task 18.0 in stage 16.0 (TID 352) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:52 INFO TaskSetManager: Finished task 9.0 in stage 16.0 (TID 343) in 809 ms on 10.0.0.133 (executor driver) (11/200) +26/04/01 08:38:52 INFO Executor: Running task 18.0 in stage 16.0 (TID 352) +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO Executor: Finished task 11.0 in stage 16.0 (TID 345). 7309 bytes result sent to driver +26/04/01 08:38:52 INFO TaskSetManager: Starting task 19.0 in stage 16.0 (TID 353) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:52 INFO Executor: Running task 19.0 in stage 16.0 (TID 353) +26/04/01 08:38:52 INFO TaskSetManager: Finished task 11.0 in stage 16.0 (TID 345) in 793 ms on 10.0.0.133 (executor driver) (12/200) +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO Executor: Finished task 12.0 in stage 16.0 (TID 346). 7309 bytes result sent to driver +26/04/01 08:38:52 INFO TaskSetManager: Starting task 20.0 in stage 16.0 (TID 354) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:52 INFO TaskSetManager: Finished task 12.0 in stage 16.0 (TID 346) in 788 ms on 10.0.0.133 (executor driver) (13/200) +26/04/01 08:38:52 INFO Executor: Running task 20.0 in stage 16.0 (TID 354) +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO Executor: Finished task 13.0 in stage 16.0 (TID 347). 7266 bytes result sent to driver +26/04/01 08:38:52 INFO TaskSetManager: Starting task 21.0 in stage 16.0 (TID 355) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:52 INFO TaskSetManager: Finished task 13.0 in stage 16.0 (TID 347) in 792 ms on 10.0.0.133 (executor driver) (14/200) +26/04/01 08:38:52 INFO Executor: Running task 21.0 in stage 16.0 (TID 355) +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO Executor: Finished task 14.0 in stage 16.0 (TID 348). 7266 bytes result sent to driver +26/04/01 08:38:52 INFO TaskSetManager: Starting task 22.0 in stage 16.0 (TID 356) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:52 INFO TaskSetManager: Finished task 14.0 in stage 16.0 (TID 348) in 792 ms on 10.0.0.133 (executor driver) (15/200) +26/04/01 08:38:52 INFO Executor: Running task 22.0 in stage 16.0 (TID 356) +26/04/01 08:38:52 INFO Executor: Finished task 15.0 in stage 16.0 (TID 349). 7309 bytes result sent to driver +26/04/01 08:38:52 INFO TaskSetManager: Starting task 23.0 in stage 16.0 (TID 357) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:52 INFO Executor: Running task 23.0 in stage 16.0 (TID 357) +26/04/01 08:38:52 INFO TaskSetManager: Finished task 15.0 in stage 16.0 (TID 349) in 792 ms on 10.0.0.133 (executor driver) (16/200) +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO Executor: Finished task 16.0 in stage 16.0 (TID 350). 7266 bytes result sent to driver +26/04/01 08:38:52 INFO TaskSetManager: Starting task 24.0 in stage 16.0 (TID 358) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:52 INFO Executor: Running task 24.0 in stage 16.0 (TID 358) +26/04/01 08:38:52 INFO TaskSetManager: Finished task 16.0 in stage 16.0 (TID 350) in 728 ms on 10.0.0.133 (executor driver) (17/200) +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO Executor: Finished task 17.0 in stage 16.0 (TID 351). 7266 bytes result sent to driver +26/04/01 08:38:52 INFO TaskSetManager: Starting task 25.0 in stage 16.0 (TID 359) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:52 INFO TaskSetManager: Finished task 17.0 in stage 16.0 (TID 351) in 754 ms on 10.0.0.133 (executor driver) (18/200) +26/04/01 08:38:52 INFO Executor: Running task 25.0 in stage 16.0 (TID 359) +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO Executor: Finished task 18.0 in stage 16.0 (TID 352). 7223 bytes result sent to driver +26/04/01 08:38:52 INFO TaskSetManager: Starting task 26.0 in stage 16.0 (TID 360) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:52 INFO Executor: Running task 26.0 in stage 16.0 (TID 360) +26/04/01 08:38:52 INFO TaskSetManager: Finished task 18.0 in stage 16.0 (TID 352) in 766 ms on 10.0.0.133 (executor driver) (19/200) +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO Executor: Finished task 19.0 in stage 16.0 (TID 353). 7223 bytes result sent to driver +26/04/01 08:38:52 INFO TaskSetManager: Starting task 27.0 in stage 16.0 (TID 361) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:52 INFO TaskSetManager: Finished task 19.0 in stage 16.0 (TID 353) in 791 ms on 10.0.0.133 (executor driver) (20/200) +26/04/01 08:38:52 INFO Executor: Running task 27.0 in stage 16.0 (TID 361) +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO Executor: Finished task 21.0 in stage 16.0 (TID 355). 7223 bytes result sent to driver +26/04/01 08:38:52 INFO TaskSetManager: Starting task 28.0 in stage 16.0 (TID 362) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:52 INFO Executor: Running task 28.0 in stage 16.0 (TID 362) +26/04/01 08:38:52 INFO TaskSetManager: Finished task 21.0 in stage 16.0 (TID 355) in 790 ms on 10.0.0.133 (executor driver) (21/200) +26/04/01 08:38:52 INFO Executor: Finished task 20.0 in stage 16.0 (TID 354). 7223 bytes result sent to driver +26/04/01 08:38:52 INFO TaskSetManager: Starting task 29.0 in stage 16.0 (TID 363) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:52 INFO Executor: Running task 29.0 in stage 16.0 (TID 363) +26/04/01 08:38:52 INFO TaskSetManager: Finished task 20.0 in stage 16.0 (TID 354) in 797 ms on 10.0.0.133 (executor driver) (22/200) +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.2 KiB) non-empty blocks including 8 (30.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO Executor: Finished task 22.0 in stage 16.0 (TID 356). 7223 bytes result sent to driver +26/04/01 08:38:52 INFO TaskSetManager: Starting task 30.0 in stage 16.0 (TID 364) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:52 INFO TaskSetManager: Finished task 22.0 in stage 16.0 (TID 356) in 792 ms on 10.0.0.133 (executor driver) (23/200) +26/04/01 08:38:52 INFO Executor: Running task 30.0 in stage 16.0 (TID 364) +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO Executor: Finished task 23.0 in stage 16.0 (TID 357). 7223 bytes result sent to driver +26/04/01 08:38:52 INFO TaskSetManager: Starting task 31.0 in stage 16.0 (TID 365) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:52 INFO Executor: Running task 31.0 in stage 16.0 (TID 365) +26/04/01 08:38:52 INFO TaskSetManager: Finished task 23.0 in stage 16.0 (TID 357) in 793 ms on 10.0.0.133 (executor driver) (24/200) +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO Executor: Finished task 24.0 in stage 16.0 (TID 358). 7309 bytes result sent to driver +26/04/01 08:38:53 INFO TaskSetManager: Starting task 32.0 in stage 16.0 (TID 366) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:53 INFO Executor: Running task 32.0 in stage 16.0 (TID 366) +26/04/01 08:38:53 INFO TaskSetManager: Finished task 24.0 in stage 16.0 (TID 358) in 710 ms on 10.0.0.133 (executor driver) (25/200) +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO Executor: Finished task 25.0 in stage 16.0 (TID 359). 7309 bytes result sent to driver +26/04/01 08:38:53 INFO TaskSetManager: Starting task 33.0 in stage 16.0 (TID 367) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:53 INFO TaskSetManager: Finished task 25.0 in stage 16.0 (TID 359) in 871 ms on 10.0.0.133 (executor driver) (26/200) +26/04/01 08:38:53 INFO Executor: Running task 33.0 in stage 16.0 (TID 367) +26/04/01 08:38:53 INFO Executor: Finished task 26.0 in stage 16.0 (TID 360). 7309 bytes result sent to driver +26/04/01 08:38:53 INFO TaskSetManager: Starting task 34.0 in stage 16.0 (TID 368) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:53 INFO Executor: Running task 34.0 in stage 16.0 (TID 368) +26/04/01 08:38:53 INFO TaskSetManager: Finished task 26.0 in stage 16.0 (TID 360) in 851 ms on 10.0.0.133 (executor driver) (27/200) +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO Executor: Finished task 27.0 in stage 16.0 (TID 361). 7309 bytes result sent to driver +26/04/01 08:38:53 INFO TaskSetManager: Starting task 35.0 in stage 16.0 (TID 369) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:53 INFO Executor: Running task 35.0 in stage 16.0 (TID 369) +26/04/01 08:38:53 INFO TaskSetManager: Finished task 27.0 in stage 16.0 (TID 361) in 805 ms on 10.0.0.133 (executor driver) (28/200) +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO Executor: Finished task 28.0 in stage 16.0 (TID 362). 7309 bytes result sent to driver +26/04/01 08:38:53 INFO TaskSetManager: Starting task 36.0 in stage 16.0 (TID 370) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:53 INFO TaskSetManager: Finished task 28.0 in stage 16.0 (TID 362) in 801 ms on 10.0.0.133 (executor driver) (29/200) +26/04/01 08:38:53 INFO Executor: Running task 36.0 in stage 16.0 (TID 370) +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO Executor: Finished task 29.0 in stage 16.0 (TID 363). 7266 bytes result sent to driver +26/04/01 08:38:53 INFO TaskSetManager: Starting task 37.0 in stage 16.0 (TID 371) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:53 INFO TaskSetManager: Finished task 29.0 in stage 16.0 (TID 363) in 813 ms on 10.0.0.133 (executor driver) (30/200) +26/04/01 08:38:53 INFO Executor: Running task 37.0 in stage 16.0 (TID 371) +26/04/01 08:38:53 INFO Executor: Finished task 30.0 in stage 16.0 (TID 364). 7266 bytes result sent to driver +26/04/01 08:38:53 INFO TaskSetManager: Starting task 38.0 in stage 16.0 (TID 372) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:53 INFO Executor: Running task 38.0 in stage 16.0 (TID 372) +26/04/01 08:38:53 INFO TaskSetManager: Finished task 30.0 in stage 16.0 (TID 364) in 811 ms on 10.0.0.133 (executor driver) (31/200) +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (28.0 KiB) non-empty blocks including 8 (28.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO Executor: Finished task 31.0 in stage 16.0 (TID 365). 7266 bytes result sent to driver +26/04/01 08:38:53 INFO TaskSetManager: Starting task 39.0 in stage 16.0 (TID 373) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:53 INFO TaskSetManager: Finished task 31.0 in stage 16.0 (TID 365) in 812 ms on 10.0.0.133 (executor driver) (32/200) +26/04/01 08:38:53 INFO Executor: Running task 39.0 in stage 16.0 (TID 373) +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (28.0 KiB) non-empty blocks including 8 (28.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO Executor: Finished task 32.0 in stage 16.0 (TID 366). 7266 bytes result sent to driver +26/04/01 08:38:54 INFO TaskSetManager: Starting task 40.0 in stage 16.0 (TID 374) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:54 INFO Executor: Running task 40.0 in stage 16.0 (TID 374) +26/04/01 08:38:54 INFO TaskSetManager: Finished task 32.0 in stage 16.0 (TID 366) in 669 ms on 10.0.0.133 (executor driver) (33/200) +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO Executor: Finished task 33.0 in stage 16.0 (TID 367). 7266 bytes result sent to driver +26/04/01 08:38:54 INFO TaskSetManager: Starting task 41.0 in stage 16.0 (TID 375) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:54 INFO Executor: Running task 41.0 in stage 16.0 (TID 375) +26/04/01 08:38:54 INFO TaskSetManager: Finished task 33.0 in stage 16.0 (TID 367) in 801 ms on 10.0.0.133 (executor driver) (34/200) +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO Executor: Finished task 34.0 in stage 16.0 (TID 368). 7266 bytes result sent to driver +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO TaskSetManager: Starting task 42.0 in stage 16.0 (TID 376) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:54 INFO Executor: Running task 42.0 in stage 16.0 (TID 376) +26/04/01 08:38:54 INFO TaskSetManager: Finished task 34.0 in stage 16.0 (TID 368) in 804 ms on 10.0.0.133 (executor driver) (35/200) +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO Executor: Finished task 35.0 in stage 16.0 (TID 369). 7223 bytes result sent to driver +26/04/01 08:38:54 INFO TaskSetManager: Starting task 43.0 in stage 16.0 (TID 377) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:54 INFO TaskSetManager: Finished task 35.0 in stage 16.0 (TID 369) in 798 ms on 10.0.0.133 (executor driver) (36/200) +26/04/01 08:38:54 INFO Executor: Running task 43.0 in stage 16.0 (TID 377) +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO Executor: Finished task 36.0 in stage 16.0 (TID 370). 7266 bytes result sent to driver +26/04/01 08:38:54 INFO TaskSetManager: Starting task 44.0 in stage 16.0 (TID 378) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:54 INFO TaskSetManager: Finished task 36.0 in stage 16.0 (TID 370) in 788 ms on 10.0.0.133 (executor driver) (37/200) +26/04/01 08:38:54 INFO Executor: Running task 44.0 in stage 16.0 (TID 378) +26/04/01 08:38:54 INFO Executor: Finished task 38.0 in stage 16.0 (TID 372). 7223 bytes result sent to driver +26/04/01 08:38:54 INFO TaskSetManager: Starting task 45.0 in stage 16.0 (TID 379) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:54 INFO Executor: Running task 45.0 in stage 16.0 (TID 379) +26/04/01 08:38:54 INFO TaskSetManager: Finished task 38.0 in stage 16.0 (TID 372) in 775 ms on 10.0.0.133 (executor driver) (38/200) +26/04/01 08:38:54 INFO Executor: Finished task 39.0 in stage 16.0 (TID 373). 7223 bytes result sent to driver +26/04/01 08:38:54 INFO TaskSetManager: Starting task 46.0 in stage 16.0 (TID 380) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:54 INFO Executor: Running task 46.0 in stage 16.0 (TID 380) +26/04/01 08:38:54 INFO TaskSetManager: Finished task 39.0 in stage 16.0 (TID 373) in 774 ms on 10.0.0.133 (executor driver) (39/200) +26/04/01 08:38:54 INFO Executor: Finished task 37.0 in stage 16.0 (TID 371). 7266 bytes result sent to driver +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO TaskSetManager: Starting task 47.0 in stage 16.0 (TID 381) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:54 INFO Executor: Running task 47.0 in stage 16.0 (TID 381) +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO TaskSetManager: Finished task 37.0 in stage 16.0 (TID 371) in 778 ms on 10.0.0.133 (executor driver) (40/200) +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO Executor: Finished task 40.0 in stage 16.0 (TID 374). 7266 bytes result sent to driver +26/04/01 08:38:54 INFO TaskSetManager: Starting task 48.0 in stage 16.0 (TID 382) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:54 INFO Executor: Running task 48.0 in stage 16.0 (TID 382) +26/04/01 08:38:54 INFO TaskSetManager: Finished task 40.0 in stage 16.0 (TID 374) in 667 ms on 10.0.0.133 (executor driver) (41/200) +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO Executor: Finished task 42.0 in stage 16.0 (TID 376). 7266 bytes result sent to driver +26/04/01 08:38:55 INFO TaskSetManager: Starting task 49.0 in stage 16.0 (TID 383) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:55 INFO TaskSetManager: Finished task 42.0 in stage 16.0 (TID 376) in 792 ms on 10.0.0.133 (executor driver) (42/200) +26/04/01 08:38:55 INFO Executor: Running task 49.0 in stage 16.0 (TID 383) +26/04/01 08:38:55 INFO Executor: Finished task 41.0 in stage 16.0 (TID 375). 7309 bytes result sent to driver +26/04/01 08:38:55 INFO TaskSetManager: Starting task 50.0 in stage 16.0 (TID 384) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:55 INFO Executor: Running task 50.0 in stage 16.0 (TID 384) +26/04/01 08:38:55 INFO TaskSetManager: Finished task 41.0 in stage 16.0 (TID 375) in 796 ms on 10.0.0.133 (executor driver) (43/200) +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO Executor: Finished task 43.0 in stage 16.0 (TID 377). 7266 bytes result sent to driver +26/04/01 08:38:55 INFO TaskSetManager: Starting task 51.0 in stage 16.0 (TID 385) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:55 INFO TaskSetManager: Finished task 43.0 in stage 16.0 (TID 377) in 816 ms on 10.0.0.133 (executor driver) (44/200) +26/04/01 08:38:55 INFO Executor: Running task 51.0 in stage 16.0 (TID 385) +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO Executor: Finished task 44.0 in stage 16.0 (TID 378). 7266 bytes result sent to driver +26/04/01 08:38:55 INFO TaskSetManager: Starting task 52.0 in stage 16.0 (TID 386) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:55 INFO TaskSetManager: Finished task 44.0 in stage 16.0 (TID 378) in 825 ms on 10.0.0.133 (executor driver) (45/200) +26/04/01 08:38:55 INFO Executor: Running task 52.0 in stage 16.0 (TID 386) +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO Executor: Finished task 46.0 in stage 16.0 (TID 380). 7266 bytes result sent to driver +26/04/01 08:38:55 INFO TaskSetManager: Starting task 53.0 in stage 16.0 (TID 387) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:55 INFO TaskSetManager: Finished task 46.0 in stage 16.0 (TID 380) in 833 ms on 10.0.0.133 (executor driver) (46/200) +26/04/01 08:38:55 INFO Executor: Running task 53.0 in stage 16.0 (TID 387) +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO Executor: Finished task 45.0 in stage 16.0 (TID 379). 7266 bytes result sent to driver +26/04/01 08:38:55 INFO TaskSetManager: Starting task 54.0 in stage 16.0 (TID 388) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:55 INFO Executor: Running task 54.0 in stage 16.0 (TID 388) +26/04/01 08:38:55 INFO TaskSetManager: Finished task 45.0 in stage 16.0 (TID 379) in 837 ms on 10.0.0.133 (executor driver) (47/200) +26/04/01 08:38:55 INFO Executor: Finished task 47.0 in stage 16.0 (TID 381). 7266 bytes result sent to driver +26/04/01 08:38:55 INFO TaskSetManager: Starting task 55.0 in stage 16.0 (TID 389) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:55 INFO TaskSetManager: Finished task 47.0 in stage 16.0 (TID 381) in 836 ms on 10.0.0.133 (executor driver) (48/200) +26/04/01 08:38:55 INFO Executor: Running task 55.0 in stage 16.0 (TID 389) +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO Executor: Finished task 48.0 in stage 16.0 (TID 382). 7266 bytes result sent to driver +26/04/01 08:38:55 INFO TaskSetManager: Starting task 56.0 in stage 16.0 (TID 390) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:55 INFO Executor: Running task 56.0 in stage 16.0 (TID 390) +26/04/01 08:38:55 INFO TaskSetManager: Finished task 48.0 in stage 16.0 (TID 382) in 729 ms on 10.0.0.133 (executor driver) (49/200) +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO Executor: Finished task 49.0 in stage 16.0 (TID 383). 7266 bytes result sent to driver +26/04/01 08:38:56 INFO TaskSetManager: Starting task 57.0 in stage 16.0 (TID 391) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:56 INFO Executor: Running task 57.0 in stage 16.0 (TID 391) +26/04/01 08:38:56 INFO TaskSetManager: Finished task 49.0 in stage 16.0 (TID 383) in 940 ms on 10.0.0.133 (executor driver) (50/200) +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO Executor: Finished task 51.0 in stage 16.0 (TID 385). 7223 bytes result sent to driver +26/04/01 08:38:56 INFO TaskSetManager: Starting task 58.0 in stage 16.0 (TID 392) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:56 INFO Executor: Running task 58.0 in stage 16.0 (TID 392) +26/04/01 08:38:56 INFO TaskSetManager: Finished task 51.0 in stage 16.0 (TID 385) in 906 ms on 10.0.0.133 (executor driver) (51/200) +26/04/01 08:38:56 INFO Executor: Finished task 50.0 in stage 16.0 (TID 384). 7223 bytes result sent to driver +26/04/01 08:38:56 INFO TaskSetManager: Starting task 59.0 in stage 16.0 (TID 393) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:56 INFO Executor: Running task 59.0 in stage 16.0 (TID 393) +26/04/01 08:38:56 INFO TaskSetManager: Finished task 50.0 in stage 16.0 (TID 384) in 942 ms on 10.0.0.133 (executor driver) (52/200) +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO Executor: Finished task 52.0 in stage 16.0 (TID 386). 7223 bytes result sent to driver +26/04/01 08:38:56 INFO TaskSetManager: Starting task 60.0 in stage 16.0 (TID 394) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:56 INFO Executor: Running task 60.0 in stage 16.0 (TID 394) +26/04/01 08:38:56 INFO TaskSetManager: Finished task 52.0 in stage 16.0 (TID 386) in 898 ms on 10.0.0.133 (executor driver) (53/200) +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO Executor: Finished task 53.0 in stage 16.0 (TID 387). 7223 bytes result sent to driver +26/04/01 08:38:56 INFO TaskSetManager: Starting task 61.0 in stage 16.0 (TID 395) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:56 INFO TaskSetManager: Finished task 53.0 in stage 16.0 (TID 387) in 891 ms on 10.0.0.133 (executor driver) (54/200) +26/04/01 08:38:56 INFO Executor: Running task 61.0 in stage 16.0 (TID 395) +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO Executor: Finished task 54.0 in stage 16.0 (TID 388). 7223 bytes result sent to driver +26/04/01 08:38:56 INFO TaskSetManager: Starting task 62.0 in stage 16.0 (TID 396) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:56 INFO Executor: Running task 62.0 in stage 16.0 (TID 396) +26/04/01 08:38:56 INFO TaskSetManager: Finished task 54.0 in stage 16.0 (TID 388) in 890 ms on 10.0.0.133 (executor driver) (55/200) +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO Executor: Finished task 56.0 in stage 16.0 (TID 390). 7223 bytes result sent to driver +26/04/01 08:38:56 INFO TaskSetManager: Starting task 63.0 in stage 16.0 (TID 397) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:56 INFO TaskSetManager: Finished task 56.0 in stage 16.0 (TID 390) in 873 ms on 10.0.0.133 (executor driver) (56/200) +26/04/01 08:38:56 INFO Executor: Running task 63.0 in stage 16.0 (TID 397) +26/04/01 08:38:56 INFO Executor: Finished task 55.0 in stage 16.0 (TID 389). 7223 bytes result sent to driver +26/04/01 08:38:56 INFO TaskSetManager: Starting task 64.0 in stage 16.0 (TID 398) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:56 INFO TaskSetManager: Finished task 55.0 in stage 16.0 (TID 389) in 896 ms on 10.0.0.133 (executor driver) (57/200) +26/04/01 08:38:56 INFO Executor: Running task 64.0 in stage 16.0 (TID 398) +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO Executor: Finished task 57.0 in stage 16.0 (TID 391). 7309 bytes result sent to driver +26/04/01 08:38:57 INFO TaskSetManager: Starting task 65.0 in stage 16.0 (TID 399) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 57.0 in stage 16.0 (TID 391) in 779 ms on 10.0.0.133 (executor driver) (58/200) +26/04/01 08:38:57 INFO Executor: Running task 65.0 in stage 16.0 (TID 399) +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO Executor: Finished task 58.0 in stage 16.0 (TID 392). 7266 bytes result sent to driver +26/04/01 08:38:57 INFO TaskSetManager: Starting task 66.0 in stage 16.0 (TID 400) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO Executor: Running task 66.0 in stage 16.0 (TID 400) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 58.0 in stage 16.0 (TID 392) in 832 ms on 10.0.0.133 (executor driver) (59/200) +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO Executor: Finished task 59.0 in stage 16.0 (TID 393). 7266 bytes result sent to driver +26/04/01 08:38:57 INFO TaskSetManager: Starting task 67.0 in stage 16.0 (TID 401) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 59.0 in stage 16.0 (TID 393) in 838 ms on 10.0.0.133 (executor driver) (60/200) +26/04/01 08:38:57 INFO Executor: Running task 67.0 in stage 16.0 (TID 401) +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO Executor: Finished task 60.0 in stage 16.0 (TID 394). 7266 bytes result sent to driver +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO TaskSetManager: Starting task 68.0 in stage 16.0 (TID 402) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO Executor: Running task 68.0 in stage 16.0 (TID 402) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 60.0 in stage 16.0 (TID 394) in 836 ms on 10.0.0.133 (executor driver) (61/200) +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO Executor: Finished task 61.0 in stage 16.0 (TID 395). 7266 bytes result sent to driver +26/04/01 08:38:57 INFO TaskSetManager: Starting task 69.0 in stage 16.0 (TID 403) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 61.0 in stage 16.0 (TID 395) in 838 ms on 10.0.0.133 (executor driver) (62/200) +26/04/01 08:38:57 INFO Executor: Running task 69.0 in stage 16.0 (TID 403) +26/04/01 08:38:57 INFO Executor: Finished task 63.0 in stage 16.0 (TID 397). 7309 bytes result sent to driver +26/04/01 08:38:57 INFO TaskSetManager: Starting task 70.0 in stage 16.0 (TID 404) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO Executor: Running task 70.0 in stage 16.0 (TID 404) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 63.0 in stage 16.0 (TID 397) in 831 ms on 10.0.0.133 (executor driver) (63/200) +26/04/01 08:38:57 INFO Executor: Finished task 62.0 in stage 16.0 (TID 396). 7266 bytes result sent to driver +26/04/01 08:38:57 INFO TaskSetManager: Starting task 71.0 in stage 16.0 (TID 405) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 62.0 in stage 16.0 (TID 396) in 837 ms on 10.0.0.133 (executor driver) (64/200) +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO Executor: Running task 71.0 in stage 16.0 (TID 405) +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO Executor: Finished task 64.0 in stage 16.0 (TID 398). 7266 bytes result sent to driver +26/04/01 08:38:57 INFO TaskSetManager: Starting task 72.0 in stage 16.0 (TID 406) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 64.0 in stage 16.0 (TID 398) in 831 ms on 10.0.0.133 (executor driver) (65/200) +26/04/01 08:38:57 INFO Executor: Running task 72.0 in stage 16.0 (TID 406) +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO Executor: Finished task 65.0 in stage 16.0 (TID 399). 7266 bytes result sent to driver +26/04/01 08:38:57 INFO TaskSetManager: Starting task 73.0 in stage 16.0 (TID 407) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO Executor: Running task 73.0 in stage 16.0 (TID 407) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 65.0 in stage 16.0 (TID 399) in 698 ms on 10.0.0.133 (executor driver) (66/200) +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO Executor: Finished task 66.0 in stage 16.0 (TID 400). 7266 bytes result sent to driver +26/04/01 08:38:57 INFO TaskSetManager: Starting task 74.0 in stage 16.0 (TID 408) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO Executor: Running task 74.0 in stage 16.0 (TID 408) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 66.0 in stage 16.0 (TID 400) in 693 ms on 10.0.0.133 (executor driver) (67/200) +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO Executor: Finished task 67.0 in stage 16.0 (TID 401). 7223 bytes result sent to driver +26/04/01 08:38:57 INFO TaskSetManager: Starting task 75.0 in stage 16.0 (TID 409) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO Executor: Running task 75.0 in stage 16.0 (TID 409) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 67.0 in stage 16.0 (TID 401) in 731 ms on 10.0.0.133 (executor driver) (68/200) +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO Executor: Finished task 68.0 in stage 16.0 (TID 402). 7223 bytes result sent to driver +26/04/01 08:38:57 INFO TaskSetManager: Starting task 76.0 in stage 16.0 (TID 410) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 68.0 in stage 16.0 (TID 402) in 760 ms on 10.0.0.133 (executor driver) (69/200) +26/04/01 08:38:57 INFO Executor: Running task 76.0 in stage 16.0 (TID 410) +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO Executor: Finished task 69.0 in stage 16.0 (TID 403). 7223 bytes result sent to driver +26/04/01 08:38:57 INFO TaskSetManager: Starting task 77.0 in stage 16.0 (TID 411) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO Executor: Running task 77.0 in stage 16.0 (TID 411) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 69.0 in stage 16.0 (TID 403) in 773 ms on 10.0.0.133 (executor driver) (70/200) +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO Executor: Finished task 70.0 in stage 16.0 (TID 404). 7266 bytes result sent to driver +26/04/01 08:38:57 INFO TaskSetManager: Starting task 78.0 in stage 16.0 (TID 412) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO Executor: Running task 78.0 in stage 16.0 (TID 412) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 70.0 in stage 16.0 (TID 404) in 781 ms on 10.0.0.133 (executor driver) (71/200) +26/04/01 08:38:57 INFO Executor: Finished task 72.0 in stage 16.0 (TID 406). 7223 bytes result sent to driver +26/04/01 08:38:57 INFO TaskSetManager: Starting task 79.0 in stage 16.0 (TID 413) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 72.0 in stage 16.0 (TID 406) in 780 ms on 10.0.0.133 (executor driver) (72/200) +26/04/01 08:38:57 INFO Executor: Running task 79.0 in stage 16.0 (TID 413) +26/04/01 08:38:57 INFO Executor: Finished task 71.0 in stage 16.0 (TID 405). 7223 bytes result sent to driver +26/04/01 08:38:57 INFO TaskSetManager: Starting task 80.0 in stage 16.0 (TID 414) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:57 INFO Executor: Running task 80.0 in stage 16.0 (TID 414) +26/04/01 08:38:57 INFO TaskSetManager: Finished task 71.0 in stage 16.0 (TID 405) in 780 ms on 10.0.0.133 (executor driver) (73/200) +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO Executor: Finished task 73.0 in stage 16.0 (TID 407). 7309 bytes result sent to driver +26/04/01 08:38:58 INFO TaskSetManager: Starting task 81.0 in stage 16.0 (TID 415) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:58 INFO TaskSetManager: Finished task 73.0 in stage 16.0 (TID 407) in 688 ms on 10.0.0.133 (executor driver) (74/200) +26/04/01 08:38:58 INFO Executor: Running task 81.0 in stage 16.0 (TID 415) +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO Executor: Finished task 74.0 in stage 16.0 (TID 408). 7266 bytes result sent to driver +26/04/01 08:38:58 INFO TaskSetManager: Starting task 82.0 in stage 16.0 (TID 416) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:58 INFO TaskSetManager: Finished task 74.0 in stage 16.0 (TID 408) in 692 ms on 10.0.0.133 (executor driver) (75/200) +26/04/01 08:38:58 INFO Executor: Running task 82.0 in stage 16.0 (TID 416) +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO Executor: Finished task 75.0 in stage 16.0 (TID 409). 7266 bytes result sent to driver +26/04/01 08:38:58 INFO TaskSetManager: Starting task 83.0 in stage 16.0 (TID 417) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:58 INFO TaskSetManager: Finished task 75.0 in stage 16.0 (TID 409) in 691 ms on 10.0.0.133 (executor driver) (76/200) +26/04/01 08:38:58 INFO Executor: Running task 83.0 in stage 16.0 (TID 417) +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO Executor: Finished task 76.0 in stage 16.0 (TID 410). 7309 bytes result sent to driver +26/04/01 08:38:58 INFO TaskSetManager: Starting task 84.0 in stage 16.0 (TID 418) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:58 INFO Executor: Running task 84.0 in stage 16.0 (TID 418) +26/04/01 08:38:58 INFO TaskSetManager: Finished task 76.0 in stage 16.0 (TID 410) in 770 ms on 10.0.0.133 (executor driver) (77/200) +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO Executor: Finished task 77.0 in stage 16.0 (TID 411). 7266 bytes result sent to driver +26/04/01 08:38:58 INFO TaskSetManager: Starting task 85.0 in stage 16.0 (TID 419) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:58 INFO TaskSetManager: Finished task 77.0 in stage 16.0 (TID 411) in 800 ms on 10.0.0.133 (executor driver) (78/200) +26/04/01 08:38:58 INFO Executor: Running task 85.0 in stage 16.0 (TID 419) +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO Executor: Finished task 78.0 in stage 16.0 (TID 412). 7266 bytes result sent to driver +26/04/01 08:38:58 INFO TaskSetManager: Starting task 86.0 in stage 16.0 (TID 420) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:58 INFO TaskSetManager: Finished task 78.0 in stage 16.0 (TID 412) in 808 ms on 10.0.0.133 (executor driver) (79/200) +26/04/01 08:38:58 INFO Executor: Running task 86.0 in stage 16.0 (TID 420) +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO Executor: Finished task 79.0 in stage 16.0 (TID 413). 7266 bytes result sent to driver +26/04/01 08:38:58 INFO TaskSetManager: Starting task 87.0 in stage 16.0 (TID 421) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:58 INFO TaskSetManager: Finished task 79.0 in stage 16.0 (TID 413) in 810 ms on 10.0.0.133 (executor driver) (80/200) +26/04/01 08:38:58 INFO Executor: Running task 87.0 in stage 16.0 (TID 421) +26/04/01 08:38:58 INFO Executor: Finished task 80.0 in stage 16.0 (TID 414). 7266 bytes result sent to driver +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO TaskSetManager: Starting task 88.0 in stage 16.0 (TID 422) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:58 INFO Executor: Running task 88.0 in stage 16.0 (TID 422) +26/04/01 08:38:58 INFO TaskSetManager: Finished task 80.0 in stage 16.0 (TID 414) in 812 ms on 10.0.0.133 (executor driver) (81/200) +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO Executor: Finished task 81.0 in stage 16.0 (TID 415). 7223 bytes result sent to driver +26/04/01 08:38:59 INFO TaskSetManager: Starting task 89.0 in stage 16.0 (TID 423) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:59 INFO Executor: Running task 89.0 in stage 16.0 (TID 423) +26/04/01 08:38:59 INFO TaskSetManager: Finished task 81.0 in stage 16.0 (TID 415) in 695 ms on 10.0.0.133 (executor driver) (82/200) +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO Executor: Finished task 82.0 in stage 16.0 (TID 416). 7223 bytes result sent to driver +26/04/01 08:38:59 INFO TaskSetManager: Starting task 90.0 in stage 16.0 (TID 424) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:59 INFO Executor: Running task 90.0 in stage 16.0 (TID 424) +26/04/01 08:38:59 INFO TaskSetManager: Finished task 82.0 in stage 16.0 (TID 416) in 688 ms on 10.0.0.133 (executor driver) (83/200) +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO Executor: Finished task 83.0 in stage 16.0 (TID 417). 7223 bytes result sent to driver +26/04/01 08:38:59 INFO TaskSetManager: Starting task 91.0 in stage 16.0 (TID 425) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:59 INFO TaskSetManager: Finished task 83.0 in stage 16.0 (TID 417) in 704 ms on 10.0.0.133 (executor driver) (84/200) +26/04/01 08:38:59 INFO Executor: Running task 91.0 in stage 16.0 (TID 425) +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO Executor: Finished task 84.0 in stage 16.0 (TID 418). 7266 bytes result sent to driver +26/04/01 08:38:59 INFO TaskSetManager: Starting task 92.0 in stage 16.0 (TID 426) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:59 INFO TaskSetManager: Finished task 84.0 in stage 16.0 (TID 418) in 666 ms on 10.0.0.133 (executor driver) (85/200) +26/04/01 08:38:59 INFO Executor: Running task 92.0 in stage 16.0 (TID 426) +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO Executor: Finished task 85.0 in stage 16.0 (TID 419). 7223 bytes result sent to driver +26/04/01 08:38:59 INFO TaskSetManager: Starting task 93.0 in stage 16.0 (TID 427) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:59 INFO TaskSetManager: Finished task 85.0 in stage 16.0 (TID 419) in 686 ms on 10.0.0.133 (executor driver) (86/200) +26/04/01 08:38:59 INFO Executor: Running task 93.0 in stage 16.0 (TID 427) +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO Executor: Finished task 86.0 in stage 16.0 (TID 420). 7223 bytes result sent to driver +26/04/01 08:38:59 INFO TaskSetManager: Starting task 94.0 in stage 16.0 (TID 428) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:59 INFO TaskSetManager: Finished task 86.0 in stage 16.0 (TID 420) in 684 ms on 10.0.0.133 (executor driver) (87/200) +26/04/01 08:38:59 INFO Executor: Running task 94.0 in stage 16.0 (TID 428) +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO Executor: Finished task 87.0 in stage 16.0 (TID 421). 7223 bytes result sent to driver +26/04/01 08:38:59 INFO TaskSetManager: Starting task 95.0 in stage 16.0 (TID 429) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:59 INFO Executor: Running task 95.0 in stage 16.0 (TID 429) +26/04/01 08:38:59 INFO TaskSetManager: Finished task 87.0 in stage 16.0 (TID 421) in 698 ms on 10.0.0.133 (executor driver) (88/200) +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO Executor: Finished task 88.0 in stage 16.0 (TID 422). 7223 bytes result sent to driver +26/04/01 08:38:59 INFO TaskSetManager: Starting task 96.0 in stage 16.0 (TID 430) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:59 INFO Executor: Running task 96.0 in stage 16.0 (TID 430) +26/04/01 08:38:59 INFO TaskSetManager: Finished task 88.0 in stage 16.0 (TID 422) in 698 ms on 10.0.0.133 (executor driver) (89/200) +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO Executor: Finished task 89.0 in stage 16.0 (TID 423). 7266 bytes result sent to driver +26/04/01 08:38:59 INFO TaskSetManager: Starting task 97.0 in stage 16.0 (TID 431) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:59 INFO Executor: Running task 97.0 in stage 16.0 (TID 431) +26/04/01 08:38:59 INFO TaskSetManager: Finished task 89.0 in stage 16.0 (TID 423) in 654 ms on 10.0.0.133 (executor driver) (90/200) +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (28.1 KiB) non-empty blocks including 8 (28.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO Executor: Finished task 90.0 in stage 16.0 (TID 424). 7266 bytes result sent to driver +26/04/01 08:38:59 INFO TaskSetManager: Starting task 98.0 in stage 16.0 (TID 432) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:59 INFO Executor: Running task 98.0 in stage 16.0 (TID 432) +26/04/01 08:38:59 INFO TaskSetManager: Finished task 90.0 in stage 16.0 (TID 424) in 669 ms on 10.0.0.133 (executor driver) (91/200) +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (29.5 KiB) non-empty blocks including 8 (29.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO Executor: Finished task 91.0 in stage 16.0 (TID 425). 7266 bytes result sent to driver +26/04/01 08:38:59 INFO TaskSetManager: Starting task 99.0 in stage 16.0 (TID 433) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9270 bytes) +26/04/01 08:38:59 INFO Executor: Running task 99.0 in stage 16.0 (TID 433) +26/04/01 08:38:59 INFO TaskSetManager: Finished task 91.0 in stage 16.0 (TID 425) in 702 ms on 10.0.0.133 (executor driver) (92/200) +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:38:59 INFO Executor: Finished task 92.0 in stage 16.0 (TID 426). 7266 bytes result sent to driver +26/04/01 08:39:00 INFO TaskSetManager: Starting task 100.0 in stage 16.0 (TID 434) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:00 INFO Executor: Running task 100.0 in stage 16.0 (TID 434) +26/04/01 08:39:00 INFO TaskSetManager: Finished task 92.0 in stage 16.0 (TID 426) in 697 ms on 10.0.0.133 (executor driver) (93/200) +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO Executor: Finished task 93.0 in stage 16.0 (TID 427). 7266 bytes result sent to driver +26/04/01 08:39:00 INFO TaskSetManager: Starting task 101.0 in stage 16.0 (TID 435) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:00 INFO TaskSetManager: Finished task 93.0 in stage 16.0 (TID 427) in 833 ms on 10.0.0.133 (executor driver) (94/200) +26/04/01 08:39:00 INFO Executor: Running task 101.0 in stage 16.0 (TID 435) +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO Executor: Finished task 94.0 in stage 16.0 (TID 428). 7266 bytes result sent to driver +26/04/01 08:39:00 INFO TaskSetManager: Starting task 102.0 in stage 16.0 (TID 436) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:00 INFO Executor: Running task 102.0 in stage 16.0 (TID 436) +26/04/01 08:39:00 INFO TaskSetManager: Finished task 94.0 in stage 16.0 (TID 428) in 828 ms on 10.0.0.133 (executor driver) (95/200) +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO Executor: Finished task 96.0 in stage 16.0 (TID 430). 7266 bytes result sent to driver +26/04/01 08:39:00 INFO TaskSetManager: Starting task 103.0 in stage 16.0 (TID 437) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:00 INFO TaskSetManager: Finished task 96.0 in stage 16.0 (TID 430) in 814 ms on 10.0.0.133 (executor driver) (96/200) +26/04/01 08:39:00 INFO Executor: Running task 103.0 in stage 16.0 (TID 437) +26/04/01 08:39:00 INFO Executor: Finished task 95.0 in stage 16.0 (TID 429). 7266 bytes result sent to driver +26/04/01 08:39:00 INFO TaskSetManager: Starting task 104.0 in stage 16.0 (TID 438) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:00 INFO TaskSetManager: Finished task 95.0 in stage 16.0 (TID 429) in 816 ms on 10.0.0.133 (executor driver) (97/200) +26/04/01 08:39:00 INFO Executor: Running task 104.0 in stage 16.0 (TID 438) +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO Executor: Finished task 97.0 in stage 16.0 (TID 431). 7223 bytes result sent to driver +26/04/01 08:39:00 INFO TaskSetManager: Starting task 105.0 in stage 16.0 (TID 439) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:00 INFO Executor: Running task 105.0 in stage 16.0 (TID 439) +26/04/01 08:39:00 INFO TaskSetManager: Finished task 97.0 in stage 16.0 (TID 431) in 727 ms on 10.0.0.133 (executor driver) (98/200) +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO Executor: Finished task 98.0 in stage 16.0 (TID 432). 7223 bytes result sent to driver +26/04/01 08:39:00 INFO TaskSetManager: Starting task 106.0 in stage 16.0 (TID 440) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:00 INFO Executor: Running task 106.0 in stage 16.0 (TID 440) +26/04/01 08:39:00 INFO TaskSetManager: Finished task 98.0 in stage 16.0 (TID 432) in 675 ms on 10.0.0.133 (executor driver) (99/200) +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO Executor: Finished task 99.0 in stage 16.0 (TID 433). 7223 bytes result sent to driver +26/04/01 08:39:00 INFO TaskSetManager: Starting task 107.0 in stage 16.0 (TID 441) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:00 INFO Executor: Running task 107.0 in stage 16.0 (TID 441) +26/04/01 08:39:00 INFO TaskSetManager: Finished task 99.0 in stage 16.0 (TID 433) in 657 ms on 10.0.0.133 (executor driver) (100/200) +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO Executor: Finished task 100.0 in stage 16.0 (TID 434). 7266 bytes result sent to driver +26/04/01 08:39:00 INFO TaskSetManager: Starting task 108.0 in stage 16.0 (TID 442) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:00 INFO TaskSetManager: Finished task 100.0 in stage 16.0 (TID 434) in 666 ms on 10.0.0.133 (executor driver) (101/200) +26/04/01 08:39:00 INFO Executor: Running task 108.0 in stage 16.0 (TID 442) +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO Executor: Finished task 101.0 in stage 16.0 (TID 435). 7266 bytes result sent to driver +26/04/01 08:39:00 INFO TaskSetManager: Starting task 109.0 in stage 16.0 (TID 443) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:00 INFO TaskSetManager: Finished task 101.0 in stage 16.0 (TID 435) in 705 ms on 10.0.0.133 (executor driver) (102/200) +26/04/01 08:39:00 INFO Executor: Running task 109.0 in stage 16.0 (TID 443) +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO Executor: Finished task 102.0 in stage 16.0 (TID 436). 7223 bytes result sent to driver +26/04/01 08:39:00 INFO TaskSetManager: Starting task 110.0 in stage 16.0 (TID 444) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:00 INFO Executor: Running task 110.0 in stage 16.0 (TID 444) +26/04/01 08:39:00 INFO TaskSetManager: Finished task 102.0 in stage 16.0 (TID 436) in 707 ms on 10.0.0.133 (executor driver) (103/200) +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO Executor: Finished task 103.0 in stage 16.0 (TID 437). 7223 bytes result sent to driver +26/04/01 08:39:00 INFO TaskSetManager: Starting task 111.0 in stage 16.0 (TID 445) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:00 INFO Executor: Running task 111.0 in stage 16.0 (TID 445) +26/04/01 08:39:00 INFO TaskSetManager: Finished task 103.0 in stage 16.0 (TID 437) in 708 ms on 10.0.0.133 (executor driver) (104/200) +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO Executor: Finished task 104.0 in stage 16.0 (TID 438). 7223 bytes result sent to driver +26/04/01 08:39:00 INFO TaskSetManager: Starting task 112.0 in stage 16.0 (TID 446) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:00 INFO Executor: Running task 112.0 in stage 16.0 (TID 446) +26/04/01 08:39:00 INFO TaskSetManager: Finished task 104.0 in stage 16.0 (TID 438) in 710 ms on 10.0.0.133 (executor driver) (105/200) +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO Executor: Finished task 105.0 in stage 16.0 (TID 439). 7266 bytes result sent to driver +26/04/01 08:39:01 INFO TaskSetManager: Starting task 113.0 in stage 16.0 (TID 447) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:01 INFO Executor: Running task 113.0 in stage 16.0 (TID 447) +26/04/01 08:39:01 INFO TaskSetManager: Finished task 105.0 in stage 16.0 (TID 439) in 661 ms on 10.0.0.133 (executor driver) (106/200) +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO Executor: Finished task 106.0 in stage 16.0 (TID 440). 7223 bytes result sent to driver +26/04/01 08:39:01 INFO TaskSetManager: Starting task 114.0 in stage 16.0 (TID 448) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:01 INFO Executor: Running task 114.0 in stage 16.0 (TID 448) +26/04/01 08:39:01 INFO TaskSetManager: Finished task 106.0 in stage 16.0 (TID 440) in 655 ms on 10.0.0.133 (executor driver) (107/200) +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO Executor: Finished task 107.0 in stage 16.0 (TID 441). 7223 bytes result sent to driver +26/04/01 08:39:01 INFO TaskSetManager: Starting task 115.0 in stage 16.0 (TID 449) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:01 INFO TaskSetManager: Finished task 107.0 in stage 16.0 (TID 441) in 661 ms on 10.0.0.133 (executor driver) (108/200) +26/04/01 08:39:01 INFO Executor: Running task 115.0 in stage 16.0 (TID 449) +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO Executor: Finished task 108.0 in stage 16.0 (TID 442). 7266 bytes result sent to driver +26/04/01 08:39:01 INFO TaskSetManager: Starting task 116.0 in stage 16.0 (TID 450) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:01 INFO TaskSetManager: Finished task 108.0 in stage 16.0 (TID 442) in 677 ms on 10.0.0.133 (executor driver) (109/200) +26/04/01 08:39:01 INFO Executor: Running task 116.0 in stage 16.0 (TID 450) +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO Executor: Finished task 109.0 in stage 16.0 (TID 443). 7266 bytes result sent to driver +26/04/01 08:39:01 INFO TaskSetManager: Starting task 117.0 in stage 16.0 (TID 451) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:01 INFO Executor: Running task 117.0 in stage 16.0 (TID 451) +26/04/01 08:39:01 INFO TaskSetManager: Finished task 109.0 in stage 16.0 (TID 443) in 723 ms on 10.0.0.133 (executor driver) (110/200) +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO Executor: Finished task 110.0 in stage 16.0 (TID 444). 7266 bytes result sent to driver +26/04/01 08:39:01 INFO TaskSetManager: Starting task 118.0 in stage 16.0 (TID 452) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:01 INFO Executor: Running task 118.0 in stage 16.0 (TID 452) +26/04/01 08:39:01 INFO TaskSetManager: Finished task 110.0 in stage 16.0 (TID 444) in 719 ms on 10.0.0.133 (executor driver) (111/200) +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO Executor: Finished task 111.0 in stage 16.0 (TID 445). 7266 bytes result sent to driver +26/04/01 08:39:01 INFO TaskSetManager: Starting task 119.0 in stage 16.0 (TID 453) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:01 INFO TaskSetManager: Finished task 111.0 in stage 16.0 (TID 445) in 717 ms on 10.0.0.133 (executor driver) (112/200) +26/04/01 08:39:01 INFO Executor: Running task 119.0 in stage 16.0 (TID 453) +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO Executor: Finished task 112.0 in stage 16.0 (TID 446). 7266 bytes result sent to driver +26/04/01 08:39:01 INFO TaskSetManager: Starting task 120.0 in stage 16.0 (TID 454) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:01 INFO TaskSetManager: Finished task 112.0 in stage 16.0 (TID 446) in 715 ms on 10.0.0.133 (executor driver) (113/200) +26/04/01 08:39:01 INFO Executor: Running task 120.0 in stage 16.0 (TID 454) +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO Executor: Finished task 113.0 in stage 16.0 (TID 447). 7266 bytes result sent to driver +26/04/01 08:39:01 INFO TaskSetManager: Starting task 121.0 in stage 16.0 (TID 455) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:01 INFO Executor: Running task 121.0 in stage 16.0 (TID 455) +26/04/01 08:39:01 INFO TaskSetManager: Finished task 113.0 in stage 16.0 (TID 447) in 701 ms on 10.0.0.133 (executor driver) (114/200) +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO Executor: Finished task 114.0 in stage 16.0 (TID 448). 7266 bytes result sent to driver +26/04/01 08:39:01 INFO TaskSetManager: Starting task 122.0 in stage 16.0 (TID 456) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO Executor: Running task 122.0 in stage 16.0 (TID 456) +26/04/01 08:39:01 INFO TaskSetManager: Finished task 114.0 in stage 16.0 (TID 448) in 699 ms on 10.0.0.133 (executor driver) (115/200) +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO Executor: Finished task 115.0 in stage 16.0 (TID 449). 7266 bytes result sent to driver +26/04/01 08:39:01 INFO TaskSetManager: Starting task 123.0 in stage 16.0 (TID 457) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:01 INFO Executor: Running task 123.0 in stage 16.0 (TID 457) +26/04/01 08:39:01 INFO TaskSetManager: Finished task 115.0 in stage 16.0 (TID 449) in 686 ms on 10.0.0.133 (executor driver) (116/200) +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO Executor: Finished task 116.0 in stage 16.0 (TID 450). 7223 bytes result sent to driver +26/04/01 08:39:02 INFO TaskSetManager: Starting task 124.0 in stage 16.0 (TID 458) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:02 INFO Executor: Running task 124.0 in stage 16.0 (TID 458) +26/04/01 08:39:02 INFO TaskSetManager: Finished task 116.0 in stage 16.0 (TID 450) in 672 ms on 10.0.0.133 (executor driver) (117/200) +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO Executor: Finished task 117.0 in stage 16.0 (TID 451). 7223 bytes result sent to driver +26/04/01 08:39:02 INFO TaskSetManager: Starting task 125.0 in stage 16.0 (TID 459) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:02 INFO TaskSetManager: Finished task 117.0 in stage 16.0 (TID 451) in 713 ms on 10.0.0.133 (executor driver) (118/200) +26/04/01 08:39:02 INFO Executor: Running task 125.0 in stage 16.0 (TID 459) +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO Executor: Finished task 118.0 in stage 16.0 (TID 452). 7223 bytes result sent to driver +26/04/01 08:39:02 INFO TaskSetManager: Starting task 126.0 in stage 16.0 (TID 460) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:02 INFO TaskSetManager: Finished task 118.0 in stage 16.0 (TID 452) in 710 ms on 10.0.0.133 (executor driver) (119/200) +26/04/01 08:39:02 INFO Executor: Running task 126.0 in stage 16.0 (TID 460) +26/04/01 08:39:02 INFO Executor: Finished task 120.0 in stage 16.0 (TID 454). 7223 bytes result sent to driver +26/04/01 08:39:02 INFO TaskSetManager: Starting task 127.0 in stage 16.0 (TID 461) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:02 INFO TaskSetManager: Finished task 120.0 in stage 16.0 (TID 454) in 708 ms on 10.0.0.133 (executor driver) (120/200) +26/04/01 08:39:02 INFO Executor: Running task 127.0 in stage 16.0 (TID 461) +26/04/01 08:39:02 INFO Executor: Finished task 119.0 in stage 16.0 (TID 453). 7223 bytes result sent to driver +26/04/01 08:39:02 INFO TaskSetManager: Starting task 128.0 in stage 16.0 (TID 462) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO TaskSetManager: Finished task 119.0 in stage 16.0 (TID 453) in 709 ms on 10.0.0.133 (executor driver) (121/200) +26/04/01 08:39:02 INFO Executor: Running task 128.0 in stage 16.0 (TID 462) +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO Executor: Finished task 121.0 in stage 16.0 (TID 455). 7223 bytes result sent to driver +26/04/01 08:39:02 INFO TaskSetManager: Starting task 129.0 in stage 16.0 (TID 463) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:02 INFO Executor: Running task 129.0 in stage 16.0 (TID 463) +26/04/01 08:39:02 INFO TaskSetManager: Finished task 121.0 in stage 16.0 (TID 455) in 652 ms on 10.0.0.133 (executor driver) (122/200) +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO Executor: Finished task 122.0 in stage 16.0 (TID 456). 7223 bytes result sent to driver +26/04/01 08:39:02 INFO TaskSetManager: Starting task 130.0 in stage 16.0 (TID 464) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:02 INFO TaskSetManager: Finished task 122.0 in stage 16.0 (TID 456) in 657 ms on 10.0.0.133 (executor driver) (123/200) +26/04/01 08:39:02 INFO Executor: Running task 130.0 in stage 16.0 (TID 464) +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO Executor: Finished task 123.0 in stage 16.0 (TID 457). 7266 bytes result sent to driver +26/04/01 08:39:02 INFO TaskSetManager: Starting task 131.0 in stage 16.0 (TID 465) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:02 INFO Executor: Running task 131.0 in stage 16.0 (TID 465) +26/04/01 08:39:02 INFO TaskSetManager: Finished task 123.0 in stage 16.0 (TID 457) in 655 ms on 10.0.0.133 (executor driver) (124/200) +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO Executor: Finished task 124.0 in stage 16.0 (TID 458). 7223 bytes result sent to driver +26/04/01 08:39:02 INFO TaskSetManager: Starting task 132.0 in stage 16.0 (TID 466) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:02 INFO TaskSetManager: Finished task 124.0 in stage 16.0 (TID 458) in 666 ms on 10.0.0.133 (executor driver) (125/200) +26/04/01 08:39:02 INFO Executor: Running task 132.0 in stage 16.0 (TID 466) +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO Executor: Finished task 126.0 in stage 16.0 (TID 460). 7266 bytes result sent to driver +26/04/01 08:39:03 INFO TaskSetManager: Starting task 133.0 in stage 16.0 (TID 467) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO Executor: Finished task 127.0 in stage 16.0 (TID 461). 7309 bytes result sent to driver +26/04/01 08:39:03 INFO Executor: Running task 133.0 in stage 16.0 (TID 467) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 126.0 in stage 16.0 (TID 460) in 697 ms on 10.0.0.133 (executor driver) (126/200) +26/04/01 08:39:03 INFO TaskSetManager: Starting task 134.0 in stage 16.0 (TID 468) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO Executor: Running task 134.0 in stage 16.0 (TID 468) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 127.0 in stage 16.0 (TID 461) in 697 ms on 10.0.0.133 (executor driver) (127/200) +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO Executor: Finished task 125.0 in stage 16.0 (TID 459). 7309 bytes result sent to driver +26/04/01 08:39:03 INFO TaskSetManager: Starting task 135.0 in stage 16.0 (TID 469) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO Executor: Running task 135.0 in stage 16.0 (TID 469) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 125.0 in stage 16.0 (TID 459) in 749 ms on 10.0.0.133 (executor driver) (128/200) +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO Executor: Finished task 128.0 in stage 16.0 (TID 462). 7266 bytes result sent to driver +26/04/01 08:39:03 INFO TaskSetManager: Starting task 136.0 in stage 16.0 (TID 470) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO Executor: Running task 136.0 in stage 16.0 (TID 470) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 128.0 in stage 16.0 (TID 462) in 759 ms on 10.0.0.133 (executor driver) (129/200) +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO Executor: Finished task 129.0 in stage 16.0 (TID 463). 7266 bytes result sent to driver +26/04/01 08:39:03 INFO TaskSetManager: Starting task 137.0 in stage 16.0 (TID 471) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 129.0 in stage 16.0 (TID 463) in 672 ms on 10.0.0.133 (executor driver) (130/200) +26/04/01 08:39:03 INFO Executor: Running task 137.0 in stage 16.0 (TID 471) +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO Executor: Finished task 130.0 in stage 16.0 (TID 464). 7266 bytes result sent to driver +26/04/01 08:39:03 INFO TaskSetManager: Starting task 138.0 in stage 16.0 (TID 472) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 130.0 in stage 16.0 (TID 464) in 676 ms on 10.0.0.133 (executor driver) (131/200) +26/04/01 08:39:03 INFO Executor: Running task 138.0 in stage 16.0 (TID 472) +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO Executor: Finished task 131.0 in stage 16.0 (TID 465). 7266 bytes result sent to driver +26/04/01 08:39:03 INFO TaskSetManager: Starting task 139.0 in stage 16.0 (TID 473) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 131.0 in stage 16.0 (TID 465) in 696 ms on 10.0.0.133 (executor driver) (132/200) +26/04/01 08:39:03 INFO Executor: Running task 139.0 in stage 16.0 (TID 473) +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO Executor: Finished task 132.0 in stage 16.0 (TID 466). 7266 bytes result sent to driver +26/04/01 08:39:03 INFO TaskSetManager: Starting task 140.0 in stage 16.0 (TID 474) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 132.0 in stage 16.0 (TID 466) in 645 ms on 10.0.0.133 (executor driver) (133/200) +26/04/01 08:39:03 INFO Executor: Running task 140.0 in stage 16.0 (TID 474) +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO Executor: Finished task 133.0 in stage 16.0 (TID 467). 7223 bytes result sent to driver +26/04/01 08:39:03 INFO TaskSetManager: Starting task 141.0 in stage 16.0 (TID 475) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO Executor: Running task 141.0 in stage 16.0 (TID 475) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 133.0 in stage 16.0 (TID 467) in 695 ms on 10.0.0.133 (executor driver) (134/200) +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO Executor: Finished task 134.0 in stage 16.0 (TID 468). 7223 bytes result sent to driver +26/04/01 08:39:03 INFO TaskSetManager: Starting task 142.0 in stage 16.0 (TID 476) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO Executor: Running task 142.0 in stage 16.0 (TID 476) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 134.0 in stage 16.0 (TID 468) in 699 ms on 10.0.0.133 (executor driver) (135/200) +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO Executor: Finished task 135.0 in stage 16.0 (TID 469). 7223 bytes result sent to driver +26/04/01 08:39:03 INFO TaskSetManager: Starting task 143.0 in stage 16.0 (TID 477) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO Executor: Running task 143.0 in stage 16.0 (TID 477) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 135.0 in stage 16.0 (TID 469) in 677 ms on 10.0.0.133 (executor driver) (136/200) +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO Executor: Finished task 136.0 in stage 16.0 (TID 470). 7223 bytes result sent to driver +26/04/01 08:39:03 INFO TaskSetManager: Starting task 144.0 in stage 16.0 (TID 478) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO Executor: Running task 144.0 in stage 16.0 (TID 478) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 136.0 in stage 16.0 (TID 470) in 676 ms on 10.0.0.133 (executor driver) (137/200) +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO Executor: Finished task 137.0 in stage 16.0 (TID 471). 7223 bytes result sent to driver +26/04/01 08:39:03 INFO TaskSetManager: Starting task 145.0 in stage 16.0 (TID 479) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 137.0 in stage 16.0 (TID 471) in 677 ms on 10.0.0.133 (executor driver) (138/200) +26/04/01 08:39:03 INFO Executor: Running task 145.0 in stage 16.0 (TID 479) +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO Executor: Finished task 138.0 in stage 16.0 (TID 472). 7223 bytes result sent to driver +26/04/01 08:39:03 INFO TaskSetManager: Starting task 146.0 in stage 16.0 (TID 480) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 138.0 in stage 16.0 (TID 472) in 676 ms on 10.0.0.133 (executor driver) (139/200) +26/04/01 08:39:03 INFO Executor: Running task 146.0 in stage 16.0 (TID 480) +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO Executor: Finished task 139.0 in stage 16.0 (TID 473). 7223 bytes result sent to driver +26/04/01 08:39:03 INFO TaskSetManager: Starting task 147.0 in stage 16.0 (TID 481) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO Executor: Running task 147.0 in stage 16.0 (TID 481) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 139.0 in stage 16.0 (TID 473) in 673 ms on 10.0.0.133 (executor driver) (140/200) +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO Executor: Finished task 140.0 in stage 16.0 (TID 474). 7223 bytes result sent to driver +26/04/01 08:39:03 INFO TaskSetManager: Starting task 148.0 in stage 16.0 (TID 482) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:03 INFO TaskSetManager: Finished task 140.0 in stage 16.0 (TID 474) in 656 ms on 10.0.0.133 (executor driver) (141/200) +26/04/01 08:39:03 INFO Executor: Running task 148.0 in stage 16.0 (TID 482) +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO Executor: Finished task 141.0 in stage 16.0 (TID 475). 7266 bytes result sent to driver +26/04/01 08:39:04 INFO TaskSetManager: Starting task 149.0 in stage 16.0 (TID 483) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:04 INFO TaskSetManager: Finished task 141.0 in stage 16.0 (TID 475) in 676 ms on 10.0.0.133 (executor driver) (142/200) +26/04/01 08:39:04 INFO Executor: Running task 149.0 in stage 16.0 (TID 483) +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO Executor: Finished task 142.0 in stage 16.0 (TID 476). 7266 bytes result sent to driver +26/04/01 08:39:04 INFO TaskSetManager: Starting task 150.0 in stage 16.0 (TID 484) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:04 INFO Executor: Running task 150.0 in stage 16.0 (TID 484) +26/04/01 08:39:04 INFO TaskSetManager: Finished task 142.0 in stage 16.0 (TID 476) in 681 ms on 10.0.0.133 (executor driver) (143/200) +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO Executor: Finished task 143.0 in stage 16.0 (TID 477). 7266 bytes result sent to driver +26/04/01 08:39:04 INFO TaskSetManager: Starting task 151.0 in stage 16.0 (TID 485) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:04 INFO TaskSetManager: Finished task 143.0 in stage 16.0 (TID 477) in 677 ms on 10.0.0.133 (executor driver) (144/200) +26/04/01 08:39:04 INFO Executor: Running task 151.0 in stage 16.0 (TID 485) +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO Executor: Finished task 144.0 in stage 16.0 (TID 478). 7266 bytes result sent to driver +26/04/01 08:39:04 INFO TaskSetManager: Starting task 152.0 in stage 16.0 (TID 486) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:04 INFO TaskSetManager: Finished task 144.0 in stage 16.0 (TID 478) in 688 ms on 10.0.0.133 (executor driver) (145/200) +26/04/01 08:39:04 INFO Executor: Running task 152.0 in stage 16.0 (TID 486) +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO Executor: Finished task 145.0 in stage 16.0 (TID 479). 7266 bytes result sent to driver +26/04/01 08:39:04 INFO TaskSetManager: Starting task 153.0 in stage 16.0 (TID 487) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:04 INFO TaskSetManager: Finished task 145.0 in stage 16.0 (TID 479) in 677 ms on 10.0.0.133 (executor driver) (146/200) +26/04/01 08:39:04 INFO Executor: Running task 153.0 in stage 16.0 (TID 487) +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO Executor: Finished task 146.0 in stage 16.0 (TID 480). 7266 bytes result sent to driver +26/04/01 08:39:04 INFO TaskSetManager: Starting task 154.0 in stage 16.0 (TID 488) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:04 INFO TaskSetManager: Finished task 146.0 in stage 16.0 (TID 480) in 682 ms on 10.0.0.133 (executor driver) (147/200) +26/04/01 08:39:04 INFO Executor: Running task 154.0 in stage 16.0 (TID 488) +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO Executor: Finished task 147.0 in stage 16.0 (TID 481). 7266 bytes result sent to driver +26/04/01 08:39:04 INFO TaskSetManager: Starting task 155.0 in stage 16.0 (TID 489) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:04 INFO TaskSetManager: Finished task 147.0 in stage 16.0 (TID 481) in 667 ms on 10.0.0.133 (executor driver) (148/200) +26/04/01 08:39:04 INFO Executor: Running task 155.0 in stage 16.0 (TID 489) +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO Executor: Finished task 148.0 in stage 16.0 (TID 482). 7266 bytes result sent to driver +26/04/01 08:39:04 INFO TaskSetManager: Starting task 156.0 in stage 16.0 (TID 490) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:04 INFO Executor: Running task 156.0 in stage 16.0 (TID 490) +26/04/01 08:39:04 INFO TaskSetManager: Finished task 148.0 in stage 16.0 (TID 482) in 671 ms on 10.0.0.133 (executor driver) (149/200) +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 149.0 in stage 16.0 (TID 483). 7223 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 157.0 in stage 16.0 (TID 491) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO Executor: Running task 157.0 in stage 16.0 (TID 491) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 149.0 in stage 16.0 (TID 483) in 730 ms on 10.0.0.133 (executor driver) (150/200) +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 150.0 in stage 16.0 (TID 484). 7223 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 158.0 in stage 16.0 (TID 492) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO Executor: Running task 158.0 in stage 16.0 (TID 492) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 150.0 in stage 16.0 (TID 484) in 745 ms on 10.0.0.133 (executor driver) (151/200) +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 151.0 in stage 16.0 (TID 485). 7223 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 159.0 in stage 16.0 (TID 493) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO Executor: Running task 159.0 in stage 16.0 (TID 493) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 151.0 in stage 16.0 (TID 485) in 729 ms on 10.0.0.133 (executor driver) (152/200) +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 152.0 in stage 16.0 (TID 486). 7223 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 160.0 in stage 16.0 (TID 494) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 152.0 in stage 16.0 (TID 486) in 732 ms on 10.0.0.133 (executor driver) (153/200) +26/04/01 08:39:05 INFO Executor: Running task 160.0 in stage 16.0 (TID 494) +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 153.0 in stage 16.0 (TID 487). 7223 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 161.0 in stage 16.0 (TID 495) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 153.0 in stage 16.0 (TID 487) in 706 ms on 10.0.0.133 (executor driver) (154/200) +26/04/01 08:39:05 INFO Executor: Running task 161.0 in stage 16.0 (TID 495) +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 154.0 in stage 16.0 (TID 488). 7223 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 162.0 in stage 16.0 (TID 496) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 154.0 in stage 16.0 (TID 488) in 709 ms on 10.0.0.133 (executor driver) (155/200) +26/04/01 08:39:05 INFO Executor: Running task 162.0 in stage 16.0 (TID 496) +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 155.0 in stage 16.0 (TID 489). 7223 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 163.0 in stage 16.0 (TID 497) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 155.0 in stage 16.0 (TID 489) in 663 ms on 10.0.0.133 (executor driver) (156/200) +26/04/01 08:39:05 INFO Executor: Running task 163.0 in stage 16.0 (TID 497) +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 156.0 in stage 16.0 (TID 490). 7223 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 164.0 in stage 16.0 (TID 498) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO Executor: Running task 164.0 in stage 16.0 (TID 498) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 156.0 in stage 16.0 (TID 490) in 660 ms on 10.0.0.133 (executor driver) (157/200) +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 157.0 in stage 16.0 (TID 491). 7266 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 165.0 in stage 16.0 (TID 499) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO Executor: Running task 165.0 in stage 16.0 (TID 499) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 157.0 in stage 16.0 (TID 491) in 689 ms on 10.0.0.133 (executor driver) (158/200) +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 158.0 in stage 16.0 (TID 492). 7309 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 166.0 in stage 16.0 (TID 500) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO Executor: Running task 166.0 in stage 16.0 (TID 500) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 158.0 in stage 16.0 (TID 492) in 689 ms on 10.0.0.133 (executor driver) (159/200) +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 159.0 in stage 16.0 (TID 493). 7266 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 167.0 in stage 16.0 (TID 501) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 159.0 in stage 16.0 (TID 493) in 695 ms on 10.0.0.133 (executor driver) (160/200) +26/04/01 08:39:05 INFO Executor: Running task 167.0 in stage 16.0 (TID 501) +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 160.0 in stage 16.0 (TID 494). 7266 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 168.0 in stage 16.0 (TID 502) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 160.0 in stage 16.0 (TID 494) in 677 ms on 10.0.0.133 (executor driver) (161/200) +26/04/01 08:39:05 INFO Executor: Running task 168.0 in stage 16.0 (TID 502) +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 161.0 in stage 16.0 (TID 495). 7266 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 169.0 in stage 16.0 (TID 503) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 161.0 in stage 16.0 (TID 495) in 702 ms on 10.0.0.133 (executor driver) (162/200) +26/04/01 08:39:05 INFO Executor: Running task 169.0 in stage 16.0 (TID 503) +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 163.0 in stage 16.0 (TID 497). 7266 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 170.0 in stage 16.0 (TID 504) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO Executor: Running task 170.0 in stage 16.0 (TID 504) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 163.0 in stage 16.0 (TID 497) in 693 ms on 10.0.0.133 (executor driver) (163/200) +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 162.0 in stage 16.0 (TID 496). 7266 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 171.0 in stage 16.0 (TID 505) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO Executor: Running task 171.0 in stage 16.0 (TID 505) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 162.0 in stage 16.0 (TID 496) in 717 ms on 10.0.0.133 (executor driver) (164/200) +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:05 INFO Executor: Finished task 164.0 in stage 16.0 (TID 498). 7266 bytes result sent to driver +26/04/01 08:39:05 INFO TaskSetManager: Starting task 172.0 in stage 16.0 (TID 506) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:05 INFO Executor: Running task 172.0 in stage 16.0 (TID 506) +26/04/01 08:39:05 INFO TaskSetManager: Finished task 164.0 in stage 16.0 (TID 498) in 688 ms on 10.0.0.133 (executor driver) (165/200) +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO Executor: Finished task 165.0 in stage 16.0 (TID 499). 7223 bytes result sent to driver +26/04/01 08:39:06 INFO TaskSetManager: Starting task 173.0 in stage 16.0 (TID 507) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:06 INFO Executor: Running task 173.0 in stage 16.0 (TID 507) +26/04/01 08:39:06 INFO TaskSetManager: Finished task 165.0 in stage 16.0 (TID 499) in 746 ms on 10.0.0.133 (executor driver) (166/200) +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO Executor: Finished task 167.0 in stage 16.0 (TID 501). 7223 bytes result sent to driver +26/04/01 08:39:06 INFO TaskSetManager: Starting task 174.0 in stage 16.0 (TID 508) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:06 INFO Executor: Running task 174.0 in stage 16.0 (TID 508) +26/04/01 08:39:06 INFO TaskSetManager: Finished task 167.0 in stage 16.0 (TID 501) in 722 ms on 10.0.0.133 (executor driver) (167/200) +26/04/01 08:39:06 INFO Executor: Finished task 166.0 in stage 16.0 (TID 500). 7223 bytes result sent to driver +26/04/01 08:39:06 INFO TaskSetManager: Starting task 175.0 in stage 16.0 (TID 509) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:06 INFO Executor: Running task 175.0 in stage 16.0 (TID 509) +26/04/01 08:39:06 INFO TaskSetManager: Finished task 166.0 in stage 16.0 (TID 500) in 736 ms on 10.0.0.133 (executor driver) (168/200) +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO Executor: Finished task 168.0 in stage 16.0 (TID 502). 7223 bytes result sent to driver +26/04/01 08:39:06 INFO TaskSetManager: Starting task 176.0 in stage 16.0 (TID 510) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:06 INFO Executor: Running task 176.0 in stage 16.0 (TID 510) +26/04/01 08:39:06 INFO TaskSetManager: Finished task 168.0 in stage 16.0 (TID 502) in 721 ms on 10.0.0.133 (executor driver) (169/200) +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO Executor: Finished task 169.0 in stage 16.0 (TID 503). 7266 bytes result sent to driver +26/04/01 08:39:06 INFO TaskSetManager: Starting task 177.0 in stage 16.0 (TID 511) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:06 INFO TaskSetManager: Finished task 169.0 in stage 16.0 (TID 503) in 776 ms on 10.0.0.133 (executor driver) (170/200) +26/04/01 08:39:06 INFO Executor: Running task 177.0 in stage 16.0 (TID 511) +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO Executor: Finished task 170.0 in stage 16.0 (TID 504). 7223 bytes result sent to driver +26/04/01 08:39:06 INFO TaskSetManager: Starting task 178.0 in stage 16.0 (TID 512) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:06 INFO TaskSetManager: Finished task 170.0 in stage 16.0 (TID 504) in 776 ms on 10.0.0.133 (executor driver) (171/200) +26/04/01 08:39:06 INFO Executor: Running task 178.0 in stage 16.0 (TID 512) +26/04/01 08:39:06 INFO Executor: Finished task 171.0 in stage 16.0 (TID 505). 7223 bytes result sent to driver +26/04/01 08:39:06 INFO TaskSetManager: Starting task 179.0 in stage 16.0 (TID 513) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:06 INFO Executor: Running task 179.0 in stage 16.0 (TID 513) +26/04/01 08:39:06 INFO TaskSetManager: Finished task 171.0 in stage 16.0 (TID 505) in 774 ms on 10.0.0.133 (executor driver) (172/200) +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO Executor: Finished task 172.0 in stage 16.0 (TID 506). 7223 bytes result sent to driver +26/04/01 08:39:06 INFO TaskSetManager: Starting task 180.0 in stage 16.0 (TID 514) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:06 INFO Executor: Running task 180.0 in stage 16.0 (TID 514) +26/04/01 08:39:06 INFO TaskSetManager: Finished task 172.0 in stage 16.0 (TID 506) in 763 ms on 10.0.0.133 (executor driver) (173/200) +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO Executor: Finished task 173.0 in stage 16.0 (TID 507). 7266 bytes result sent to driver +26/04/01 08:39:07 INFO TaskSetManager: Starting task 181.0 in stage 16.0 (TID 515) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:07 INFO Executor: Running task 181.0 in stage 16.0 (TID 515) +26/04/01 08:39:07 INFO TaskSetManager: Finished task 173.0 in stage 16.0 (TID 507) in 724 ms on 10.0.0.133 (executor driver) (174/200) +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO Executor: Finished task 174.0 in stage 16.0 (TID 508). 7266 bytes result sent to driver +26/04/01 08:39:07 INFO TaskSetManager: Starting task 182.0 in stage 16.0 (TID 516) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:07 INFO Executor: Running task 182.0 in stage 16.0 (TID 516) +26/04/01 08:39:07 INFO TaskSetManager: Finished task 174.0 in stage 16.0 (TID 508) in 721 ms on 10.0.0.133 (executor driver) (175/200) +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO Executor: Finished task 176.0 in stage 16.0 (TID 510). 7266 bytes result sent to driver +26/04/01 08:39:07 INFO TaskSetManager: Starting task 183.0 in stage 16.0 (TID 517) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:07 INFO TaskSetManager: Finished task 176.0 in stage 16.0 (TID 510) in 718 ms on 10.0.0.133 (executor driver) (176/200) +26/04/01 08:39:07 INFO Executor: Running task 183.0 in stage 16.0 (TID 517) +26/04/01 08:39:07 INFO Executor: Finished task 175.0 in stage 16.0 (TID 509). 7266 bytes result sent to driver +26/04/01 08:39:07 INFO TaskSetManager: Starting task 184.0 in stage 16.0 (TID 518) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:07 INFO TaskSetManager: Finished task 175.0 in stage 16.0 (TID 509) in 727 ms on 10.0.0.133 (executor driver) (177/200) +26/04/01 08:39:07 INFO Executor: Running task 184.0 in stage 16.0 (TID 518) +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO Executor: Finished task 177.0 in stage 16.0 (TID 511). 7266 bytes result sent to driver +26/04/01 08:39:07 INFO TaskSetManager: Starting task 185.0 in stage 16.0 (TID 519) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:07 INFO Executor: Running task 185.0 in stage 16.0 (TID 519) +26/04/01 08:39:07 INFO TaskSetManager: Finished task 177.0 in stage 16.0 (TID 511) in 691 ms on 10.0.0.133 (executor driver) (178/200) +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO Executor: Finished task 178.0 in stage 16.0 (TID 512). 7309 bytes result sent to driver +26/04/01 08:39:07 INFO TaskSetManager: Starting task 186.0 in stage 16.0 (TID 520) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:07 INFO Executor: Running task 186.0 in stage 16.0 (TID 520) +26/04/01 08:39:07 INFO TaskSetManager: Finished task 178.0 in stage 16.0 (TID 512) in 695 ms on 10.0.0.133 (executor driver) (179/200) +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO Executor: Finished task 179.0 in stage 16.0 (TID 513). 7266 bytes result sent to driver +26/04/01 08:39:07 INFO TaskSetManager: Starting task 187.0 in stage 16.0 (TID 521) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:07 INFO Executor: Running task 187.0 in stage 16.0 (TID 521) +26/04/01 08:39:07 INFO TaskSetManager: Finished task 179.0 in stage 16.0 (TID 513) in 696 ms on 10.0.0.133 (executor driver) (180/200) +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO Executor: Finished task 180.0 in stage 16.0 (TID 514). 7266 bytes result sent to driver +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO TaskSetManager: Starting task 188.0 in stage 16.0 (TID 522) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:07 INFO Executor: Running task 188.0 in stage 16.0 (TID 522) +26/04/01 08:39:07 INFO TaskSetManager: Finished task 180.0 in stage 16.0 (TID 514) in 693 ms on 10.0.0.133 (executor driver) (181/200) +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (30.2 KiB) non-empty blocks including 8 (30.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO Executor: Finished task 181.0 in stage 16.0 (TID 515). 7223 bytes result sent to driver +26/04/01 08:39:07 INFO TaskSetManager: Starting task 189.0 in stage 16.0 (TID 523) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:07 INFO Executor: Running task 189.0 in stage 16.0 (TID 523) +26/04/01 08:39:07 INFO TaskSetManager: Finished task 181.0 in stage 16.0 (TID 515) in 671 ms on 10.0.0.133 (executor driver) (182/200) +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO Executor: Finished task 182.0 in stage 16.0 (TID 516). 7223 bytes result sent to driver +26/04/01 08:39:07 INFO TaskSetManager: Starting task 190.0 in stage 16.0 (TID 524) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:07 INFO Executor: Running task 190.0 in stage 16.0 (TID 524) +26/04/01 08:39:07 INFO TaskSetManager: Finished task 182.0 in stage 16.0 (TID 516) in 681 ms on 10.0.0.133 (executor driver) (183/200) +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO Executor: Finished task 183.0 in stage 16.0 (TID 517). 7223 bytes result sent to driver +26/04/01 08:39:08 INFO TaskSetManager: Starting task 191.0 in stage 16.0 (TID 525) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:08 INFO Executor: Running task 191.0 in stage 16.0 (TID 525) +26/04/01 08:39:08 INFO TaskSetManager: Finished task 183.0 in stage 16.0 (TID 517) in 686 ms on 10.0.0.133 (executor driver) (184/200) +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO Executor: Finished task 184.0 in stage 16.0 (TID 518). 7266 bytes result sent to driver +26/04/01 08:39:08 INFO TaskSetManager: Starting task 192.0 in stage 16.0 (TID 526) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:08 INFO TaskSetManager: Finished task 184.0 in stage 16.0 (TID 518) in 692 ms on 10.0.0.133 (executor driver) (185/200) +26/04/01 08:39:08 INFO Executor: Running task 192.0 in stage 16.0 (TID 526) +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (28.1 KiB) non-empty blocks including 8 (28.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO Executor: Finished task 185.0 in stage 16.0 (TID 519). 7266 bytes result sent to driver +26/04/01 08:39:08 INFO TaskSetManager: Starting task 193.0 in stage 16.0 (TID 527) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:08 INFO Executor: Running task 193.0 in stage 16.0 (TID 527) +26/04/01 08:39:08 INFO TaskSetManager: Finished task 185.0 in stage 16.0 (TID 519) in 707 ms on 10.0.0.133 (executor driver) (186/200) +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO Executor: Finished task 186.0 in stage 16.0 (TID 520). 7223 bytes result sent to driver +26/04/01 08:39:08 INFO TaskSetManager: Starting task 194.0 in stage 16.0 (TID 528) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:08 INFO Executor: Running task 194.0 in stage 16.0 (TID 528) +26/04/01 08:39:08 INFO TaskSetManager: Finished task 186.0 in stage 16.0 (TID 520) in 708 ms on 10.0.0.133 (executor driver) (187/200) +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO Executor: Finished task 187.0 in stage 16.0 (TID 521). 7223 bytes result sent to driver +26/04/01 08:39:08 INFO TaskSetManager: Starting task 195.0 in stage 16.0 (TID 529) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:08 INFO TaskSetManager: Finished task 187.0 in stage 16.0 (TID 521) in 718 ms on 10.0.0.133 (executor driver) (188/200) +26/04/01 08:39:08 INFO Executor: Running task 195.0 in stage 16.0 (TID 529) +26/04/01 08:39:08 INFO Executor: Finished task 188.0 in stage 16.0 (TID 522). 7223 bytes result sent to driver +26/04/01 08:39:08 INFO TaskSetManager: Starting task 196.0 in stage 16.0 (TID 530) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:08 INFO Executor: Running task 196.0 in stage 16.0 (TID 530) +26/04/01 08:39:08 INFO TaskSetManager: Finished task 188.0 in stage 16.0 (TID 522) in 716 ms on 10.0.0.133 (executor driver) (189/200) +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO Executor: Finished task 189.0 in stage 16.0 (TID 523). 7266 bytes result sent to driver +26/04/01 08:39:08 INFO TaskSetManager: Starting task 197.0 in stage 16.0 (TID 531) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:08 INFO Executor: Running task 197.0 in stage 16.0 (TID 531) +26/04/01 08:39:08 INFO TaskSetManager: Finished task 189.0 in stage 16.0 (TID 523) in 701 ms on 10.0.0.133 (executor driver) (190/200) +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO Executor: Finished task 190.0 in stage 16.0 (TID 524). 7266 bytes result sent to driver +26/04/01 08:39:08 INFO TaskSetManager: Starting task 198.0 in stage 16.0 (TID 532) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:08 INFO Executor: Running task 198.0 in stage 16.0 (TID 532) +26/04/01 08:39:08 INFO TaskSetManager: Finished task 190.0 in stage 16.0 (TID 524) in 700 ms on 10.0.0.133 (executor driver) (191/200) +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO Executor: Finished task 191.0 in stage 16.0 (TID 525). 7266 bytes result sent to driver +26/04/01 08:39:08 INFO TaskSetManager: Starting task 199.0 in stage 16.0 (TID 533) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:08 INFO TaskSetManager: Finished task 191.0 in stage 16.0 (TID 525) in 699 ms on 10.0.0.133 (executor driver) (192/200) +26/04/01 08:39:08 INFO Executor: Running task 199.0 in stage 16.0 (TID 533) +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:08 INFO Executor: Finished task 192.0 in stage 16.0 (TID 526). 7266 bytes result sent to driver +26/04/01 08:39:08 INFO TaskSetManager: Finished task 192.0 in stage 16.0 (TID 526) in 695 ms on 10.0.0.133 (executor driver) (193/200) +26/04/01 08:39:08 INFO Executor: Finished task 193.0 in stage 16.0 (TID 527). 7266 bytes result sent to driver +26/04/01 08:39:08 INFO TaskSetManager: Finished task 193.0 in stage 16.0 (TID 527) in 668 ms on 10.0.0.133 (executor driver) (194/200) +26/04/01 08:39:08 INFO Executor: Finished task 194.0 in stage 16.0 (TID 528). 7266 bytes result sent to driver +26/04/01 08:39:08 INFO TaskSetManager: Finished task 194.0 in stage 16.0 (TID 528) in 694 ms on 10.0.0.133 (executor driver) (195/200) +26/04/01 08:39:08 INFO Executor: Finished task 195.0 in stage 16.0 (TID 529). 7266 bytes result sent to driver +26/04/01 08:39:08 INFO TaskSetManager: Finished task 195.0 in stage 16.0 (TID 529) in 696 ms on 10.0.0.133 (executor driver) (196/200) +26/04/01 08:39:08 INFO Executor: Finished task 196.0 in stage 16.0 (TID 530). 7266 bytes result sent to driver +26/04/01 08:39:08 INFO TaskSetManager: Finished task 196.0 in stage 16.0 (TID 530) in 698 ms on 10.0.0.133 (executor driver) (197/200) +26/04/01 08:39:09 INFO Executor: Finished task 197.0 in stage 16.0 (TID 531). 7223 bytes result sent to driver +26/04/01 08:39:09 INFO TaskSetManager: Finished task 197.0 in stage 16.0 (TID 531) in 779 ms on 10.0.0.133 (executor driver) (198/200) +26/04/01 08:39:09 INFO Executor: Finished task 198.0 in stage 16.0 (TID 532). 7223 bytes result sent to driver +26/04/01 08:39:09 INFO TaskSetManager: Finished task 198.0 in stage 16.0 (TID 532) in 774 ms on 10.0.0.133 (executor driver) (199/200) +26/04/01 08:39:09 INFO Executor: Finished task 199.0 in stage 16.0 (TID 533). 7223 bytes result sent to driver +26/04/01 08:39:09 INFO TaskSetManager: Finished task 199.0 in stage 16.0 (TID 533) in 783 ms on 10.0.0.133 (executor driver) (200/200) +26/04/01 08:39:09 INFO TaskSchedulerImpl: Removed TaskSet 16.0, whose tasks have all completed, from pool +26/04/01 08:39:09 INFO DAGScheduler: ShuffleMapStage 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 28.434 s +26/04/01 08:39:09 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:39:09 INFO DAGScheduler: running: Set() +26/04/01 08:39:09 INFO DAGScheduler: waiting: Set() +26/04/01 08:39:09 INFO DAGScheduler: failed: Set() +26/04/01 08:39:09 INFO ShufflePartitionsUtil: For shuffle(5, 2), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 08:39:09 INFO CodeGenerator: Code generated in 4.300083 ms +26/04/01 08:39:09 INFO CodeGenerator: Code generated in 2.088959 ms +26/04/01 08:39:09 INFO CodeGenerator: Code generated in 2.233375 ms +26/04/01 08:39:09 INFO DAGScheduler: Registering RDD 53 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 6 +26/04/01 08:39:09 INFO DAGScheduler: Got map stage job 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 13 output partitions +26/04/01 08:39:09 INFO DAGScheduler: Final stage: ShuffleMapStage 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:39:09 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 19, ShuffleMapStage 20) +26/04/01 08:39:09 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:09 INFO DAGScheduler: Submitting ShuffleMapStage 21 (MapPartitionsRDD[53] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:39:09 INFO MemoryStore: Block broadcast_22 stored as values in memory (estimated size 65.8 KiB, free 8.6 GiB) +26/04/01 08:39:09 INFO MemoryStore: Block broadcast_22_piece0 stored as bytes in memory (estimated size 30.0 KiB, free 8.6 GiB) +26/04/01 08:39:09 INFO BlockManagerInfo: Added broadcast_22_piece0 in memory on 10.0.0.133:59288 (size: 30.0 KiB, free: 8.6 GiB) +26/04/01 08:39:09 INFO SparkContext: Created broadcast 22 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:09 INFO DAGScheduler: Submitting 13 missing tasks from ShuffleMapStage 21 (MapPartitionsRDD[53] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) +26/04/01 08:39:09 INFO TaskSchedulerImpl: Adding task set 21.0 with 13 tasks resource profile 0 +26/04/01 08:39:09 INFO TaskSetManager: Starting task 0.0 in stage 21.0 (TID 534) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:09 INFO TaskSetManager: Starting task 1.0 in stage 21.0 (TID 535) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:09 INFO TaskSetManager: Starting task 2.0 in stage 21.0 (TID 536) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:09 INFO TaskSetManager: Starting task 3.0 in stage 21.0 (TID 537) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:09 INFO TaskSetManager: Starting task 4.0 in stage 21.0 (TID 538) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:09 INFO TaskSetManager: Starting task 5.0 in stage 21.0 (TID 539) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:09 INFO TaskSetManager: Starting task 6.0 in stage 21.0 (TID 540) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:09 INFO TaskSetManager: Starting task 7.0 in stage 21.0 (TID 541) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:09 INFO Executor: Running task 1.0 in stage 21.0 (TID 535) +26/04/01 08:39:09 INFO Executor: Running task 0.0 in stage 21.0 (TID 534) +26/04/01 08:39:09 INFO Executor: Running task 3.0 in stage 21.0 (TID 537) +26/04/01 08:39:09 INFO Executor: Running task 7.0 in stage 21.0 (TID 541) +26/04/01 08:39:09 INFO Executor: Running task 5.0 in stage 21.0 (TID 539) +26/04/01 08:39:09 INFO Executor: Running task 6.0 in stage 21.0 (TID 540) +26/04/01 08:39:09 INFO Executor: Running task 4.0 in stage 21.0 (TID 538) +26/04/01 08:39:09 INFO Executor: Running task 2.0 in stage 21.0 (TID 536) +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (59.6 MiB) non-empty blocks including 200 (59.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (59.7 MiB) non-empty blocks including 200 (59.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (59.5 MiB) non-empty blocks including 200 (59.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (63.0 MiB) non-empty blocks including 200 (63.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (59.5 MiB) non-empty blocks including 200 (59.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (62.9 MiB) non-empty blocks including 200 (62.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (63.1 MiB) non-empty blocks including 200 (63.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms +26/04/01 08:39:09 INFO CodeGenerator: Code generated in 2.935459 ms +26/04/01 08:39:09 INFO CodeGenerator: Code generated in 1.423291 ms +26/04/01 08:39:09 INFO CodeGenerator: Code generated in 1.563958 ms +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (699.8 KiB) non-empty blocks including 8 (699.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (741.9 KiB) non-empty blocks including 8 (741.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (743.3 KiB) non-empty blocks including 8 (743.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (698.8 KiB) non-empty blocks including 8 (698.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (704.9 KiB) non-empty blocks including 8 (704.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (701.5 KiB) non-empty blocks including 8 (701.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (741.5 KiB) non-empty blocks including 8 (741.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (743.9 KiB) non-empty blocks including 8 (743.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:09 INFO CodeGenerator: Code generated in 2.077167 ms +26/04/01 08:39:09 INFO CodeGenerator: Code generated in 1.392583 ms +26/04/01 08:39:09 INFO CodeGenerator: Code generated in 1.329041 ms +26/04/01 08:39:09 INFO CodeGenerator: Code generated in 3.886667 ms +26/04/01 08:39:09 INFO CodeGenerator: Code generated in 2.125125 ms +26/04/01 08:39:12 INFO Executor: Finished task 6.0 in stage 21.0 (TID 540). 10893 bytes result sent to driver +26/04/01 08:39:12 INFO TaskSetManager: Starting task 8.0 in stage 21.0 (TID 542) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:12 INFO Executor: Running task 8.0 in stage 21.0 (TID 542) +26/04/01 08:39:12 INFO TaskSetManager: Finished task 6.0 in stage 21.0 (TID 540) in 3084 ms on 10.0.0.133 (executor driver) (1/13) +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 200 (59.6 MiB) non-empty blocks including 200 (59.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 8 (703.8 KiB) non-empty blocks including 8 (703.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:12 INFO Executor: Finished task 3.0 in stage 21.0 (TID 537). 10893 bytes result sent to driver +26/04/01 08:39:12 INFO TaskSetManager: Starting task 9.0 in stage 21.0 (TID 543) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:12 INFO Executor: Running task 9.0 in stage 21.0 (TID 543) +26/04/01 08:39:12 INFO TaskSetManager: Finished task 3.0 in stage 21.0 (TID 537) in 3349 ms on 10.0.0.133 (executor driver) (2/13) +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 200 (63.3 MiB) non-empty blocks including 200 (63.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 8 (742.1 KiB) non-empty blocks including 8 (742.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:12 INFO Executor: Finished task 0.0 in stage 21.0 (TID 534). 10893 bytes result sent to driver +26/04/01 08:39:12 INFO TaskSetManager: Starting task 10.0 in stage 21.0 (TID 544) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:12 INFO TaskSetManager: Finished task 0.0 in stage 21.0 (TID 534) in 3413 ms on 10.0.0.133 (executor driver) (3/13) +26/04/01 08:39:12 INFO Executor: Running task 10.0 in stage 21.0 (TID 544) +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 200 (59.4 MiB) non-empty blocks including 200 (59.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 8 (699.9 KiB) non-empty blocks including 8 (699.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:12 INFO Executor: Finished task 5.0 in stage 21.0 (TID 539). 10893 bytes result sent to driver +26/04/01 08:39:12 INFO TaskSetManager: Starting task 11.0 in stage 21.0 (TID 545) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:12 INFO TaskSetManager: Finished task 5.0 in stage 21.0 (TID 539) in 3447 ms on 10.0.0.133 (executor driver) (4/13) +26/04/01 08:39:12 INFO Executor: Running task 11.0 in stage 21.0 (TID 545) +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 200 (59.5 MiB) non-empty blocks including 200 (59.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 8 (696.0 KiB) non-empty blocks including 8 (696.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:12 INFO Executor: Finished task 1.0 in stage 21.0 (TID 535). 10893 bytes result sent to driver +26/04/01 08:39:12 INFO TaskSetManager: Starting task 12.0 in stage 21.0 (TID 546) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:12 INFO Executor: Running task 12.0 in stage 21.0 (TID 546) +26/04/01 08:39:12 INFO TaskSetManager: Finished task 1.0 in stage 21.0 (TID 535) in 3461 ms on 10.0.0.133 (executor driver) (5/13) +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 200 (59.5 MiB) non-empty blocks including 200 (59.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 8 (699.3 KiB) non-empty blocks including 8 (699.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:13 INFO Executor: Finished task 2.0 in stage 21.0 (TID 536). 10893 bytes result sent to driver +26/04/01 08:39:13 INFO TaskSetManager: Finished task 2.0 in stage 21.0 (TID 536) in 3472 ms on 10.0.0.133 (executor driver) (6/13) +26/04/01 08:39:13 INFO Executor: Finished task 4.0 in stage 21.0 (TID 538). 10893 bytes result sent to driver +26/04/01 08:39:13 INFO TaskSetManager: Finished task 4.0 in stage 21.0 (TID 538) in 3474 ms on 10.0.0.133 (executor driver) (7/13) +26/04/01 08:39:13 INFO Executor: Finished task 7.0 in stage 21.0 (TID 541). 10893 bytes result sent to driver +26/04/01 08:39:13 INFO TaskSetManager: Finished task 7.0 in stage 21.0 (TID 541) in 3474 ms on 10.0.0.133 (executor driver) (8/13) +26/04/01 08:39:14 INFO Executor: Finished task 8.0 in stage 21.0 (TID 542). 10893 bytes result sent to driver +26/04/01 08:39:14 INFO TaskSetManager: Finished task 8.0 in stage 21.0 (TID 542) in 1539 ms on 10.0.0.133 (executor driver) (9/13) +26/04/01 08:39:14 INFO Executor: Finished task 11.0 in stage 21.0 (TID 545). 10850 bytes result sent to driver +26/04/01 08:39:14 INFO TaskSetManager: Finished task 11.0 in stage 21.0 (TID 545) in 1601 ms on 10.0.0.133 (executor driver) (10/13) +26/04/01 08:39:14 INFO Executor: Finished task 10.0 in stage 21.0 (TID 544). 10893 bytes result sent to driver +26/04/01 08:39:14 INFO TaskSetManager: Finished task 10.0 in stage 21.0 (TID 544) in 1654 ms on 10.0.0.133 (executor driver) (11/13) +26/04/01 08:39:14 INFO Executor: Finished task 9.0 in stage 21.0 (TID 543). 10893 bytes result sent to driver +26/04/01 08:39:14 INFO TaskSetManager: Finished task 9.0 in stage 21.0 (TID 543) in 1767 ms on 10.0.0.133 (executor driver) (12/13) +26/04/01 08:39:14 INFO Executor: Finished task 12.0 in stage 21.0 (TID 546). 10893 bytes result sent to driver +26/04/01 08:39:14 INFO TaskSetManager: Finished task 12.0 in stage 21.0 (TID 546) in 1661 ms on 10.0.0.133 (executor driver) (13/13) +26/04/01 08:39:14 INFO TaskSchedulerImpl: Removed TaskSet 21.0, whose tasks have all completed, from pool +26/04/01 08:39:14 INFO DAGScheduler: ShuffleMapStage 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 5.122 s +26/04/01 08:39:14 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:39:14 INFO DAGScheduler: running: Set() +26/04/01 08:39:14 INFO DAGScheduler: waiting: Set() +26/04/01 08:39:14 INFO DAGScheduler: failed: Set() +26/04/01 08:39:14 INFO ShufflePartitionsUtil: For shuffle(6, 3), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 08:39:14 INFO CodeGenerator: Code generated in 3.866792 ms +26/04/01 08:39:14 INFO CodeGenerator: Code generated in 2.012292 ms +26/04/01 08:39:14 INFO CodeGenerator: Code generated in 1.779375 ms +26/04/01 08:39:14 INFO DAGScheduler: Registering RDD 60 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 7 +26/04/01 08:39:14 INFO DAGScheduler: Got map stage job 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 34 output partitions +26/04/01 08:39:14 INFO DAGScheduler: Final stage: ShuffleMapStage 28 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:39:14 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 27, ShuffleMapStage 26) +26/04/01 08:39:14 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:14 INFO DAGScheduler: Submitting ShuffleMapStage 28 (MapPartitionsRDD[60] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:39:14 INFO MemoryStore: Block broadcast_23 stored as values in memory (estimated size 77.9 KiB, free 8.6 GiB) +26/04/01 08:39:14 INFO MemoryStore: Block broadcast_23_piece0 stored as bytes in memory (estimated size 33.7 KiB, free 8.6 GiB) +26/04/01 08:39:14 INFO BlockManagerInfo: Added broadcast_23_piece0 in memory on 10.0.0.133:59288 (size: 33.7 KiB, free: 8.6 GiB) +26/04/01 08:39:14 INFO SparkContext: Created broadcast 23 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:14 INFO DAGScheduler: Submitting 34 missing tasks from ShuffleMapStage 28 (MapPartitionsRDD[60] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:39:14 INFO TaskSchedulerImpl: Adding task set 28.0 with 34 tasks resource profile 0 +26/04/01 08:39:14 INFO TaskSetManager: Starting task 0.0 in stage 28.0 (TID 547) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:14 INFO TaskSetManager: Starting task 1.0 in stage 28.0 (TID 548) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:14 INFO TaskSetManager: Starting task 2.0 in stage 28.0 (TID 549) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:14 INFO TaskSetManager: Starting task 3.0 in stage 28.0 (TID 550) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:14 INFO TaskSetManager: Starting task 4.0 in stage 28.0 (TID 551) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:14 INFO TaskSetManager: Starting task 5.0 in stage 28.0 (TID 552) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:14 INFO TaskSetManager: Starting task 6.0 in stage 28.0 (TID 553) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:14 INFO TaskSetManager: Starting task 7.0 in stage 28.0 (TID 554) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:14 INFO Executor: Running task 2.0 in stage 28.0 (TID 549) +26/04/01 08:39:14 INFO Executor: Running task 5.0 in stage 28.0 (TID 552) +26/04/01 08:39:14 INFO Executor: Running task 3.0 in stage 28.0 (TID 550) +26/04/01 08:39:14 INFO Executor: Running task 1.0 in stage 28.0 (TID 548) +26/04/01 08:39:14 INFO Executor: Running task 6.0 in stage 28.0 (TID 553) +26/04/01 08:39:14 INFO Executor: Running task 4.0 in stage 28.0 (TID 551) +26/04/01 08:39:14 INFO Executor: Running task 0.0 in stage 28.0 (TID 547) +26/04/01 08:39:14 INFO Executor: Running task 7.0 in stage 28.0 (TID 554) +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO CodeGenerator: Code generated in 2.429166 ms +26/04/01 08:39:14 INFO CodeGenerator: Code generated in 1.740458 ms +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:14 INFO CodeGenerator: Code generated in 2.482916 ms +26/04/01 08:39:14 INFO CodeGenerator: Code generated in 1.349833 ms +26/04/01 08:39:14 INFO CodeGenerator: Code generated in 3.366125 ms +26/04/01 08:39:15 INFO BlockManagerInfo: Removed broadcast_22_piece0 on 10.0.0.133:59288 in memory (size: 30.0 KiB, free: 8.6 GiB) +26/04/01 08:39:16 INFO Executor: Finished task 0.0 in stage 28.0 (TID 547). 14477 bytes result sent to driver +26/04/01 08:39:16 INFO TaskSetManager: Starting task 8.0 in stage 28.0 (TID 555) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:16 INFO TaskSetManager: Finished task 0.0 in stage 28.0 (TID 547) in 1938 ms on 10.0.0.133 (executor driver) (1/34) +26/04/01 08:39:16 INFO Executor: Running task 8.0 in stage 28.0 (TID 555) +26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:16 INFO Executor: Finished task 5.0 in stage 28.0 (TID 552). 14477 bytes result sent to driver +26/04/01 08:39:16 INFO TaskSetManager: Starting task 9.0 in stage 28.0 (TID 556) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:16 INFO Executor: Running task 9.0 in stage 28.0 (TID 556) +26/04/01 08:39:16 INFO TaskSetManager: Finished task 5.0 in stage 28.0 (TID 552) in 2307 ms on 10.0.0.133 (executor driver) (2/34) +26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Getting 13 (24.9 MiB) non-empty blocks including 13 (24.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:16 INFO Executor: Finished task 2.0 in stage 28.0 (TID 549). 14477 bytes result sent to driver +26/04/01 08:39:16 INFO TaskSetManager: Starting task 10.0 in stage 28.0 (TID 557) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:16 INFO TaskSetManager: Finished task 2.0 in stage 28.0 (TID 549) in 2314 ms on 10.0.0.133 (executor driver) (3/34) +26/04/01 08:39:16 INFO Executor: Running task 10.0 in stage 28.0 (TID 557) +26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:17 INFO Executor: Finished task 3.0 in stage 28.0 (TID 550). 14477 bytes result sent to driver +26/04/01 08:39:17 INFO TaskSetManager: Starting task 11.0 in stage 28.0 (TID 558) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:17 INFO Executor: Running task 11.0 in stage 28.0 (TID 558) +26/04/01 08:39:17 INFO TaskSetManager: Finished task 3.0 in stage 28.0 (TID 550) in 2329 ms on 10.0.0.133 (executor driver) (4/34) +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 13 (25.0 MiB) non-empty blocks including 13 (25.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:17 INFO Executor: Finished task 4.0 in stage 28.0 (TID 551). 14477 bytes result sent to driver +26/04/01 08:39:17 INFO TaskSetManager: Starting task 12.0 in stage 28.0 (TID 559) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:17 INFO TaskSetManager: Finished task 4.0 in stage 28.0 (TID 551) in 2345 ms on 10.0.0.133 (executor driver) (5/34) +26/04/01 08:39:17 INFO Executor: Running task 12.0 in stage 28.0 (TID 559) +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:17 INFO Executor: Finished task 7.0 in stage 28.0 (TID 554). 14477 bytes result sent to driver +26/04/01 08:39:17 INFO TaskSetManager: Starting task 13.0 in stage 28.0 (TID 560) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:17 INFO Executor: Running task 13.0 in stage 28.0 (TID 560) +26/04/01 08:39:17 INFO TaskSetManager: Finished task 7.0 in stage 28.0 (TID 554) in 2348 ms on 10.0.0.133 (executor driver) (6/34) +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 13 (25.4 MiB) non-empty blocks including 13 (25.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:17 INFO Executor: Finished task 1.0 in stage 28.0 (TID 548). 14477 bytes result sent to driver +26/04/01 08:39:17 INFO TaskSetManager: Starting task 14.0 in stage 28.0 (TID 561) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:17 INFO Executor: Running task 14.0 in stage 28.0 (TID 561) +26/04/01 08:39:17 INFO TaskSetManager: Finished task 1.0 in stage 28.0 (TID 548) in 2354 ms on 10.0.0.133 (executor driver) (7/34) +26/04/01 08:39:17 INFO Executor: Finished task 6.0 in stage 28.0 (TID 553). 14477 bytes result sent to driver +26/04/01 08:39:17 INFO TaskSetManager: Starting task 15.0 in stage 28.0 (TID 562) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:17 INFO TaskSetManager: Finished task 6.0 in stage 28.0 (TID 553) in 2355 ms on 10.0.0.133 (executor driver) (8/34) +26/04/01 08:39:17 INFO Executor: Running task 15.0 in stage 28.0 (TID 562) +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:17 INFO Executor: Finished task 8.0 in stage 28.0 (TID 555). 14434 bytes result sent to driver +26/04/01 08:39:17 INFO TaskSetManager: Starting task 16.0 in stage 28.0 (TID 563) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:17 INFO TaskSetManager: Finished task 8.0 in stage 28.0 (TID 555) in 1211 ms on 10.0.0.133 (executor driver) (9/34) +26/04/01 08:39:17 INFO Executor: Running task 16.0 in stage 28.0 (TID 563) +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:18 INFO Executor: Finished task 10.0 in stage 28.0 (TID 557). 14477 bytes result sent to driver +26/04/01 08:39:18 INFO TaskSetManager: Starting task 17.0 in stage 28.0 (TID 564) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:18 INFO TaskSetManager: Finished task 10.0 in stage 28.0 (TID 557) in 1303 ms on 10.0.0.133 (executor driver) (10/34) +26/04/01 08:39:18 INFO Executor: Running task 17.0 in stage 28.0 (TID 564) +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:18 INFO Executor: Finished task 9.0 in stage 28.0 (TID 556). 14477 bytes result sent to driver +26/04/01 08:39:18 INFO TaskSetManager: Starting task 18.0 in stage 28.0 (TID 565) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:18 INFO Executor: Running task 18.0 in stage 28.0 (TID 565) +26/04/01 08:39:18 INFO TaskSetManager: Finished task 9.0 in stage 28.0 (TID 556) in 1426 ms on 10.0.0.133 (executor driver) (11/34) +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:18 INFO Executor: Finished task 12.0 in stage 28.0 (TID 559). 14477 bytes result sent to driver +26/04/01 08:39:18 INFO TaskSetManager: Starting task 19.0 in stage 28.0 (TID 566) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:18 INFO Executor: Running task 19.0 in stage 28.0 (TID 566) +26/04/01 08:39:18 INFO TaskSetManager: Finished task 12.0 in stage 28.0 (TID 559) in 1448 ms on 10.0.0.133 (executor driver) (12/34) +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:18 INFO Executor: Finished task 11.0 in stage 28.0 (TID 558). 14477 bytes result sent to driver +26/04/01 08:39:18 INFO TaskSetManager: Starting task 20.0 in stage 28.0 (TID 567) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:18 INFO Executor: Running task 20.0 in stage 28.0 (TID 567) +26/04/01 08:39:18 INFO TaskSetManager: Finished task 11.0 in stage 28.0 (TID 558) in 1478 ms on 10.0.0.133 (executor driver) (13/34) +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:18 INFO Executor: Finished task 13.0 in stage 28.0 (TID 560). 14434 bytes result sent to driver +26/04/01 08:39:18 INFO TaskSetManager: Starting task 21.0 in stage 28.0 (TID 568) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:18 INFO Executor: Running task 21.0 in stage 28.0 (TID 568) +26/04/01 08:39:18 INFO TaskSetManager: Finished task 13.0 in stage 28.0 (TID 560) in 1508 ms on 10.0.0.133 (executor driver) (14/34) +26/04/01 08:39:18 INFO Executor: Finished task 14.0 in stage 28.0 (TID 561). 14434 bytes result sent to driver +26/04/01 08:39:18 INFO TaskSetManager: Starting task 22.0 in stage 28.0 (TID 569) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:18 INFO TaskSetManager: Finished task 14.0 in stage 28.0 (TID 561) in 1502 ms on 10.0.0.133 (executor driver) (15/34) +26/04/01 08:39:18 INFO Executor: Running task 22.0 in stage 28.0 (TID 569) +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:18 INFO Executor: Finished task 15.0 in stage 28.0 (TID 562). 14434 bytes result sent to driver +26/04/01 08:39:18 INFO TaskSetManager: Starting task 23.0 in stage 28.0 (TID 570) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:18 INFO TaskSetManager: Finished task 15.0 in stage 28.0 (TID 562) in 1505 ms on 10.0.0.133 (executor driver) (16/34) +26/04/01 08:39:18 INFO Executor: Running task 23.0 in stage 28.0 (TID 570) +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:19 INFO Executor: Finished task 16.0 in stage 28.0 (TID 563). 14477 bytes result sent to driver +26/04/01 08:39:19 INFO TaskSetManager: Starting task 24.0 in stage 28.0 (TID 571) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:19 INFO Executor: Running task 24.0 in stage 28.0 (TID 571) +26/04/01 08:39:19 INFO TaskSetManager: Finished task 16.0 in stage 28.0 (TID 563) in 1216 ms on 10.0.0.133 (executor driver) (17/34) +26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:19 INFO Executor: Finished task 17.0 in stage 28.0 (TID 564). 14477 bytes result sent to driver +26/04/01 08:39:19 INFO TaskSetManager: Starting task 25.0 in stage 28.0 (TID 572) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:19 INFO Executor: Running task 25.0 in stage 28.0 (TID 572) +26/04/01 08:39:19 INFO TaskSetManager: Finished task 17.0 in stage 28.0 (TID 564) in 1481 ms on 10.0.0.133 (executor driver) (18/34) +26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:19 INFO Executor: Finished task 18.0 in stage 28.0 (TID 565). 14477 bytes result sent to driver +26/04/01 08:39:19 INFO TaskSetManager: Starting task 26.0 in stage 28.0 (TID 573) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:19 INFO Executor: Running task 26.0 in stage 28.0 (TID 573) +26/04/01 08:39:19 INFO TaskSetManager: Finished task 18.0 in stage 28.0 (TID 565) in 1386 ms on 10.0.0.133 (executor driver) (19/34) +26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:20 INFO Executor: Finished task 20.0 in stage 28.0 (TID 567). 14477 bytes result sent to driver +26/04/01 08:39:20 INFO TaskSetManager: Starting task 27.0 in stage 28.0 (TID 574) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:20 INFO Executor: Running task 27.0 in stage 28.0 (TID 574) +26/04/01 08:39:20 INFO TaskSetManager: Finished task 20.0 in stage 28.0 (TID 567) in 1600 ms on 10.0.0.133 (executor driver) (20/34) +26/04/01 08:39:20 INFO Executor: Finished task 19.0 in stage 28.0 (TID 566). 14477 bytes result sent to driver +26/04/01 08:39:20 INFO TaskSetManager: Starting task 28.0 in stage 28.0 (TID 575) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:20 INFO TaskSetManager: Finished task 19.0 in stage 28.0 (TID 566) in 1613 ms on 10.0.0.133 (executor driver) (21/34) +26/04/01 08:39:20 INFO Executor: Running task 28.0 in stage 28.0 (TID 575) +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 13 (25.0 MiB) non-empty blocks including 13 (25.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:20 INFO Executor: Finished task 21.0 in stage 28.0 (TID 568). 14477 bytes result sent to driver +26/04/01 08:39:20 INFO TaskSetManager: Starting task 29.0 in stage 28.0 (TID 576) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:20 INFO TaskSetManager: Finished task 21.0 in stage 28.0 (TID 568) in 1625 ms on 10.0.0.133 (executor driver) (22/34) +26/04/01 08:39:20 INFO Executor: Running task 29.0 in stage 28.0 (TID 576) +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:20 INFO Executor: Finished task 22.0 in stage 28.0 (TID 569). 14477 bytes result sent to driver +26/04/01 08:39:20 INFO TaskSetManager: Starting task 30.0 in stage 28.0 (TID 577) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:20 INFO TaskSetManager: Finished task 22.0 in stage 28.0 (TID 569) in 1632 ms on 10.0.0.133 (executor driver) (23/34) +26/04/01 08:39:20 INFO Executor: Running task 30.0 in stage 28.0 (TID 577) +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:20 INFO Executor: Finished task 23.0 in stage 28.0 (TID 570). 14434 bytes result sent to driver +26/04/01 08:39:20 INFO TaskSetManager: Starting task 31.0 in stage 28.0 (TID 578) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:20 INFO Executor: Running task 31.0 in stage 28.0 (TID 578) +26/04/01 08:39:20 INFO TaskSetManager: Finished task 23.0 in stage 28.0 (TID 570) in 1638 ms on 10.0.0.133 (executor driver) (24/34) +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:20 INFO Executor: Finished task 24.0 in stage 28.0 (TID 571). 14477 bytes result sent to driver +26/04/01 08:39:20 INFO TaskSetManager: Starting task 32.0 in stage 28.0 (TID 579) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:20 INFO Executor: Running task 32.0 in stage 28.0 (TID 579) +26/04/01 08:39:20 INFO TaskSetManager: Finished task 24.0 in stage 28.0 (TID 571) in 1344 ms on 10.0.0.133 (executor driver) (25/34) +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO Executor: Finished task 26.0 in stage 28.0 (TID 573). 14477 bytes result sent to driver +26/04/01 08:39:21 INFO TaskSetManager: Starting task 33.0 in stage 28.0 (TID 580) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:21 INFO TaskSetManager: Finished task 26.0 in stage 28.0 (TID 573) in 1497 ms on 10.0.0.133 (executor driver) (26/34) +26/04/01 08:39:21 INFO Executor: Running task 33.0 in stage 28.0 (TID 580) +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 13 (8.5 MiB) non-empty blocks including 13 (8.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 37 (12.3 MiB) non-empty blocks including 37 (12.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO Executor: Finished task 25.0 in stage 28.0 (TID 572). 14477 bytes result sent to driver +26/04/01 08:39:21 INFO TaskSetManager: Finished task 25.0 in stage 28.0 (TID 572) in 1541 ms on 10.0.0.133 (executor driver) (27/34) +26/04/01 08:39:21 INFO Executor: Finished task 27.0 in stage 28.0 (TID 574). 14477 bytes result sent to driver +26/04/01 08:39:21 INFO TaskSetManager: Finished task 27.0 in stage 28.0 (TID 574) in 1476 ms on 10.0.0.133 (executor driver) (28/34) +26/04/01 08:39:21 INFO Executor: Finished task 28.0 in stage 28.0 (TID 575). 14477 bytes result sent to driver +26/04/01 08:39:21 INFO TaskSetManager: Finished task 28.0 in stage 28.0 (TID 575) in 1523 ms on 10.0.0.133 (executor driver) (29/34) +26/04/01 08:39:21 INFO Executor: Finished task 29.0 in stage 28.0 (TID 576). 14477 bytes result sent to driver +26/04/01 08:39:21 INFO TaskSetManager: Finished task 29.0 in stage 28.0 (TID 576) in 1486 ms on 10.0.0.133 (executor driver) (30/34) +26/04/01 08:39:21 INFO Executor: Finished task 30.0 in stage 28.0 (TID 577). 14477 bytes result sent to driver +26/04/01 08:39:21 INFO TaskSetManager: Finished task 30.0 in stage 28.0 (TID 577) in 1572 ms on 10.0.0.133 (executor driver) (31/34) +26/04/01 08:39:21 INFO Executor: Finished task 31.0 in stage 28.0 (TID 578). 14434 bytes result sent to driver +26/04/01 08:39:21 INFO TaskSetManager: Finished task 31.0 in stage 28.0 (TID 578) in 1575 ms on 10.0.0.133 (executor driver) (32/34) +26/04/01 08:39:21 INFO Executor: Finished task 33.0 in stage 28.0 (TID 580). 14477 bytes result sent to driver +26/04/01 08:39:21 INFO TaskSetManager: Finished task 33.0 in stage 28.0 (TID 580) in 474 ms on 10.0.0.133 (executor driver) (33/34) +26/04/01 08:39:21 INFO Executor: Finished task 32.0 in stage 28.0 (TID 579). 14477 bytes result sent to driver +26/04/01 08:39:21 INFO TaskSetManager: Finished task 32.0 in stage 28.0 (TID 579) in 1407 ms on 10.0.0.133 (executor driver) (34/34) +26/04/01 08:39:21 INFO TaskSchedulerImpl: Removed TaskSet 28.0, whose tasks have all completed, from pool +26/04/01 08:39:21 INFO DAGScheduler: ShuffleMapStage 28 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 7.116 s +26/04/01 08:39:21 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:39:21 INFO DAGScheduler: running: Set() +26/04/01 08:39:21 INFO DAGScheduler: waiting: Set() +26/04/01 08:39:21 INFO DAGScheduler: failed: Set() +26/04/01 08:39:21 INFO ShufflePartitionsUtil: For shuffle(7, 4), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 +26/04/01 08:39:21 INFO HashAggregateExec: spark.sql.codegen.aggregate.map.twolevel.enabled is set to true, but current version of codegened fast hashmap does not support this aggregate. +26/04/01 08:39:21 INFO CodeGenerator: Code generated in 11.426584 ms +26/04/01 08:39:21 INFO CodeGenerator: Code generated in 1.968209 ms +26/04/01 08:39:21 INFO CodeGenerator: Code generated in 1.643583 ms +26/04/01 08:39:21 INFO DAGScheduler: Registering RDD 67 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 8 +26/04/01 08:39:21 INFO DAGScheduler: Got map stage job 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 40 output partitions +26/04/01 08:39:21 INFO DAGScheduler: Final stage: ShuffleMapStage 37 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:39:21 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 35, ShuffleMapStage 36) +26/04/01 08:39:21 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:21 INFO DAGScheduler: Submitting ShuffleMapStage 37 (MapPartitionsRDD[67] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:39:21 INFO MemoryStore: Block broadcast_24 stored as values in memory (estimated size 110.1 KiB, free 8.6 GiB) +26/04/01 08:39:21 INFO MemoryStore: Block broadcast_24_piece0 stored as bytes in memory (estimated size 46.7 KiB, free 8.6 GiB) +26/04/01 08:39:21 INFO BlockManagerInfo: Added broadcast_24_piece0 in memory on 10.0.0.133:59288 (size: 46.7 KiB, free: 8.6 GiB) +26/04/01 08:39:21 INFO SparkContext: Created broadcast 24 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:21 INFO DAGScheduler: Submitting 40 missing tasks from ShuffleMapStage 37 (MapPartitionsRDD[67] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) +26/04/01 08:39:21 INFO TaskSchedulerImpl: Adding task set 37.0 with 40 tasks resource profile 0 +26/04/01 08:39:21 INFO TaskSetManager: Starting task 0.0 in stage 37.0 (TID 581) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:21 INFO TaskSetManager: Starting task 1.0 in stage 37.0 (TID 582) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:21 INFO TaskSetManager: Starting task 2.0 in stage 37.0 (TID 583) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:21 INFO TaskSetManager: Starting task 3.0 in stage 37.0 (TID 584) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:21 INFO TaskSetManager: Starting task 4.0 in stage 37.0 (TID 585) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:21 INFO TaskSetManager: Starting task 5.0 in stage 37.0 (TID 586) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:21 INFO TaskSetManager: Starting task 6.0 in stage 37.0 (TID 587) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:21 INFO TaskSetManager: Starting task 7.0 in stage 37.0 (TID 588) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:21 INFO Executor: Running task 2.0 in stage 37.0 (TID 583) +26/04/01 08:39:21 INFO Executor: Running task 0.0 in stage 37.0 (TID 581) +26/04/01 08:39:21 INFO Executor: Running task 1.0 in stage 37.0 (TID 582) +26/04/01 08:39:21 INFO Executor: Running task 4.0 in stage 37.0 (TID 585) +26/04/01 08:39:21 INFO Executor: Running task 5.0 in stage 37.0 (TID 586) +26/04/01 08:39:21 INFO Executor: Running task 6.0 in stage 37.0 (TID 587) +26/04/01 08:39:21 INFO Executor: Running task 7.0 in stage 37.0 (TID 588) +26/04/01 08:39:21 INFO Executor: Running task 3.0 in stage 37.0 (TID 584) +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO CodeGenerator: Code generated in 2.273916 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.5 MiB) non-empty blocks including 64 (36.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:21 INFO CodeGenerator: Code generated in 2.775625 ms +26/04/01 08:39:21 INFO CodeGenerator: Code generated in 6.171875 ms +26/04/01 08:39:21 INFO CodeGenerator: Code generated in 1.881667 ms +26/04/01 08:39:21 INFO CodeGenerator: Code generated in 0.8755 ms +26/04/01 08:39:21 INFO CodeGenerator: Code generated in 1.532458 ms +26/04/01 08:39:21 INFO CodeGenerator: Code generated in 2.958333 ms +26/04/01 08:39:22 INFO BlockManagerInfo: Removed broadcast_23_piece0 on 10.0.0.133:59288 in memory (size: 33.7 KiB, free: 8.6 GiB) +26/04/01 08:39:22 INFO Executor: Finished task 4.0 in stage 37.0 (TID 585). 19388 bytes result sent to driver +26/04/01 08:39:22 INFO TaskSetManager: Starting task 8.0 in stage 37.0 (TID 589) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:22 INFO Executor: Running task 8.0 in stage 37.0 (TID 589) +26/04/01 08:39:22 INFO TaskSetManager: Finished task 4.0 in stage 37.0 (TID 585) in 1051 ms on 10.0.0.133 (executor driver) (1/40) +26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:22 INFO Executor: Finished task 1.0 in stage 37.0 (TID 582). 19388 bytes result sent to driver +26/04/01 08:39:22 INFO TaskSetManager: Starting task 9.0 in stage 37.0 (TID 590) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:22 INFO Executor: Finished task 6.0 in stage 37.0 (TID 587). 19388 bytes result sent to driver +26/04/01 08:39:22 INFO Executor: Running task 9.0 in stage 37.0 (TID 590) +26/04/01 08:39:22 INFO TaskSetManager: Starting task 10.0 in stage 37.0 (TID 591) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:22 INFO TaskSetManager: Finished task 1.0 in stage 37.0 (TID 582) in 1137 ms on 10.0.0.133 (executor driver) (2/40) +26/04/01 08:39:22 INFO Executor: Running task 10.0 in stage 37.0 (TID 591) +26/04/01 08:39:22 INFO TaskSetManager: Finished task 6.0 in stage 37.0 (TID 587) in 1137 ms on 10.0.0.133 (executor driver) (3/40) +26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO Executor: Finished task 5.0 in stage 37.0 (TID 586). 19388 bytes result sent to driver +26/04/01 08:39:23 INFO TaskSetManager: Starting task 11.0 in stage 37.0 (TID 592) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:23 INFO TaskSetManager: Finished task 5.0 in stage 37.0 (TID 586) in 1213 ms on 10.0.0.133 (executor driver) (4/40) +26/04/01 08:39:23 INFO Executor: Running task 11.0 in stage 37.0 (TID 592) +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO Executor: Finished task 0.0 in stage 37.0 (TID 581). 19388 bytes result sent to driver +26/04/01 08:39:23 INFO TaskSetManager: Starting task 12.0 in stage 37.0 (TID 593) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:23 INFO Executor: Running task 12.0 in stage 37.0 (TID 593) +26/04/01 08:39:23 INFO TaskSetManager: Finished task 0.0 in stage 37.0 (TID 581) in 1314 ms on 10.0.0.133 (executor driver) (5/40) +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO Executor: Finished task 3.0 in stage 37.0 (TID 584). 19388 bytes result sent to driver +26/04/01 08:39:23 INFO TaskSetManager: Starting task 13.0 in stage 37.0 (TID 594) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:23 INFO Executor: Running task 13.0 in stage 37.0 (TID 594) +26/04/01 08:39:23 INFO TaskSetManager: Finished task 3.0 in stage 37.0 (TID 584) in 1358 ms on 10.0.0.133 (executor driver) (6/40) +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO Executor: Finished task 2.0 in stage 37.0 (TID 583). 19388 bytes result sent to driver +26/04/01 08:39:23 INFO TaskSetManager: Starting task 14.0 in stage 37.0 (TID 595) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:23 INFO TaskSetManager: Finished task 2.0 in stage 37.0 (TID 583) in 1412 ms on 10.0.0.133 (executor driver) (7/40) +26/04/01 08:39:23 INFO Executor: Running task 14.0 in stage 37.0 (TID 595) +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO Executor: Finished task 7.0 in stage 37.0 (TID 588). 19388 bytes result sent to driver +26/04/01 08:39:23 INFO TaskSetManager: Starting task 15.0 in stage 37.0 (TID 596) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:23 INFO TaskSetManager: Finished task 7.0 in stage 37.0 (TID 588) in 1420 ms on 10.0.0.133 (executor driver) (8/40) +26/04/01 08:39:23 INFO Executor: Running task 15.0 in stage 37.0 (TID 596) +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO Executor: Finished task 8.0 in stage 37.0 (TID 589). 19388 bytes result sent to driver +26/04/01 08:39:23 INFO TaskSetManager: Starting task 16.0 in stage 37.0 (TID 597) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:23 INFO Executor: Running task 16.0 in stage 37.0 (TID 597) +26/04/01 08:39:23 INFO TaskSetManager: Finished task 8.0 in stage 37.0 (TID 589) in 843 ms on 10.0.0.133 (executor driver) (9/40) +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO Executor: Finished task 9.0 in stage 37.0 (TID 590). 19388 bytes result sent to driver +26/04/01 08:39:23 INFO TaskSetManager: Starting task 17.0 in stage 37.0 (TID 598) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:23 INFO Executor: Running task 17.0 in stage 37.0 (TID 598) +26/04/01 08:39:23 INFO TaskSetManager: Finished task 9.0 in stage 37.0 (TID 590) in 842 ms on 10.0.0.133 (executor driver) (10/40) +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO Executor: Finished task 10.0 in stage 37.0 (TID 591). 19388 bytes result sent to driver +26/04/01 08:39:23 INFO TaskSetManager: Starting task 18.0 in stage 37.0 (TID 599) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:23 INFO Executor: Running task 18.0 in stage 37.0 (TID 599) +26/04/01 08:39:23 INFO TaskSetManager: Finished task 10.0 in stage 37.0 (TID 591) in 864 ms on 10.0.0.133 (executor driver) (11/40) +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO Executor: Finished task 11.0 in stage 37.0 (TID 592). 19388 bytes result sent to driver +26/04/01 08:39:23 INFO TaskSetManager: Starting task 19.0 in stage 37.0 (TID 600) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:23 INFO Executor: Running task 19.0 in stage 37.0 (TID 600) +26/04/01 08:39:23 INFO TaskSetManager: Finished task 11.0 in stage 37.0 (TID 592) in 840 ms on 10.0.0.133 (executor driver) (12/40) +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO Executor: Finished task 12.0 in stage 37.0 (TID 593). 19388 bytes result sent to driver +26/04/01 08:39:24 INFO TaskSetManager: Starting task 20.0 in stage 37.0 (TID 601) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:24 INFO Executor: Running task 20.0 in stage 37.0 (TID 601) +26/04/01 08:39:24 INFO TaskSetManager: Finished task 12.0 in stage 37.0 (TID 593) in 903 ms on 10.0.0.133 (executor driver) (13/40) +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO Executor: Finished task 13.0 in stage 37.0 (TID 594). 19388 bytes result sent to driver +26/04/01 08:39:24 INFO TaskSetManager: Starting task 21.0 in stage 37.0 (TID 602) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:24 INFO TaskSetManager: Finished task 13.0 in stage 37.0 (TID 594) in 875 ms on 10.0.0.133 (executor driver) (14/40) +26/04/01 08:39:24 INFO Executor: Running task 21.0 in stage 37.0 (TID 602) +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO Executor: Finished task 14.0 in stage 37.0 (TID 595). 19388 bytes result sent to driver +26/04/01 08:39:24 INFO TaskSetManager: Starting task 22.0 in stage 37.0 (TID 603) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:24 INFO Executor: Running task 22.0 in stage 37.0 (TID 603) +26/04/01 08:39:24 INFO TaskSetManager: Finished task 14.0 in stage 37.0 (TID 595) in 827 ms on 10.0.0.133 (executor driver) (15/40) +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO Executor: Finished task 15.0 in stage 37.0 (TID 596). 19388 bytes result sent to driver +26/04/01 08:39:24 INFO TaskSetManager: Starting task 23.0 in stage 37.0 (TID 604) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:24 INFO Executor: Running task 23.0 in stage 37.0 (TID 604) +26/04/01 08:39:24 INFO TaskSetManager: Finished task 15.0 in stage 37.0 (TID 596) in 842 ms on 10.0.0.133 (executor driver) (16/40) +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO Executor: Finished task 16.0 in stage 37.0 (TID 597). 19388 bytes result sent to driver +26/04/01 08:39:24 INFO TaskSetManager: Starting task 24.0 in stage 37.0 (TID 605) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:24 INFO TaskSetManager: Finished task 16.0 in stage 37.0 (TID 597) in 830 ms on 10.0.0.133 (executor driver) (17/40) +26/04/01 08:39:24 INFO Executor: Running task 24.0 in stage 37.0 (TID 605) +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO Executor: Finished task 17.0 in stage 37.0 (TID 598). 19388 bytes result sent to driver +26/04/01 08:39:24 INFO TaskSetManager: Starting task 25.0 in stage 37.0 (TID 606) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:24 INFO Executor: Running task 25.0 in stage 37.0 (TID 606) +26/04/01 08:39:24 INFO TaskSetManager: Finished task 17.0 in stage 37.0 (TID 598) in 802 ms on 10.0.0.133 (executor driver) (18/40) +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO Executor: Finished task 18.0 in stage 37.0 (TID 599). 19388 bytes result sent to driver +26/04/01 08:39:24 INFO TaskSetManager: Starting task 26.0 in stage 37.0 (TID 607) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:24 INFO Executor: Running task 26.0 in stage 37.0 (TID 607) +26/04/01 08:39:24 INFO TaskSetManager: Finished task 18.0 in stage 37.0 (TID 599) in 808 ms on 10.0.0.133 (executor driver) (19/40) +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO Executor: Finished task 19.0 in stage 37.0 (TID 600). 19388 bytes result sent to driver +26/04/01 08:39:24 INFO TaskSetManager: Starting task 27.0 in stage 37.0 (TID 608) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:24 INFO Executor: Running task 27.0 in stage 37.0 (TID 608) +26/04/01 08:39:24 INFO TaskSetManager: Finished task 19.0 in stage 37.0 (TID 600) in 802 ms on 10.0.0.133 (executor driver) (20/40) +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO Executor: Finished task 20.0 in stage 37.0 (TID 601). 19388 bytes result sent to driver +26/04/01 08:39:24 INFO TaskSetManager: Starting task 28.0 in stage 37.0 (TID 609) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:24 INFO TaskSetManager: Finished task 20.0 in stage 37.0 (TID 601) in 825 ms on 10.0.0.133 (executor driver) (21/40) +26/04/01 08:39:24 INFO Executor: Running task 28.0 in stage 37.0 (TID 609) +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO Executor: Finished task 21.0 in stage 37.0 (TID 602). 19388 bytes result sent to driver +26/04/01 08:39:24 INFO TaskSetManager: Starting task 29.0 in stage 37.0 (TID 610) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:24 INFO Executor: Running task 29.0 in stage 37.0 (TID 610) +26/04/01 08:39:24 INFO TaskSetManager: Finished task 21.0 in stage 37.0 (TID 602) in 820 ms on 10.0.0.133 (executor driver) (22/40) +26/04/01 08:39:24 INFO Executor: Finished task 22.0 in stage 37.0 (TID 603). 19388 bytes result sent to driver +26/04/01 08:39:24 INFO TaskSetManager: Starting task 30.0 in stage 37.0 (TID 611) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:24 INFO Executor: Running task 30.0 in stage 37.0 (TID 611) +26/04/01 08:39:24 INFO TaskSetManager: Finished task 22.0 in stage 37.0 (TID 603) in 815 ms on 10.0.0.133 (executor driver) (23/40) +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO Executor: Finished task 23.0 in stage 37.0 (TID 604). 19388 bytes result sent to driver +26/04/01 08:39:24 INFO TaskSetManager: Starting task 31.0 in stage 37.0 (TID 612) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:24 INFO TaskSetManager: Finished task 23.0 in stage 37.0 (TID 604) in 816 ms on 10.0.0.133 (executor driver) (24/40) +26/04/01 08:39:24 INFO Executor: Running task 31.0 in stage 37.0 (TID 612) +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO Executor: Finished task 24.0 in stage 37.0 (TID 605). 19388 bytes result sent to driver +26/04/01 08:39:25 INFO TaskSetManager: Starting task 32.0 in stage 37.0 (TID 613) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:25 INFO TaskSetManager: Finished task 24.0 in stage 37.0 (TID 605) in 807 ms on 10.0.0.133 (executor driver) (25/40) +26/04/01 08:39:25 INFO Executor: Running task 32.0 in stage 37.0 (TID 613) +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO Executor: Finished task 25.0 in stage 37.0 (TID 606). 19388 bytes result sent to driver +26/04/01 08:39:25 INFO TaskSetManager: Starting task 33.0 in stage 37.0 (TID 614) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:25 INFO Executor: Running task 33.0 in stage 37.0 (TID 614) +26/04/01 08:39:25 INFO TaskSetManager: Finished task 25.0 in stage 37.0 (TID 606) in 815 ms on 10.0.0.133 (executor driver) (26/40) +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO Executor: Finished task 26.0 in stage 37.0 (TID 607). 19388 bytes result sent to driver +26/04/01 08:39:25 INFO TaskSetManager: Starting task 34.0 in stage 37.0 (TID 615) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:25 INFO Executor: Running task 34.0 in stage 37.0 (TID 615) +26/04/01 08:39:25 INFO TaskSetManager: Finished task 26.0 in stage 37.0 (TID 607) in 801 ms on 10.0.0.133 (executor driver) (27/40) +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.5 MiB) non-empty blocks including 64 (36.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO Executor: Finished task 27.0 in stage 37.0 (TID 608). 19388 bytes result sent to driver +26/04/01 08:39:25 INFO TaskSetManager: Starting task 35.0 in stage 37.0 (TID 616) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:25 INFO Executor: Running task 35.0 in stage 37.0 (TID 616) +26/04/01 08:39:25 INFO TaskSetManager: Finished task 27.0 in stage 37.0 (TID 608) in 819 ms on 10.0.0.133 (executor driver) (28/40) +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO Executor: Finished task 29.0 in stage 37.0 (TID 610). 19388 bytes result sent to driver +26/04/01 08:39:25 INFO TaskSetManager: Starting task 36.0 in stage 37.0 (TID 617) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:25 INFO TaskSetManager: Finished task 29.0 in stage 37.0 (TID 610) in 818 ms on 10.0.0.133 (executor driver) (29/40) +26/04/01 08:39:25 INFO Executor: Running task 36.0 in stage 37.0 (TID 617) +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO Executor: Finished task 28.0 in stage 37.0 (TID 609). 19388 bytes result sent to driver +26/04/01 08:39:25 INFO TaskSetManager: Starting task 37.0 in stage 37.0 (TID 618) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:25 INFO Executor: Running task 37.0 in stage 37.0 (TID 618) +26/04/01 08:39:25 INFO TaskSetManager: Finished task 28.0 in stage 37.0 (TID 609) in 832 ms on 10.0.0.133 (executor driver) (30/40) +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO Executor: Finished task 30.0 in stage 37.0 (TID 611). 19388 bytes result sent to driver +26/04/01 08:39:25 INFO TaskSetManager: Starting task 38.0 in stage 37.0 (TID 619) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:25 INFO Executor: Running task 38.0 in stage 37.0 (TID 619) +26/04/01 08:39:25 INFO TaskSetManager: Finished task 30.0 in stage 37.0 (TID 611) in 823 ms on 10.0.0.133 (executor driver) (31/40) +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.5 MiB) non-empty blocks including 64 (36.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO Executor: Finished task 31.0 in stage 37.0 (TID 612). 19388 bytes result sent to driver +26/04/01 08:39:25 INFO TaskSetManager: Starting task 39.0 in stage 37.0 (TID 620) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9270 bytes) +26/04/01 08:39:25 INFO Executor: Running task 39.0 in stage 37.0 (TID 620) +26/04/01 08:39:25 INFO TaskSetManager: Finished task 31.0 in stage 37.0 (TID 612) in 803 ms on 10.0.0.133 (executor driver) (32/40) +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:26 INFO Executor: Finished task 32.0 in stage 37.0 (TID 613). 19388 bytes result sent to driver +26/04/01 08:39:26 INFO TaskSetManager: Finished task 32.0 in stage 37.0 (TID 613) in 788 ms on 10.0.0.133 (executor driver) (33/40) +26/04/01 08:39:26 INFO Executor: Finished task 34.0 in stage 37.0 (TID 615). 19388 bytes result sent to driver +26/04/01 08:39:26 INFO TaskSetManager: Finished task 34.0 in stage 37.0 (TID 615) in 788 ms on 10.0.0.133 (executor driver) (34/40) +26/04/01 08:39:26 INFO Executor: Finished task 33.0 in stage 37.0 (TID 614). 19388 bytes result sent to driver +26/04/01 08:39:26 INFO TaskSetManager: Finished task 33.0 in stage 37.0 (TID 614) in 833 ms on 10.0.0.133 (executor driver) (35/40) +26/04/01 08:39:26 INFO Executor: Finished task 35.0 in stage 37.0 (TID 616). 19388 bytes result sent to driver +26/04/01 08:39:26 INFO TaskSetManager: Finished task 35.0 in stage 37.0 (TID 616) in 828 ms on 10.0.0.133 (executor driver) (36/40) +26/04/01 08:39:26 INFO Executor: Finished task 37.0 in stage 37.0 (TID 618). 19388 bytes result sent to driver +26/04/01 08:39:26 INFO TaskSetManager: Finished task 37.0 in stage 37.0 (TID 618) in 799 ms on 10.0.0.133 (executor driver) (37/40) +26/04/01 08:39:26 INFO Executor: Finished task 36.0 in stage 37.0 (TID 617). 19388 bytes result sent to driver +26/04/01 08:39:26 INFO TaskSetManager: Finished task 36.0 in stage 37.0 (TID 617) in 809 ms on 10.0.0.133 (executor driver) (38/40) +26/04/01 08:39:26 INFO Executor: Finished task 39.0 in stage 37.0 (TID 620). 19388 bytes result sent to driver +26/04/01 08:39:26 INFO TaskSetManager: Finished task 39.0 in stage 37.0 (TID 620) in 805 ms on 10.0.0.133 (executor driver) (39/40) +26/04/01 08:39:26 INFO Executor: Finished task 38.0 in stage 37.0 (TID 619). 19388 bytes result sent to driver +26/04/01 08:39:26 INFO TaskSetManager: Finished task 38.0 in stage 37.0 (TID 619) in 813 ms on 10.0.0.133 (executor driver) (40/40) +26/04/01 08:39:26 INFO TaskSchedulerImpl: Removed TaskSet 37.0, whose tasks have all completed, from pool +26/04/01 08:39:26 INFO DAGScheduler: ShuffleMapStage 37 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 4.693 s +26/04/01 08:39:26 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:39:26 INFO DAGScheduler: running: Set() +26/04/01 08:39:26 INFO DAGScheduler: waiting: Set() +26/04/01 08:39:26 INFO DAGScheduler: failed: Set() +26/04/01 08:39:26 INFO ShufflePartitionsUtil: For shuffle(8), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 08:39:26 INFO HashAggregateExec: spark.sql.codegen.aggregate.map.twolevel.enabled is set to true, but current version of codegened fast hashmap does not support this aggregate. +26/04/01 08:39:26 INFO CodeGenerator: Code generated in 4.282291 ms +26/04/01 08:39:26 INFO CodeGenerator: Code generated in 1.394541 ms +26/04/01 08:39:26 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:39:26 INFO DAGScheduler: Got job 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 08:39:26 INFO DAGScheduler: Final stage: ResultStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:39:26 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 46) +26/04/01 08:39:26 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:26 INFO DAGScheduler: Submitting ResultStage 47 (MapPartitionsRDD[72] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:39:26 INFO MemoryStore: Block broadcast_25 stored as values in memory (estimated size 107.4 KiB, free 8.6 GiB) +26/04/01 08:39:26 INFO MemoryStore: Block broadcast_25_piece0 stored as bytes in memory (estimated size 44.1 KiB, free 8.6 GiB) +26/04/01 08:39:26 INFO BlockManagerInfo: Added broadcast_25_piece0 in memory on 10.0.0.133:59288 (size: 44.1 KiB, free: 8.6 GiB) +26/04/01 08:39:26 INFO SparkContext: Created broadcast 25 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:26 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 47 (MapPartitionsRDD[72] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:39:26 INFO TaskSchedulerImpl: Adding task set 47.0 with 1 tasks resource profile 0 +26/04/01 08:39:26 INFO TaskSetManager: Starting task 0.0 in stage 47.0 (TID 621) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8999 bytes) +26/04/01 08:39:26 INFO Executor: Running task 0.0 in stage 47.0 (TID 621) +26/04/01 08:39:26 INFO ShuffleBlockFetcherIterator: Getting 40 (595.2 KiB) non-empty blocks including 40 (595.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:26 INFO CodeGenerator: Code generated in 3.784167 ms +26/04/01 08:39:26 INFO Executor: Finished task 0.0 in stage 47.0 (TID 621). 29688 bytes result sent to driver +26/04/01 08:39:26 INFO TaskSetManager: Finished task 0.0 in stage 47.0 (TID 621) in 28 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:39:26 INFO TaskSchedulerImpl: Removed TaskSet 47.0, whose tasks have all completed, from pool +26/04/01 08:39:26 INFO DAGScheduler: ResultStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.031 s +26/04/01 08:39:26 INFO DAGScheduler: Job 18 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:39:26 INFO TaskSchedulerImpl: Killing all running tasks in stage 47: Stage finished +26/04/01 08:39:26 INFO DAGScheduler: Job 18 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.033149 s +26/04/01 08:39:26 INFO DAGScheduler: Registering RDD 73 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 9 +26/04/01 08:39:26 INFO DAGScheduler: Got map stage job 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 08:39:26 INFO DAGScheduler: Final stage: ShuffleMapStage 57 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:39:26 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 56) +26/04/01 08:39:26 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:26 INFO DAGScheduler: Submitting ShuffleMapStage 57 (MapPartitionsRDD[73] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:39:26 INFO MemoryStore: Block broadcast_26 stored as values in memory (estimated size 116.3 KiB, free 8.6 GiB) +26/04/01 08:39:26 INFO MemoryStore: Block broadcast_26_piece0 stored as bytes in memory (estimated size 45.9 KiB, free 8.6 GiB) +26/04/01 08:39:26 INFO BlockManagerInfo: Added broadcast_26_piece0 in memory on 10.0.0.133:59288 (size: 45.9 KiB, free: 8.6 GiB) +26/04/01 08:39:26 INFO SparkContext: Created broadcast 26 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:26 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 57 (MapPartitionsRDD[73] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:39:26 INFO TaskSchedulerImpl: Adding task set 57.0 with 1 tasks resource profile 0 +26/04/01 08:39:26 INFO TaskSetManager: Starting task 0.0 in stage 57.0 (TID 622) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8988 bytes) +26/04/01 08:39:26 INFO Executor: Running task 0.0 in stage 57.0 (TID 622) +26/04/01 08:39:26 INFO CodeGenerator: Code generated in 1.754625 ms +26/04/01 08:39:26 INFO ShuffleBlockFetcherIterator: Getting 40 (595.2 KiB) non-empty blocks including 40 (595.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:26 INFO Executor: Finished task 0.0 in stage 57.0 (TID 622). 20944 bytes result sent to driver +26/04/01 08:39:26 INFO TaskSetManager: Finished task 0.0 in stage 57.0 (TID 622) in 52 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:39:26 INFO TaskSchedulerImpl: Removed TaskSet 57.0, whose tasks have all completed, from pool +26/04/01 08:39:26 INFO DAGScheduler: ShuffleMapStage 57 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.057 s +26/04/01 08:39:26 INFO DAGScheduler: looking for newly runnable stages +26/04/01 08:39:26 INFO DAGScheduler: running: Set() +26/04/01 08:39:26 INFO DAGScheduler: waiting: Set() +26/04/01 08:39:26 INFO DAGScheduler: failed: Set() +26/04/01 08:39:26 INFO ShufflePartitionsUtil: For shuffle(9), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 +26/04/01 08:39:26 INFO CodeGenerator: Code generated in 2.3705 ms +26/04/01 08:39:26 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 +26/04/01 08:39:26 INFO DAGScheduler: Got job 20 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions +26/04/01 08:39:26 INFO DAGScheduler: Final stage: ResultStage 68 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) +26/04/01 08:39:26 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 67) +26/04/01 08:39:26 INFO DAGScheduler: Missing parents: List() +26/04/01 08:39:26 INFO DAGScheduler: Submitting ResultStage 68 (MapPartitionsRDD[76] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents +26/04/01 08:39:26 INFO MemoryStore: Block broadcast_27 stored as values in memory (estimated size 102.4 KiB, free 8.6 GiB) +26/04/01 08:39:26 INFO MemoryStore: Block broadcast_27_piece0 stored as bytes in memory (estimated size 42.7 KiB, free 8.6 GiB) +26/04/01 08:39:26 INFO BlockManagerInfo: Added broadcast_27_piece0 in memory on 10.0.0.133:59288 (size: 42.7 KiB, free: 8.6 GiB) +26/04/01 08:39:26 INFO SparkContext: Created broadcast 27 from broadcast at DAGScheduler.scala:1611 +26/04/01 08:39:26 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 68 (MapPartitionsRDD[76] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) +26/04/01 08:39:26 INFO TaskSchedulerImpl: Adding task set 68.0 with 1 tasks resource profile 0 +26/04/01 08:39:26 INFO TaskSetManager: Starting task 0.0 in stage 68.0 (TID 623) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8999 bytes) +26/04/01 08:39:26 INFO Executor: Running task 0.0 in stage 68.0 (TID 623) +26/04/01 08:39:26 INFO ShuffleBlockFetcherIterator: Getting 1 (17.3 KiB) non-empty blocks including 1 (17.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks +26/04/01 08:39:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms +26/04/01 08:39:26 INFO CodeGenerator: Code generated in 1.805 ms +26/04/01 08:39:26 INFO CodeGenerator: Code generated in 2.202917 ms +26/04/01 08:39:26 INFO Executor: Finished task 0.0 in stage 68.0 (TID 623). 25767 bytes result sent to driver +26/04/01 08:39:26 INFO TaskSetManager: Finished task 0.0 in stage 68.0 (TID 623) in 9 ms on 10.0.0.133 (executor driver) (1/1) +26/04/01 08:39:26 INFO TaskSchedulerImpl: Removed TaskSet 68.0, whose tasks have all completed, from pool +26/04/01 08:39:26 INFO DAGScheduler: ResultStage 68 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.012 s +26/04/01 08:39:26 INFO DAGScheduler: Job 20 is finished. Cancelling potential speculative or zombie tasks for this job +26/04/01 08:39:26 INFO TaskSchedulerImpl: Killing all running tasks in stage 68: Stage finished +26/04/01 08:39:26 INFO DAGScheduler: Job 20 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.013881 s +26/04/01 08:39:26 INFO SparkContext: SparkContext is stopping with exitCode 0. +26/04/01 08:39:26 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! +26/04/01 08:39:29 INFO MemoryStore: MemoryStore cleared +26/04/01 08:39:29 INFO BlockManager: BlockManager stopped +26/04/01 08:39:29 INFO BlockManagerMaster: BlockManagerMaster stopped +26/04/01 08:39:29 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! +26/04/01 08:39:29 INFO SparkContext: Successfully stopped SparkContext +26/04/01 08:39:29 INFO ShutdownHookManager: Shutdown hook called +26/04/01 08:39:29 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-d176a20a-d434-4356-bca6-cae43ef66498 +26/04/01 08:39:29 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-d176a20a-d434-4356-bca6-cae43ef66498/pyspark-15bb5836-3e4b-4658-b524-ea35cdb6f558 +26/04/01 08:39:29 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-0b85ad85-88e5-4003-afbe-ce40c48d1d47 + 86.21 real 432.85 user 104.97 sys + 8887762944 maximum resident set size + 0 average shared memory size + 0 average unshared data size + 0 average unshared stack size + 3508536 page reclaims + 48 page faults + 0 swaps + 0 block input operations + 0 block output operations + 1016 messages sent + 1034 messages received + 251 signals received + 114132 voluntary context switches + 1088013 involuntary context switches + 5685978480737 instructions retired + 1928666289311 cycles elapsed + 8867438272 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/results.csv b/benchmarks/tpc/memory-profile-results/results.csv new file mode 100644 index 0000000000..e0bc63b1e0 --- /dev/null +++ b/benchmarks/tpc/memory-profile-results/results.csv @@ -0,0 +1,4 @@ +engine,offheap_size,query,peak_rss_mb,wall_time_sec,exit_code +spark,4g,9,8476,86.21,0 +comet,4g,9,8525,384.09,0 +comet,8g,9,8448,384.01,0 diff --git a/docs/superpowers/plans/2026-04-01-spill-callback.md b/docs/superpowers/plans/2026-04-01-spill-callback.md new file mode 100644 index 0000000000..f0770a5229 --- /dev/null +++ b/docs/superpowers/plans/2026-04-01-spill-callback.md @@ -0,0 +1,782 @@ +# Spill Callback 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:** Enable Spark to reclaim memory from Comet's native operators via the `spill()` callback, so cross-task memory eviction works and Comet can run at lower off-heap memory settings. + +**Architecture:** Add a shared `SpillState` (atomics for pressure/freed) that bridges `NativeMemoryConsumer.spill()` on the JVM side with `CometUnifiedMemoryPool.try_grow()` on the Rust side. When Spark calls `spill()`, it JNI-calls into native to set spill pressure. The pool's `try_grow()` checks this flag and returns `ResourcesExhausted`, causing DataFusion's Sort/Aggregate/Shuffle operators to spill internally. As operators release memory via `shrink()`, the freed bytes are tracked and returned to Spark. + +**Tech Stack:** Rust (native memory pool), Java (JNI bridge, Spark MemoryConsumer), JNI + +--- + +## File Map + +| File | Action | Responsibility | +| ------------------------------------------------------------------ | --------- | ------------------------------------------------------------------------- | +| `native/core/src/execution/memory_pools/spill.rs` | Create | `SpillState` struct with atomics and wait/notify | +| `native/core/src/execution/memory_pools/unified_pool.rs` | Modify | Check `SpillState` in `try_grow`, track freed in `shrink` | +| `native/core/src/execution/memory_pools/mod.rs` | Modify | Export `spill` module, pass `SpillState` through `create_memory_pool` | +| `native/core/src/execution/memory_pools/config.rs` | No change | Pool config unchanged | +| `native/core/src/execution/jni_api.rs` | Modify | Store `SpillState` in `ExecutionContext`, add JNI function `requestSpill` | +| `spark/src/main/java/org/apache/spark/CometTaskMemoryManager.java` | Modify | Implement `spill()`, store native handle, add JNI call | +| `spark/src/main/scala/org/apache/comet/Native.scala` | Modify | Add `requestSpill` native method declaration | +| `spark/src/main/scala/org/apache/comet/CometExecIterator.scala` | Modify | Pass native plan handle back to `CometTaskMemoryManager` | + +## Scope + +This plan covers the `GreedyUnified` pool only (the default off-heap pool). The `FairUnified` pool (`fair_pool.rs`) uses the same Spark JNI path and can be updated in a follow-up using the same pattern. + +--- + +### Task 1: Create SpillState + +**Files:** + +- Create: `native/core/src/execution/memory_pools/spill.rs` + +- [ ] **Step 1: Create the SpillState struct** + +```rust +// native/core/src/execution/memory_pools/spill.rs + +// 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 std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::{Condvar, Mutex}; +use std::time::Duration; + +/// Shared state for coordinating spill requests between Spark's memory manager +/// (which calls `NativeMemoryConsumer.spill()` on a Spark thread) and DataFusion +/// operators (which call `try_grow()`/`shrink()` on tokio threads). +/// +/// When Spark needs to reclaim memory from Comet, it sets `pressure` via +/// `request_spill()`. The memory pool's `try_grow()` checks this and returns +/// `ResourcesExhausted`, causing operators to spill. As operators call `shrink()`, +/// freed bytes are accumulated and the waiting Spark thread is notified. +#[derive(Debug)] +pub struct SpillState { + /// Bytes requested to be freed. Set by Spark's spill() callback. + pressure: AtomicUsize, + /// Bytes actually freed since pressure was set. + freed: AtomicUsize, + /// Mutex + Condvar to allow the spill requester to wait for operators to react. + notify: (Mutex<()>, Condvar), +} + +impl SpillState { + pub fn new() -> Self { + Self { + pressure: AtomicUsize::new(0), + freed: AtomicUsize::new(0), + notify: (Mutex::new(()), Condvar::new()), + } + } + + /// Returns the current spill pressure in bytes. Called by the memory pool's + /// `try_grow()` to decide whether to deny allocations. + pub fn pressure(&self) -> usize { + self.pressure.load(Ordering::Acquire) + } + + /// Record that `size` bytes were freed (called from pool's `shrink()`). + /// Wakes the waiting spill requester. + pub fn record_freed(&self, size: usize) { + self.freed.fetch_add(size, Ordering::Release); + let (_lock, cvar) = &self.notify; + cvar.notify_all(); + } + + /// Called from JNI when Spark's `NativeMemoryConsumer.spill()` is invoked. + /// Sets spill pressure and waits (up to `timeout`) for operators to free memory. + /// Returns the actual number of bytes freed. + pub fn request_spill(&self, size: usize, timeout: Duration) -> usize { + // Reset freed counter and set pressure + self.freed.store(0, Ordering::Release); + self.pressure.store(size, Ordering::Release); + + // Wait for operators to react + let (lock, cvar) = &self.notify; + let mut guard = lock.lock().unwrap(); + let deadline = std::time::Instant::now() + timeout; + while self.freed.load(Ordering::Acquire) < size { + let remaining = deadline.saturating_duration_since(std::time::Instant::now()); + if remaining.is_zero() { + break; + } + let (new_guard, _timeout_result) = cvar.wait_timeout(guard, remaining).unwrap(); + guard = new_guard; + } + + // Clear pressure and return freed bytes + self.pressure.store(0, Ordering::Release); + self.freed.load(Ordering::Acquire) + } +} + +impl Default for SpillState { + fn default() -> Self { + Self::new() + } +} +``` + +- [ ] **Step 2: Add module to mod.rs** + +Add to the top of `native/core/src/execution/memory_pools/mod.rs`, after the existing module declarations: + +```rust +pub(crate) mod spill; +``` + +- [ ] **Step 3: Build to verify compilation** + +Run: `cargo build --manifest-path native/Cargo.toml` +Expected: Build succeeds with no errors. + +- [ ] **Step 4: Commit** + +```bash +git add native/core/src/execution/memory_pools/spill.rs native/core/src/execution/memory_pools/mod.rs +git commit -m "feat: add SpillState for cross-thread spill coordination" +``` + +--- + +### Task 2: Wire SpillState into CometUnifiedMemoryPool + +**Files:** + +- Modify: `native/core/src/execution/memory_pools/unified_pool.rs` +- Modify: `native/core/src/execution/memory_pools/mod.rs` + +- [ ] **Step 1: Add SpillState to CometUnifiedMemoryPool** + +In `unified_pool.rs`, add the `spill_state` field and update the constructor: + +```rust +use super::spill::SpillState; +use std::sync::Arc; +``` + +Change the struct definition: + +```rust +pub struct CometUnifiedMemoryPool { + task_memory_manager_handle: Arc, + used: AtomicUsize, + task_attempt_id: i64, + spill_state: Arc, +} +``` + +Update `Debug` impl to include the new field: + +```rust +impl Debug for CometUnifiedMemoryPool { + fn fmt(&self, f: &mut Formatter<'_>) -> FmtResult { + f.debug_struct("CometUnifiedMemoryPool") + .field("used", &self.used.load(Relaxed)) + .field("spill_pressure", &self.spill_state.pressure()) + .finish() + } +} +``` + +Update the constructor: + +```rust +pub fn new( + task_memory_manager_handle: Arc, + task_attempt_id: i64, + spill_state: Arc, +) -> CometUnifiedMemoryPool { + Self { + task_memory_manager_handle, + task_attempt_id, + used: AtomicUsize::new(0), + spill_state, + } +} +``` + +- [ ] **Step 2: Check spill pressure in try_grow** + +Replace the `try_grow` implementation: + +```rust +fn try_grow(&self, _: &MemoryReservation, additional: usize) -> Result<(), DataFusionError> { + if additional > 0 { + // If there is spill pressure, deny the allocation to trigger operator spill + if self.spill_state.pressure() > 0 { + return Err(resources_datafusion_err!( + "Task {} denied {} bytes due to spill pressure. Reserved: {}", + self.task_attempt_id, + additional, + self.reserved() + )); + } + + let acquired = self.acquire_from_spark(additional)?; + // If the number of bytes we acquired is less than the requested, return an error, + // and hopefully will trigger spilling from the caller side. + if acquired < additional as i64 { + // Release the acquired bytes before throwing error + self.release_to_spark(acquired as usize)?; + + return Err(resources_datafusion_err!( + "Task {} failed to acquire {} bytes, only got {}. Reserved: {}", + self.task_attempt_id, + additional, + acquired, + self.reserved() + )); + } + if let Err(prev) = self + .used + .fetch_update(Relaxed, Relaxed, |old| old.checked_add(acquired as usize)) + { + return Err(resources_datafusion_err!( + "Task {} failed to acquire {} bytes due to overflow. Reserved: {}", + self.task_attempt_id, + additional, + prev + )); + } + } + Ok(()) +} +``` + +- [ ] **Step 3: Record freed bytes in shrink** + +Replace the `shrink` implementation: + +```rust +fn shrink(&self, _: &MemoryReservation, size: usize) { + if let Err(e) = self.release_to_spark(size) { + panic!( + "Task {} failed to return {size} bytes to Spark: {e:?}", + self.task_attempt_id + ); + } + if let Err(prev) = self + .used + .fetch_update(Relaxed, Relaxed, |old| old.checked_sub(size)) + { + panic!( + "Task {} overflow when releasing {size} of {prev} bytes", + self.task_attempt_id + ); + } + // Notify the spill requester that memory was freed + if self.spill_state.pressure() > 0 { + self.spill_state.record_freed(size); + } +} +``` + +- [ ] **Step 4: Update create_memory_pool to accept and pass SpillState** + +In `native/core/src/execution/memory_pools/mod.rs`, update the function signature and the `GreedyUnified` arm: + +```rust +use super::memory_pools::spill::SpillState; +``` + +Change the signature of `create_memory_pool`: + +```rust +pub(crate) fn create_memory_pool( + memory_pool_config: &MemoryPoolConfig, + comet_task_memory_manager: Arc, + task_attempt_id: i64, + spill_state: Option>, +) -> Arc { +``` + +Update the `GreedyUnified` match arm: + +```rust +MemoryPoolType::GreedyUnified => { + let spill_state = spill_state + .unwrap_or_else(|| Arc::new(SpillState::new())); + let memory_pool = CometUnifiedMemoryPool::new( + comet_task_memory_manager, + task_attempt_id, + spill_state, + ); + Arc::new(TrackConsumersPool::new( + memory_pool, + NonZeroUsize::new(NUM_TRACKED_CONSUMERS).unwrap(), + )) +} +``` + +- [ ] **Step 5: Update call site in jni_api.rs** + +In `jni_api.rs`, update the call to `create_memory_pool` (around line 260): + +```rust +let memory_pool = + create_memory_pool(&memory_pool_config, task_memory_manager, task_attempt_id, None); +``` + +We pass `None` for now — Task 3 will wire in the real `SpillState`. + +- [ ] **Step 6: Build to verify compilation** + +Run: `cargo build --manifest-path native/Cargo.toml` +Expected: Build succeeds. + +- [ ] **Step 7: Commit** + +```bash +git add native/core/src/execution/memory_pools/unified_pool.rs \ + native/core/src/execution/memory_pools/mod.rs \ + native/core/src/execution/jni_api.rs +git commit -m "feat: wire SpillState into CometUnifiedMemoryPool" +``` + +--- + +### Task 3: Add JNI requestSpill Function and Store SpillState in ExecutionContext + +**Files:** + +- Modify: `native/core/src/execution/jni_api.rs` + +- [ ] **Step 1: Add SpillState to ExecutionContext and pass through create_memory_pool** + +Add field to `ExecutionContext` struct (around line 179): + +```rust +pub spill_state: Option>, +``` + +Add the import near the top of `jni_api.rs`: + +```rust +use crate::execution::memory_pools::spill::SpillState; +``` + +In the `createPlan` JNI function, create the `SpillState` before pool creation (around line 259): + +```rust +let spill_state = if off_heap_mode != JNI_FALSE + && matches!(memory_pool_config.pool_type, MemoryPoolType::GreedyUnified) +{ + Some(Arc::new(SpillState::new())) +} else { + None +}; + +let memory_pool = create_memory_pool( + &memory_pool_config, + task_memory_manager, + task_attempt_id, + spill_state.clone(), +); +``` + +Add the field to the `ExecutionContext` initialization (around line 328): + +```rust +spill_state, +``` + +- [ ] **Step 2: Add the JNI requestSpill function** + +Add this function at module level in `jni_api.rs` (after the existing JNI functions): + +```rust +/// Called from `CometTaskMemoryManager.spill()` via JNI to request that native +/// operators free memory. Returns the number of bytes actually freed. +/// # 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_requestSpill( + e: JNIEnv, + _class: JClass, + exec_context: jlong, + size: jlong, +) -> jlong { + try_unwrap_or_throw(&e, |_env| { + let exec_context = get_execution_context(exec_context); + if let Some(ref spill_state) = exec_context.spill_state { + let timeout = std::time::Duration::from_secs(10); + let freed = spill_state.request_spill(size as usize, timeout); + Ok(freed as jlong) + } else { + // No spill state (not using unified pool) — can't spill + Ok(0i64) + } + }) +} +``` + +- [ ] **Step 3: Build to verify compilation** + +Run: `cargo build --manifest-path native/Cargo.toml` +Expected: Build succeeds. + +- [ ] **Step 4: Commit** + +```bash +git add native/core/src/execution/jni_api.rs +git commit -m "feat: add JNI requestSpill function with SpillState in ExecutionContext" +``` + +--- + +### Task 4: Update JVM Side — CometTaskMemoryManager and Native + +**Files:** + +- Modify: `spark/src/main/java/org/apache/spark/CometTaskMemoryManager.java` +- Modify: `spark/src/main/scala/org/apache/comet/Native.scala` + +- [ ] **Step 1: Add requestSpill to Native.scala** + +In `spark/src/main/scala/org/apache/comet/Native.scala`, add after the `releasePlan` method (around line 104): + +```scala + /** + * Request that native operators spill memory. Called from CometTaskMemoryManager.spill(). + * + * @param nativePlanHandle + * the native ExecutionContext pointer + * @param size + * bytes requested to free + * @return + * actual bytes freed + */ + @native def requestSpill(nativePlanHandle: Long, size: Long): Long +``` + +- [ ] **Step 2: Update CometTaskMemoryManager to store native handle and implement spill** + +Replace the contents of `CometTaskMemoryManager.java`: + +```java +package org.apache.spark; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicLong; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.comet.Native; +import org.apache.spark.memory.MemoryConsumer; +import org.apache.spark.memory.MemoryMode; +import org.apache.spark.memory.TaskMemoryManager; + +/** + * A adapter class that is used by Comet native to acquire & release memory through Spark's unified + * memory manager. This assumes Spark's off-heap memory mode is enabled. + */ +public class CometTaskMemoryManager { + + private static final Logger logger = LoggerFactory.getLogger(CometTaskMemoryManager.class); + + /** The id uniquely identifies the native plan this memory manager is associated to */ + private final long id; + + private final long taskAttemptId; + + public final TaskMemoryManager internal; + private final NativeMemoryConsumer nativeMemoryConsumer; + private final AtomicLong used = new AtomicLong(); + + /** + * The native ExecutionContext handle. Set after native plan creation. Used to route spill() + * requests to the native memory pool. + */ + private volatile long nativePlanHandle = 0; + + public CometTaskMemoryManager(long id, long taskAttemptId) { + this.id = id; + this.taskAttemptId = taskAttemptId; + this.internal = TaskContext$.MODULE$.get().taskMemoryManager(); + this.nativeMemoryConsumer = new NativeMemoryConsumer(); + } + + /** + * Set the native plan handle after plan creation. This enables the spill() callback to route + * requests to the native memory pool. + */ + public void setNativePlanHandle(long handle) { + this.nativePlanHandle = handle; + } + + // Called by Comet native through JNI. + // Returns the actual amount of memory (in bytes) granted. + public long acquireMemory(long size) { + if (logger.isTraceEnabled()) { + logger.trace("Task {} requested {} bytes", taskAttemptId, size); + } + long acquired = internal.acquireExecutionMemory(size, nativeMemoryConsumer); + long newUsed = used.addAndGet(acquired); + if (acquired < size) { + logger.warn( + "Task {} requested {} bytes but only received {} bytes. Current allocation is {} and " + + "the total memory consumption is {} bytes.", + taskAttemptId, + size, + acquired, + newUsed, + internal.getMemoryConsumptionForThisTask()); + // If memory manager is not able to acquire the requested size, log memory usage + internal.showMemoryUsage(); + } + return acquired; + } + + // Called by Comet native through JNI + public void releaseMemory(long size) { + if (logger.isTraceEnabled()) { + logger.trace("Task {} released {} bytes", taskAttemptId, size); + } + long newUsed = used.addAndGet(-size); + if (newUsed < 0) { + logger.error( + "Task {} used memory is negative ({}) after releasing {} bytes", + taskAttemptId, + newUsed, + size); + } + internal.releaseExecutionMemory(size, nativeMemoryConsumer); + } + + public long getUsed() { + return used.get(); + } + + /** + * A memory consumer that routes spill requests to the native memory pool. When Spark's memory + * manager needs to reclaim memory (e.g., for another task), it calls spill() which signals the + * native pool to apply backpressure. DataFusion operators (Sort, Aggregate, Shuffle) react by + * spilling their internal state to disk. + */ + private class NativeMemoryConsumer extends MemoryConsumer { + protected NativeMemoryConsumer() { + super(CometTaskMemoryManager.this.internal, 0, MemoryMode.OFF_HEAP); + } + + @Override + public long spill(long size, MemoryConsumer trigger) throws IOException { + long handle = nativePlanHandle; + if (handle == 0) { + // Native plan not yet created or already destroyed + return 0; + } + logger.info( + "Task {} received spill request for {} bytes, forwarding to native", + taskAttemptId, + size); + try { + long freed = new Native().requestSpill(handle, size); + logger.info("Task {} native spill freed {} bytes", taskAttemptId, freed); + return freed; + } catch (Exception e) { + logger.warn("Task {} native spill failed: {}", taskAttemptId, e.getMessage()); + return 0; + } + } + + @Override + public String toString() { + return String.format("NativeMemoryConsumer(id=%d, taskAttemptId=%d)", id, taskAttemptId); + } + } +} +``` + +- [ ] **Step 3: Build JVM to verify compilation** + +Run: `./mvnw compile -DskipTests` +Expected: BUILD SUCCESS + +- [ ] **Step 4: Commit** + +```bash +git add spark/src/main/java/org/apache/spark/CometTaskMemoryManager.java \ + spark/src/main/scala/org/apache/comet/Native.scala +git commit -m "feat: implement spill() callback in CometTaskMemoryManager" +``` + +--- + +### Task 5: Wire Native Plan Handle Back to CometTaskMemoryManager + +**Files:** + +- Modify: `spark/src/main/scala/org/apache/comet/CometExecIterator.scala` + +- [ ] **Step 1: Set the native plan handle after creation** + +In `CometExecIterator.scala`, after the `nativeLib.createPlan(...)` call (around line 131), add: + +```scala + private val plan = { + // ... existing code that calls nativeLib.createPlan ... + + val nativePlan = nativeLib.createPlan( + id, + inputIterators, + protobufQueryPlan, + protobufSparkConfigs, + numParts, + nativeMetrics, + metricsUpdateInterval = COMET_METRICS_UPDATE_INTERVAL.get(), + cometTaskMemoryManager, + localDiskDirs, + batchSize = COMET_BATCH_SIZE.get(), + memoryConfig.offHeapMode, + memoryConfig.memoryPoolType, + memoryConfig.memoryLimit, + memoryConfig.memoryLimitPerTask, + taskAttemptId, + taskCPUs, + keyUnwrapper) + + // Enable spill callback by giving the memory manager a handle to the native plan + cometTaskMemoryManager.setNativePlanHandle(nativePlan) + + nativePlan + } +``` + +The existing code calls `nativeLib.createPlan(...)` and that's the last expression in the block (making it the return value). Restructure so we can call `setNativePlanHandle` and still return the handle: + +```scala + val handle = nativeLib.createPlan( + id, + inputIterators, + // ... all existing args unchanged ... + keyUnwrapper) + + // Enable spill callback by giving the memory manager a handle to the native plan + cometTaskMemoryManager.setNativePlanHandle(handle) + + handle + } +``` + +- [ ] **Step 2: Clear the handle on close** + +In the `close()` method (line 231), add `cometTaskMemoryManager.setNativePlanHandle(0)` +before the `nativeLib.releasePlan(plan)` call to prevent spill callbacks after the +native plan is destroyed: + +```scala + def close(): Unit = synchronized { + if (!closed) { + if (currentBatch != null) { + currentBatch.close() + currentBatch = null + } + nativeUtil.close() + shuffleBlockIterators.values.foreach(_.close()) + cometTaskMemoryManager.setNativePlanHandle(0) + nativeLib.releasePlan(plan) +``` + +- [ ] **Step 3: Build everything** + +Run: `make` +Expected: Both native and JVM build succeed. + +- [ ] **Step 4: Commit** + +```bash +git add spark/src/main/scala/org/apache/comet/CometExecIterator.scala +git commit -m "feat: wire native plan handle to CometTaskMemoryManager for spill routing" +``` + +--- + +### Task 6: Test with TPC-H Q9 + +- [ ] **Step 1: Run the memory profile script for Q9** + +```bash +cd benchmarks/tpc +./memory-profile.sh --queries "9" --offheap-sizes "4g 8g" --cores 4 +``` + +Compare peak RSS against the baseline results: + +- Spark 4g: 4580 MB +- Comet 4g (before): 5911 MB +- Comet 8g (before): 6359 MB + +**Expected:** The elastic growth (4g→8g delta) should be reduced because Spark can +now reclaim memory from Comet's shuffle writers when other tasks need it. The absolute +numbers may also decrease slightly. + +- [ ] **Step 2: Verify spill logging** + +Check the logs for spill activity: + +```bash +grep -i "spill" benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.log +``` + +Expected: Log lines showing "received spill request" and "native spill freed N bytes" +from `CometTaskMemoryManager`. + +- [ ] **Step 3: Run full test suite for regressions** + +```bash +./mvnw test -DwildcardSuites="CometExec" +``` + +Expected: All existing tests pass. + +- [ ] **Step 4: Commit results and update analysis** + +Update `docs/source/contributor-guide/memory-management.md` with the new results. + +```bash +git add docs/source/contributor-guide/memory-management.md +git commit -m "docs: add spill callback benchmark results" +``` + +--- + +## Notes + +- **Timeout**: The 10-second timeout in `request_spill()` is conservative. If no operators + are actively allocating, the spill won't trigger until one does. In practice, during + TPC-H execution, operators allocate frequently so the response should be fast. + +- **FairUnified pool**: Not covered by this plan. It uses a different pool struct + (`CometFairMemoryPool`) but the same pattern applies — add `SpillState`, check in + `try_grow`, record in `shrink`. + +- **Thread safety**: `spill()` is called from Spark's memory manager thread. + `try_grow()`/`shrink()` are called from tokio threads. The `AtomicUsize` + `Condvar` + design handles this safely without locks on the hot path (`pressure()` is a single + atomic load). + +- **Multiple native plans per task**: A single Spark task may create multiple + `CometExecIterator` instances (e.g., for subqueries). Each has its own + `CometTaskMemoryManager` and `NativeMemoryConsumer`. Spark will call `spill()` on each + independently, which is correct — each routes to its own pool. diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala index 470f18a829..45677d93fb 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffledRowRDD.scala @@ -150,9 +150,7 @@ class CometShuffledBatchRDD( * Returns the raw InputStream of concatenated Arrow IPC streams for direct consumption by * native code via ShuffleStreamReader. */ - def computeAsRawStream( - split: Partition, - context: TaskContext): java.io.InputStream = { + def computeAsRawStream(split: Partition, context: TaskContext): java.io.InputStream = { val reader = createReader(split, context) reader.readAsRawStream() } From 63545acc3ef2ac38aaa46e1599b0eec716b7b859 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Apr 2026 10:19:43 -0600 Subject: [PATCH 43/46] fix: handle empty streams and concatenated IPC streams in shuffle reader --- native/shuffle/src/ipc.rs | 31 ++++++++++++++++++++----------- 1 file changed, 20 insertions(+), 11 deletions(-) diff --git a/native/shuffle/src/ipc.rs b/native/shuffle/src/ipc.rs index 192bcc9746..1664877b3a 100644 --- a/native/shuffle/src/ipc.rs +++ b/native/shuffle/src/ipc.rs @@ -208,21 +208,30 @@ pub struct ShuffleStreamReader { impl ShuffleStreamReader { /// Create a new `ShuffleStreamReader` over a JVM InputStream. + /// Returns a reader that yields no batches if the stream is empty. pub fn new(env: &mut jni::JNIEnv, input_stream: &JObject) -> Result { let jni_stream = SharedJniStream::new( JniInputStream::new(env, input_stream).map_err(|e| format!("JNI error: {e}"))?, ); - let reader = unsafe { - StreamReader::try_new(jni_stream.reader(), None) - .map_err(|e| format!("Arrow IPC error: {e}"))? - .with_skip_validation(true) - }; - let num_fields = reader.schema().fields().len(); - Ok(Self { - jni_stream, - reader: Some(reader), - num_fields, - }) + match StreamReader::try_new(jni_stream.reader(), None) { + Ok(reader) => { + let reader = unsafe { reader.with_skip_validation(true) }; + let num_fields = reader.schema().fields().len(); + Ok(Self { + jni_stream, + reader: Some(reader), + num_fields, + }) + } + Err(_) => { + // Empty stream — no data for this partition + Ok(Self { + jni_stream, + reader: None, + num_fields: 0, + }) + } + } } /// Read the next batch from the stream. Returns `None` when all From a1c76b2654be2b649ba473d8f483c9e0078e8688 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Apr 2026 10:30:50 -0600 Subject: [PATCH 44/46] refactor: clean up shuffle format migration dead code and review findings - Hold a single StreamWriter across all batches in process_sorted_row_partition instead of creating a fresh writer per batch - Remove read_ipc_compressed and snap/lz4_flex/zstd dependencies from shuffle crate - Remove dead CometShuffleBlockIterator.java and its JNI bridge - Rename shuffle_block_writer.rs to codec.rs to reflect its contents - Remove unused _write_time parameter from BufBatchWriter write/flush - Make CompressionCodec::Snappy return an error in ipc_write_options - Remove Snappy from shuffle writer codec mappings in planner and JNI --- native/Cargo.lock | 16 +- native/core/src/execution/jni_api.rs | 1 - native/core/src/execution/planner.rs | 1 - native/jni-bridge/src/lib.rs | 6 - .../jni-bridge/src/shuffle_block_iterator.rs | 62 -------- native/shuffle/Cargo.toml | 4 - native/shuffle/src/ipc.rs | 36 ----- native/shuffle/src/lib.rs | 2 +- .../src/partitioners/multi_partition.rs | 6 +- .../src/partitioners/single_partition.rs | 10 +- native/shuffle/src/shuffle_writer.rs | 14 +- native/shuffle/src/spark_unsafe/row.rs | 48 +++--- .../shuffle/src/writers/buf_batch_writer.rs | 2 - .../{shuffle_block_writer.rs => codec.rs} | 7 +- native/shuffle/src/writers/mod.rs | 4 +- .../comet/CometShuffleBlockIterator.java | 142 ------------------ 16 files changed, 49 insertions(+), 312 deletions(-) delete mode 100644 native/jni-bridge/src/shuffle_block_iterator.rs rename native/shuffle/src/writers/{shuffle_block_writer.rs => codec.rs} (86%) delete mode 100644 spark/src/main/java/org/apache/comet/CometShuffleBlockIterator.java diff --git a/native/Cargo.lock b/native/Cargo.lock index 88e6bebf7c..067b511fe1 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -332,7 +332,7 @@ dependencies = [ "arrow-schema", "arrow-select", "flatbuffers", - "lz4_flex 0.12.1", + "lz4_flex", "zstd", ] @@ -2031,13 +2031,10 @@ dependencies = [ "itertools 0.14.0", "jni", "log", - "lz4_flex 0.13.0", "parquet", "simd-adler32", - "snap", "tempfile", "tokio", - "zstd", ] [[package]] @@ -4081,15 +4078,6 @@ dependencies = [ "twox-hash", ] -[[package]] -name = "lz4_flex" -version = "0.13.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db9a0d582c2874f68138a16ce1867e0ffde6c0bb0a0df85e1f36d04146db488a" -dependencies = [ - "twox-hash", -] - [[package]] name = "md-5" version = "0.10.6" @@ -4519,7 +4507,7 @@ dependencies = [ "futures", "half", "hashbrown 0.16.1", - "lz4_flex 0.12.1", + "lz4_flex", "num-bigint", "num-integer", "num-traits", diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index c704d4613d..2ac89b2669 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -808,7 +808,6 @@ pub unsafe extern "system" fn Java_org_apache_comet_Native_writeSortedFileNative let compression_codec = match compression_codec.as_str() { "zstd" => CompressionCodec::Zstd(compression_level), "lz4" => CompressionCodec::Lz4Frame, - "snappy" => CompressionCodec::Snappy, _ => CompressionCodec::Lz4Frame, }; diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index d79c06bc21..c150fb480a 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -1354,7 +1354,6 @@ impl PhysicalPlanner { let codec = match writer.codec.try_into() { Ok(SparkCompressionCodec::None) => Ok(CompressionCodec::None), - Ok(SparkCompressionCodec::Snappy) => Ok(CompressionCodec::Snappy), Ok(SparkCompressionCodec::Zstd) => { Ok(CompressionCodec::Zstd(writer.compression_level)) } diff --git a/native/jni-bridge/src/lib.rs b/native/jni-bridge/src/lib.rs index a2e25c3e2f..fdc69c527e 100644 --- a/native/jni-bridge/src/lib.rs +++ b/native/jni-bridge/src/lib.rs @@ -181,12 +181,9 @@ pub use comet_exec::*; mod batch_iterator; mod comet_metric_node; mod comet_task_memory_manager; -mod shuffle_block_iterator; - use batch_iterator::CometBatchIterator; pub use comet_metric_node::*; pub use comet_task_memory_manager::*; -use shuffle_block_iterator::CometShuffleBlockIterator; /// The JVM classes that are used in the JNI calls. #[allow(dead_code)] // we need to keep references to Java items to prevent GC @@ -212,8 +209,6 @@ pub struct JVMClasses<'a> { pub comet_exec: CometExec<'a>, /// The CometBatchIterator class. Used for iterating over the batches. pub comet_batch_iterator: CometBatchIterator<'a>, - /// The CometShuffleBlockIterator class. Used for iterating over shuffle blocks. - pub comet_shuffle_block_iterator: CometShuffleBlockIterator<'a>, /// The CometTaskMemoryManager used for interacting with JVM side to /// acquire & release native memory. pub comet_task_memory_manager: CometTaskMemoryManager<'a>, @@ -267,7 +262,6 @@ impl JVMClasses<'_> { comet_metric_node: CometMetricNode::new(env).unwrap(), comet_exec: CometExec::new(env).unwrap(), comet_batch_iterator: CometBatchIterator::new(env).unwrap(), - comet_shuffle_block_iterator: CometShuffleBlockIterator::new(env).unwrap(), comet_task_memory_manager: CometTaskMemoryManager::new(env).unwrap(), } }); diff --git a/native/jni-bridge/src/shuffle_block_iterator.rs b/native/jni-bridge/src/shuffle_block_iterator.rs deleted file mode 100644 index c3bb5af5fb..0000000000 --- a/native/jni-bridge/src/shuffle_block_iterator.rs +++ /dev/null @@ -1,62 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use jni::signature::Primitive; -use jni::{ - errors::Result as JniResult, - objects::{JClass, JMethodID}, - signature::ReturnType, - JNIEnv, -}; - -/// A struct that holds all the JNI methods and fields for JVM `CometShuffleBlockIterator` class. -#[allow(dead_code)] // we need to keep references to Java items to prevent GC -pub struct CometShuffleBlockIterator<'a> { - pub class: JClass<'a>, - pub method_has_next: JMethodID, - pub method_has_next_ret: ReturnType, - pub method_get_buffer: JMethodID, - pub method_get_buffer_ret: ReturnType, - pub method_get_current_block_length: JMethodID, - pub method_get_current_block_length_ret: ReturnType, -} - -impl<'a> CometShuffleBlockIterator<'a> { - pub const JVM_CLASS: &'static str = "org/apache/comet/CometShuffleBlockIterator"; - - pub fn new(env: &mut JNIEnv<'a>) -> JniResult> { - let class = env.find_class(Self::JVM_CLASS)?; - - Ok(CometShuffleBlockIterator { - class, - method_has_next: env.get_method_id(Self::JVM_CLASS, "hasNext", "()I")?, - method_has_next_ret: ReturnType::Primitive(Primitive::Int), - method_get_buffer: env.get_method_id( - Self::JVM_CLASS, - "getBuffer", - "()Ljava/nio/ByteBuffer;", - )?, - method_get_buffer_ret: ReturnType::Object, - method_get_current_block_length: env.get_method_id( - Self::JVM_CLASS, - "getCurrentBlockLength", - "()I", - )?, - method_get_current_block_length_ret: ReturnType::Primitive(Primitive::Int), - }) - } -} diff --git a/native/shuffle/Cargo.toml b/native/shuffle/Cargo.toml index 6e2bd4d7bb..9528a66727 100644 --- a/native/shuffle/Cargo.toml +++ b/native/shuffle/Cargo.toml @@ -43,14 +43,10 @@ futures = { workspace = true } itertools = "0.14.0" jni = "0.21" log = "0.4" -lz4_flex = { version = "0.13.0", default-features = false, features = ["frame"] } # parquet is only used by the shuffle_bench binary (shuffle-bench feature) parquet = { workspace = true, optional = true } simd-adler32 = "0.3.9" -# snap, lz4_flex, zstd are still needed by ipc.rs (old read path) until Task 6 -snap = "1.1" tokio = { version = "1", features = ["rt-multi-thread"] } -zstd = "0.13.3" [dev-dependencies] criterion = { version = "0.7", features = ["async", "async_tokio", "async_std"] } diff --git a/native/shuffle/src/ipc.rs b/native/shuffle/src/ipc.rs index 1664877b3a..bf9dd2cc3e 100644 --- a/native/shuffle/src/ipc.rs +++ b/native/shuffle/src/ipc.rs @@ -17,8 +17,6 @@ use arrow::array::RecordBatch; use arrow::ipc::reader::StreamReader; -use datafusion::common::DataFusionError; -use datafusion::error::Result; use jni::objects::{GlobalRef, JObject, JValue}; use jni::JavaVM; use std::io::Read; @@ -273,37 +271,3 @@ impl ShuffleStreamReader { self.num_fields } } - -/// Read a single RecordBatch from a compressed IPC block. -/// The first 4 bytes indicate the compression codec, followed by the IPC stream data. -/// This is the legacy read path used by `ShuffleScanExec`. -pub fn read_ipc_compressed(bytes: &[u8]) -> Result { - match &bytes[0..4] { - b"SNAP" => { - let decoder = snap::read::FrameDecoder::new(&bytes[4..]); - let mut reader = - unsafe { StreamReader::try_new(decoder, None)?.with_skip_validation(true) }; - reader.next().unwrap().map_err(|e| e.into()) - } - b"LZ4_" => { - let decoder = lz4_flex::frame::FrameDecoder::new(&bytes[4..]); - let mut reader = - unsafe { StreamReader::try_new(decoder, None)?.with_skip_validation(true) }; - reader.next().unwrap().map_err(|e| e.into()) - } - b"ZSTD" => { - let decoder = zstd::Decoder::new(&bytes[4..])?; - let mut reader = - unsafe { StreamReader::try_new(decoder, None)?.with_skip_validation(true) }; - reader.next().unwrap().map_err(|e| e.into()) - } - b"NONE" => { - let mut reader = - unsafe { StreamReader::try_new(&bytes[4..], None)?.with_skip_validation(true) }; - reader.next().unwrap().map_err(|e| e.into()) - } - other => Err(DataFusionError::Execution(format!( - "Failed to decode batch: invalid compression codec: {other:?}" - ))), - } -} diff --git a/native/shuffle/src/lib.rs b/native/shuffle/src/lib.rs index cf4297404e..1c31bda5ef 100644 --- a/native/shuffle/src/lib.rs +++ b/native/shuffle/src/lib.rs @@ -25,6 +25,6 @@ pub mod spark_unsafe; pub(crate) mod writers; pub use comet_partitioning::CometPartitioning; -pub use ipc::{read_ipc_compressed, JniInputStream, ShuffleStreamReader}; +pub use ipc::{JniInputStream, ShuffleStreamReader}; pub use shuffle_writer::ShuffleWriterExec; pub use writers::CompressionCodec; diff --git a/native/shuffle/src/partitioners/multi_partition.rs b/native/shuffle/src/partitioners/multi_partition.rs index c7db1a00f9..1801a52d3e 100644 --- a/native/shuffle/src/partitioners/multi_partition.rs +++ b/native/shuffle/src/partitioners/multi_partition.rs @@ -442,7 +442,6 @@ impl MultiPartitionShuffleRepartitioner { write_options: &IpcWriteOptions, output_data: &mut BufWriter, encode_time: &Time, - write_time: &Time, batch_size: usize, ) -> datafusion::common::Result<()> { let mut buf_batch_writer = BufBatchWriter::try_new( @@ -453,9 +452,9 @@ impl MultiPartitionShuffleRepartitioner { )?; for batch in partition_iter { let batch = batch?; - buf_batch_writer.write(&batch, encode_time, write_time)?; + buf_batch_writer.write(&batch, encode_time)?; } - buf_batch_writer.flush(encode_time, write_time)?; + buf_batch_writer.flush(encode_time)?; Ok(()) } @@ -594,7 +593,6 @@ impl ShufflePartitioner for MultiPartitionShuffleRepartitioner { &self.write_options, &mut output_data, &self.metrics.encode_time, - &self.metrics.write_time, self.batch_size, )?; } diff --git a/native/shuffle/src/partitioners/single_partition.rs b/native/shuffle/src/partitioners/single_partition.rs index 1d74a7939b..d487d9310b 100644 --- a/native/shuffle/src/partitioners/single_partition.rs +++ b/native/shuffle/src/partitioners/single_partition.rs @@ -77,11 +77,8 @@ impl ShufflePartitioner for SinglePartitionShufflePartitioner { self.metrics.data_size.add(batch.get_array_memory_size()); self.metrics.baseline.record_output(num_rows); - self.output_data_writer.write( - &batch, - &self.metrics.encode_time, - &self.metrics.write_time, - )?; + self.output_data_writer + .write(&batch, &self.metrics.encode_time)?; } self.metrics.input_batches.add(1); @@ -95,8 +92,7 @@ impl ShufflePartitioner for SinglePartitionShufflePartitioner { fn shuffle_write(&mut self) -> datafusion::common::Result<()> { let start_time = Instant::now(); - self.output_data_writer - .flush(&self.metrics.encode_time, &self.metrics.write_time)?; + self.output_data_writer.flush(&self.metrics.encode_time)?; // Get data file length via filesystem metadata let data_file_length = std::fs::metadata(&self.output_data_path) diff --git a/native/shuffle/src/shuffle_writer.rs b/native/shuffle/src/shuffle_writer.rs index bbf3fd432c..63458bfeb1 100644 --- a/native/shuffle/src/shuffle_writer.rs +++ b/native/shuffle/src/shuffle_writer.rs @@ -315,14 +315,13 @@ mod test { let write_options = codec.ipc_write_options().unwrap(); let mut output = Vec::new(); let encode_time = Time::default(); - let write_time = Time::default(); { let mut writer = BufBatchWriter::try_new(&mut output, batch.schema(), write_options, 8192) .unwrap(); - writer.write(&batch, &encode_time, &write_time).unwrap(); - writer.flush(&encode_time, &write_time).unwrap(); + writer.write(&batch, &encode_time).unwrap(); + writer.flush(&encode_time).unwrap(); } assert!(!output.is_empty()); @@ -648,7 +647,6 @@ mod test { let codec = CompressionCodec::Lz4Frame; let write_options = codec.ipc_write_options().unwrap(); let encode_time = Time::default(); - let write_time = Time::default(); // Write with coalescing (batch_size=8192) let mut coalesced_output = Vec::new(); @@ -661,9 +659,9 @@ mod test { ) .unwrap(); for batch in &small_batches { - buf_writer.write(batch, &encode_time, &write_time).unwrap(); + buf_writer.write(batch, &encode_time).unwrap(); } - buf_writer.flush(&encode_time, &write_time).unwrap(); + buf_writer.flush(&encode_time).unwrap(); } // Write without coalescing (batch_size=1) @@ -677,9 +675,9 @@ mod test { ) .unwrap(); for batch in &small_batches { - buf_writer.write(batch, &encode_time, &write_time).unwrap(); + buf_writer.write(batch, &encode_time).unwrap(); } - buf_writer.flush(&encode_time, &write_time).unwrap(); + buf_writer.flush(&encode_time).unwrap(); } // Verify both roundtrip correctly by reading all batches via StreamReader diff --git a/native/shuffle/src/spark_unsafe/row.rs b/native/shuffle/src/spark_unsafe/row.rs index 3aa7b66081..0accf61daf 100644 --- a/native/shuffle/src/spark_unsafe/row.rs +++ b/native/shuffle/src/spark_unsafe/row.rs @@ -1311,8 +1311,6 @@ pub fn process_sorted_row_partition( ) -> Result<(i64, Option), CometError> { // The current row number we are reading let mut current_row = 0; - // Total number of bytes written - let mut written = 0; // The current checksum value. This is updated incrementally in the following loop. let mut current_checksum = if checksum_enabled { Some(Checksum::try_new(checksum_algo, initial_checksum)?) @@ -1335,9 +1333,14 @@ pub fn process_sorted_row_partition( .append(true) .open(&output_path)?; - // Reusable buffer for serialized batch data + // Buffer that accumulates all IPC bytes across the single stream let mut frozen: Vec = Vec::new(); + // Build a schema from the first batch's datatypes so we can create the StreamWriter + // up front. We need a placeholder schema; we'll create it from the first batch. + let mut stream_writer: Option>> = None; + let write_options = codec.ipc_write_options()?; + while current_row < row_num { let n = std::cmp::min(batch_size, row_num - current_row); @@ -1366,30 +1369,33 @@ pub fn process_sorted_row_partition( .collect(); let batch = make_batch(array_refs?, n)?; - frozen.clear(); + // Create the StreamWriter on the first batch (we need the schema) + if stream_writer.is_none() { + stream_writer = Some(arrow::ipc::writer::StreamWriter::try_new_with_options( + &mut frozen, + &batch.schema(), + write_options.clone(), + )?); + } - // Write the batch as a complete IPC stream into `frozen` - let write_options = codec.ipc_write_options()?; - let mut stream_writer = arrow::ipc::writer::StreamWriter::try_new_with_options( - &mut frozen, - &batch.schema(), - write_options, - )?; - stream_writer.write(&batch)?; - stream_writer.finish()?; - stream_writer.into_inner()?; + stream_writer.as_mut().unwrap().write(&batch)?; + current_row += n; + } - written += frozen.len(); + // Finish the IPC stream and flush remaining bytes + if let Some(mut writer) = stream_writer { + writer.finish()?; + } - if let Some(checksum) = &mut current_checksum { - let mut cursor = Cursor::new(&mut frozen); - checksum.update(&mut cursor)?; - } + let written = frozen.len(); - output_data.write_all(&frozen)?; - current_row += n; + if let Some(checksum) = &mut current_checksum { + let mut cursor = Cursor::new(&mut frozen); + checksum.update(&mut cursor)?; } + output_data.write_all(&frozen)?; + Ok((written as i64, current_checksum.map(|c| c.finalize()))) } diff --git a/native/shuffle/src/writers/buf_batch_writer.rs b/native/shuffle/src/writers/buf_batch_writer.rs index 4c8d32f25d..3c188de7ef 100644 --- a/native/shuffle/src/writers/buf_batch_writer.rs +++ b/native/shuffle/src/writers/buf_batch_writer.rs @@ -47,7 +47,6 @@ impl BufBatchWriter { &mut self, batch: &RecordBatch, encode_time: &Time, - _write_time: &Time, ) -> datafusion::common::Result<()> { self.coalescer.push_batch(batch.clone())?; @@ -67,7 +66,6 @@ impl BufBatchWriter { pub(crate) fn flush( &mut self, encode_time: &Time, - _write_time: &Time, ) -> datafusion::common::Result<()> { // Finish any remaining buffered rows in the coalescer self.coalescer.finish_buffered_batch()?; diff --git a/native/shuffle/src/writers/shuffle_block_writer.rs b/native/shuffle/src/writers/codec.rs similarity index 86% rename from native/shuffle/src/writers/shuffle_block_writer.rs rename to native/shuffle/src/writers/codec.rs index 92e416880c..5e6dc88772 100644 --- a/native/shuffle/src/writers/shuffle_block_writer.rs +++ b/native/shuffle/src/writers/codec.rs @@ -31,9 +31,14 @@ pub enum CompressionCodec { impl CompressionCodec { pub fn ipc_write_options(&self) -> datafusion::error::Result { let compression = match self { - CompressionCodec::None | CompressionCodec::Snappy => None, + CompressionCodec::None => None, CompressionCodec::Lz4Frame => Some(CompressionType::LZ4_FRAME), CompressionCodec::Zstd(_) => Some(CompressionType::ZSTD), + CompressionCodec::Snappy => { + return Err(datafusion::common::DataFusionError::Execution( + "Snappy is not supported for Arrow IPC compression".to_string(), + )); + } }; let options = IpcWriteOptions::try_new(8, false, arrow::ipc::MetadataVersion::V5) .map_err(|e| datafusion::common::DataFusionError::ArrowError(Box::from(e), None))?; diff --git a/native/shuffle/src/writers/mod.rs b/native/shuffle/src/writers/mod.rs index 5c6fc29bee..ed57562856 100644 --- a/native/shuffle/src/writers/mod.rs +++ b/native/shuffle/src/writers/mod.rs @@ -17,10 +17,10 @@ mod buf_batch_writer; mod checksum; -mod shuffle_block_writer; +mod codec; mod spill; pub(crate) use buf_batch_writer::BufBatchWriter; pub(crate) use checksum::Checksum; -pub use shuffle_block_writer::CompressionCodec; +pub use codec::CompressionCodec; pub(crate) use spill::PartitionWriter; diff --git a/spark/src/main/java/org/apache/comet/CometShuffleBlockIterator.java b/spark/src/main/java/org/apache/comet/CometShuffleBlockIterator.java deleted file mode 100644 index 9f72b20f51..0000000000 --- a/spark/src/main/java/org/apache/comet/CometShuffleBlockIterator.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.comet; - -import java.io.Closeable; -import java.io.EOFException; -import java.io.IOException; -import java.io.InputStream; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.nio.channels.Channels; -import java.nio.channels.ReadableByteChannel; - -/** - * Provides raw compressed shuffle blocks to native code via JNI. - * - *

Reads block headers (compressed length + field count) from a shuffle InputStream and loads the - * compressed body into a DirectByteBuffer. Native code pulls blocks by calling hasNext() and - * getBuffer(). - * - *

The DirectByteBuffer returned by getBuffer() is only valid until the next hasNext() call. - * Native code must fully consume it (via read_ipc_compressed which allocates new memory for the - * decompressed data) before pulling the next block. - */ -public class CometShuffleBlockIterator implements Closeable { - - private static final int INITIAL_BUFFER_SIZE = 128 * 1024; - - private final ReadableByteChannel channel; - private final InputStream inputStream; - private final ByteBuffer headerBuf = ByteBuffer.allocate(16).order(ByteOrder.LITTLE_ENDIAN); - private ByteBuffer dataBuf = ByteBuffer.allocateDirect(INITIAL_BUFFER_SIZE); - private boolean closed = false; - private int currentBlockLength = 0; - - public CometShuffleBlockIterator(InputStream in) { - this.inputStream = in; - this.channel = Channels.newChannel(in); - } - - /** - * Reads the next block header and loads the compressed body into the internal buffer. Called by - * native code via JNI. - * - *

Header format: 8-byte compressedLength (includes field count but not itself) + 8-byte - * fieldCount (discarded, schema comes from protobuf). - * - * @return the compressed body length in bytes (codec prefix + compressed IPC), or -1 if EOF - */ - public int hasNext() throws IOException { - if (closed) { - return -1; - } - - // Read 16-byte header: clear() resets position=0, limit=capacity, - // preparing the buffer for channel.read() to fill it - headerBuf.clear(); - while (headerBuf.hasRemaining()) { - int bytesRead = channel.read(headerBuf); - if (bytesRead < 0) { - if (headerBuf.position() == 0) { - close(); - return -1; - } - throw new EOFException("Data corrupt: unexpected EOF while reading batch header"); - } - } - headerBuf.flip(); - long compressedLength = headerBuf.getLong(); - // Field count discarded - schema determined by ShuffleScan protobuf fields - headerBuf.getLong(); - - // Subtract 8 because compressedLength includes the 8-byte field count we already read - long bytesToRead = compressedLength - 8; - if (bytesToRead > Integer.MAX_VALUE) { - throw new IllegalStateException( - "Native shuffle block size of " - + bytesToRead - + " exceeds maximum of " - + Integer.MAX_VALUE - + ". Try reducing spark.comet.columnar.shuffle.batch.size."); - } - - currentBlockLength = (int) bytesToRead; - - if (dataBuf.capacity() < currentBlockLength) { - int newCapacity = (int) Math.min(bytesToRead * 2L, Integer.MAX_VALUE); - dataBuf = ByteBuffer.allocateDirect(newCapacity); - } - - dataBuf.clear(); - dataBuf.limit(currentBlockLength); - while (dataBuf.hasRemaining()) { - int bytesRead = channel.read(dataBuf); - if (bytesRead < 0) { - throw new EOFException("Data corrupt: unexpected EOF while reading compressed batch"); - } - } - // Note: native side uses get_direct_buffer_address (base pointer) + currentBlockLength, - // not the buffer's position/limit. No flip needed. - - return currentBlockLength; - } - - /** - * Returns the DirectByteBuffer containing the current block's compressed bytes (4-byte codec - * prefix + compressed IPC data). Called by native code via JNI. - */ - public ByteBuffer getBuffer() { - return dataBuf; - } - - /** Returns the length of the current block in bytes. Called by native code via JNI. */ - public int getCurrentBlockLength() { - return currentBlockLength; - } - - @Override - public void close() throws IOException { - if (!closed) { - closed = true; - inputStream.close(); - } - } -} From bca76afbb41dbebddcd45f0f9f2cde5a9b15d04b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Apr 2026 10:51:08 -0600 Subject: [PATCH 45/46] chore: remove unrelated files accidentally committed --- .../logs/comet-offheap4g-q1.log | 76 - .../logs/comet-offheap4g-q1.time | 1628 ---- .../logs/comet-offheap4g-q5.log | 317 - .../logs/comet-offheap4g-q5.time | 5639 -------------- .../logs/comet-offheap4g-q9.log | 281 - .../logs/comet-offheap4g-q9.time | 6872 ----------------- .../logs/comet-offheap8g-q1.log | 76 - .../logs/comet-offheap8g-q1.time | 1629 ---- .../logs/comet-offheap8g-q5.log | 317 - .../logs/comet-offheap8g-q5.time | 5639 -------------- .../logs/comet-offheap8g-q9.log | 281 - .../logs/comet-offheap8g-q9.time | 6868 ---------------- .../logs/spark-offheap4g-q1.log | 80 - .../logs/spark-offheap4g-q1.time | 1650 ---- .../logs/spark-offheap4g-q5.log | 330 - .../logs/spark-offheap4g-q5.time | 4441 ----------- .../logs/spark-offheap4g-q9.log | 290 - .../logs/spark-offheap4g-q9.time | 5067 ------------ .../tpc/memory-profile-results/results.csv | 4 - .../plans/2026-04-01-spill-callback.md | 782 -- ...-04-02-shuffle-ipc-stream-per-partition.md | 1438 ---- ...shuffle-ipc-stream-per-partition-design.md | 252 - 22 files changed, 43957 deletions(-) delete mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.log delete mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.time delete mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.log delete mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.time delete mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.log delete mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.time delete mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.log delete mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.time delete mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.log delete mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.time delete mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.log delete mode 100644 benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.time delete mode 100644 benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.log delete mode 100644 benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.time delete mode 100644 benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.log delete mode 100644 benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.time delete mode 100644 benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.log delete mode 100644 benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.time delete mode 100644 benchmarks/tpc/memory-profile-results/results.csv delete mode 100644 docs/superpowers/plans/2026-04-01-spill-callback.md delete mode 100644 docs/superpowers/plans/2026-04-02-shuffle-ipc-stream-per-partition.md delete mode 100644 docs/superpowers/specs/2026-04-02-shuffle-ipc-stream-per-partition-design.md diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.log b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.log deleted file mode 100644 index 1aab0a482d..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.log +++ /dev/null @@ -1,76 +0,0 @@ -Registering table customer from /opt/tpch/sf100/customer -Registering table lineitem from /opt/tpch/sf100/lineitem -Registering table nation from /opt/tpch/sf100/nation -Registering table orders from /opt/tpch/sf100/orders -Registering table part from /opt/tpch/sf100/part -Registering table partsupp from /opt/tpch/sf100/partsupp -Registering table region from /opt/tpch/sf100/region -Registering table supplier from /opt/tpch/sf100/supplier - -============================================================ -Starting iteration 1 of 1 -============================================================ - -Running query 1 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q1.sql -Executing: -- CometBench-H query 1 derived from TPC-H query 1 under the terms of the TPC Fair Use Policy. --- TP... -== Physical Plan == -AdaptiveSparkPlan (9) -+- CometSort (8) - +- CometExchange (7) - +- CometHashAggregate (6) - +- CometExchange (5) - +- CometHashAggregate (4) - +- CometProject (3) - +- CometFilter (2) - +- CometNativeScan parquet (1) - - -(1) CometNativeScan parquet -Output [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] -PushedFilters: [IsNotNull(l_shipdate), LessThanOrEqual(l_shipdate,1998-09-24)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] -Condition : (isnotnull(l_shipdate#26) AND (l_shipdate#26 <= 1998-09-24)) - -(3) CometProject -Input [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] -Arguments: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25], [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25] - -(4) CometHashAggregate -Input [6]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25] -Keys [2]: [l_returnflag#24, l_linestatus#25] -Functions [8]: [partial_sum(l_quantity#20), partial_sum(l_extendedprice#21), partial_sum((l_extendedprice#21 * (1 - l_discount#22))), partial_sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23))), partial_avg(l_quantity#20), partial_avg(l_extendedprice#21), partial_avg(l_discount#22), partial_count(1)] - -(5) CometExchange -Input [17]: [l_returnflag#24, l_linestatus#25, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, count#172L, sum#173, count#174L, sum#175, count#176L, count#177L] -Arguments: hashpartitioning(l_returnflag#24, l_linestatus#25, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=83] - -(6) CometHashAggregate -Input [17]: [l_returnflag#24, l_linestatus#25, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, count#172L, sum#173, count#174L, sum#175, count#176L, count#177L] -Keys [2]: [l_returnflag#24, l_linestatus#25] -Functions [8]: [sum(l_quantity#20), sum(l_extendedprice#21), sum((l_extendedprice#21 * (1 - l_discount#22))), sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23))), avg(l_quantity#20), avg(l_extendedprice#21), avg(l_discount#22), count(1)] - -(7) CometExchange -Input [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] -Arguments: rangepartitioning(l_returnflag#24 ASC NULLS FIRST, l_linestatus#25 ASC NULLS FIRST, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=85] - -(8) CometSort -Input [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] -Arguments: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L], [l_returnflag#24 ASC NULLS FIRST, l_linestatus#25 ASC NULLS FIRST] - -(9) AdaptiveSparkPlan -Output [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] -Arguments: isFinalPlan=false - - -Query 1 returned 4 rows, hash=6839cc802df6220ed2efdaaaa48373a6 -Query 1 took 326.87 seconds - -Iteration 1 took 326.87 seconds - -Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/comet-offheap4g-q1-tpch-1775045118244.json diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.time b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.time deleted file mode 100644 index 8eef79be16..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q1.time +++ /dev/null @@ -1,1628 +0,0 @@ -26/04/01 05:59:46 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) -26/04/01 05:59:46 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address -26/04/01 05:59:46 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable -26/04/01 05:59:47 INFO SparkContext: Running Spark version 3.5.8 -26/04/01 05:59:47 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 05:59:47 INFO SparkContext: Java version 17.0.17 -26/04/01 05:59:47 INFO ResourceUtils: ============================================================== -26/04/01 05:59:47 INFO ResourceUtils: No custom resources configured for spark.driver. -26/04/01 05:59:47 INFO ResourceUtils: ============================================================== -26/04/01 05:59:47 INFO SparkContext: Submitted application: comet-offheap4g-q1 benchmark derived from tpch -26/04/01 05:59:47 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 4096, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) -26/04/01 05:59:47 INFO ResourceProfile: Limiting resource is cpu -26/04/01 05:59:47 INFO ResourceProfileManager: Added ResourceProfile id: 0 -26/04/01 05:59:47 INFO SecurityManager: Changing view acls to: andy -26/04/01 05:59:47 INFO SecurityManager: Changing modify acls to: andy -26/04/01 05:59:47 INFO SecurityManager: Changing view acls groups to: -26/04/01 05:59:47 INFO SecurityManager: Changing modify acls groups to: -26/04/01 05:59:47 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY -26/04/01 05:59:47 INFO Utils: Successfully started service 'sparkDriver' on port 58233. -26/04/01 05:59:47 INFO SparkEnv: Registering MapOutputTracker -26/04/01 05:59:47 INFO SparkEnv: Registering BlockManagerMaster -26/04/01 05:59:47 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information -26/04/01 05:59:47 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up -26/04/01 05:59:47 INFO SparkEnv: Registering BlockManagerMasterHeartbeat -26/04/01 05:59:47 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-c8f0d8ba-25f5-42f4-99d1-8b3178ae04da -26/04/01 05:59:47 INFO MemoryStore: MemoryStore started with capacity 8.6 GiB -26/04/01 05:59:47 INFO SparkEnv: Registering OutputCommitCoordinator -26/04/01 05:59:47 INFO SparkContext: Added JAR file:///Users/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar at spark://10.0.0.133:58233/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775044787060 -26/04/01 05:59:47 INFO CometDriverPlugin: CometDriverPlugin init -26/04/01 05:59:47 INFO CometDriverPlugin: Setting spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions -26/04/01 05:59:47 INFO CometDriverPlugin: Comet metrics reporting is disabled. Set spark.comet.metrics.enabled=true to enable. -26/04/01 05:59:47 INFO CometDriverPlugin: Comet is running in unified memory mode and sharing off-heap memory with Spark -26/04/01 05:59:47 INFO DriverPluginContainer: Initialized driver component for plugin org.apache.spark.CometPlugin. -26/04/01 05:59:47 INFO Executor: Starting executor ID driver on host 10.0.0.133 -26/04/01 05:59:47 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 05:59:47 INFO Executor: Java version 17.0.17 -26/04/01 05:59:47 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' -26/04/01 05:59:47 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@1306f9cd for default. -26/04/01 05:59:47 INFO Executor: Fetching spark://10.0.0.133:58233/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775044787060 -26/04/01 05:59:47 INFO TransportClientFactory: Successfully created connection to /10.0.0.133:58233 after 9 ms (0 ms spent in bootstraps) -26/04/01 05:59:47 INFO Utils: Fetching spark://10.0.0.133:58233/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-8b7018bf-f101-4273-93fa-9cf7eb6d820a/userFiles-45b7fef5-97ad-40d7-be54-b4c80c409c73/fetchFileTemp7788925620064686833.tmp -26/04/01 05:59:47 INFO Executor: Adding file:/private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-8b7018bf-f101-4273-93fa-9cf7eb6d820a/userFiles-45b7fef5-97ad-40d7-be54-b4c80c409c73/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to class loader default -26/04/01 05:59:47 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 58235. -26/04/01 05:59:47 INFO NettyBlockTransferService: Server created on 10.0.0.133:58235 -26/04/01 05:59:47 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy -26/04/01 05:59:47 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 58235, None) -26/04/01 05:59:47 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:58235 with 8.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 58235, None) -26/04/01 05:59:47 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 58235, None) -26/04/01 05:59:47 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 58235, None) -26/04/01 05:59:47 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. -26/04/01 05:59:47 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. -26/04/01 05:59:47 INFO InMemoryFileIndex: It took 18 ms to list leaf files for 1 paths. -26/04/01 05:59:47 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:59:47 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:59:47 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:59:47 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:59:47 INFO DAGScheduler: Missing parents: List() -26/04/01 05:59:47 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:59:47 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 05:59:48 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 05:59:48 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:48 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:59:48 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:59:48 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 -26/04/01 05:59:48 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 05:59:48 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) -26/04/01 05:59:48 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 2041 bytes result sent to driver -26/04/01 05:59:48 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 127 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:59:48 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool -26/04/01 05:59:48 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.379 s -26/04/01 05:59:48 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:59:48 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished -26/04/01 05:59:48 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.389787 s -26/04/01 05:59:48 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:50 INFO NativeBase: Couldn't locate log file from either COMET_CONF_DIR or comet.log.file.path. Using default log configuration with INFO log level which emits to stderr -26/04/01 05:59:50 INFO core/src/lib.rs: Comet native library version 0.15.0 initialized -26/04/01 05:59:50 INFO NativeBase: Setting system property arrow.enable_unsafe_memory_access to true -26/04/01 05:59:50 INFO NativeBase: Setting system property arrow.enable_null_check_for_get to false -26/04/01 05:59:51 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. -26/04/01 05:59:51 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:59:51 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:59:51 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() -26/04/01 05:59:51 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:59:51 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 -26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) -26/04/01 05:59:51 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2344 bytes result sent to driver -26/04/01 05:59:51 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 15 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:59:51 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool -26/04/01 05:59:51 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.024 s -26/04/01 05:59:51 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:59:51 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished -26/04/01 05:59:51 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.024978 s -26/04/01 05:59:51 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 05:59:51 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:59:51 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:59:51 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() -26/04/01 05:59:51 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:59:51 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 -26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) -26/04/01 05:59:51 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver -26/04/01 05:59:51 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 8 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:59:51 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool -26/04/01 05:59:51 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.016 s -26/04/01 05:59:51 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:59:51 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished -26/04/01 05:59:51 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.017191 s -26/04/01 05:59:51 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. -26/04/01 05:59:51 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:59:51 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:59:51 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() -26/04/01 05:59:51 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:59:51 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 -26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) -26/04/01 05:59:51 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver -26/04/01 05:59:51 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:59:51 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool -26/04/01 05:59:51 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s -26/04/01 05:59:51 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:59:51 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished -26/04/01 05:59:51 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.015922 s -26/04/01 05:59:51 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. -26/04/01 05:59:51 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:59:51 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:59:51 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() -26/04/01 05:59:51 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:59:51 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 -26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9299 bytes) -26/04/01 05:59:51 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) -26/04/01 05:59:51 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver -26/04/01 05:59:51 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:59:51 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool -26/04/01 05:59:51 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s -26/04/01 05:59:51 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:59:51 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished -26/04/01 05:59:51 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.016757 s -26/04/01 05:59:51 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 05:59:51 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:59:51 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:59:51 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() -26/04/01 05:59:51 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:59:51 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 -26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) -26/04/01 05:59:51 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver -26/04/01 05:59:51 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 7 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:59:51 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool -26/04/01 05:59:51 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s -26/04/01 05:59:51 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:59:51 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished -26/04/01 05:59:51 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.016127 s -26/04/01 05:59:51 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. -26/04/01 05:59:51 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:59:51 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:59:51 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() -26/04/01 05:59:51 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:59:51 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 -26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) -26/04/01 05:59:51 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver -26/04/01 05:59:51 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:59:51 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool -26/04/01 05:59:51 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 05:59:51 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:59:51 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished -26/04/01 05:59:51 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.015107 s -26/04/01 05:59:51 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 05:59:51 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:59:51 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:59:51 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() -26/04/01 05:59:51 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:58235 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:59:51 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 -26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) -26/04/01 05:59:51 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver -26/04/01 05:59:51 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:59:51 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool -26/04/01 05:59:51 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 05:59:51 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:59:51 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished -26/04/01 05:59:51 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.014655 s -26/04/01 05:59:51 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_shipdate),LessThanOrEqual(l_shipdate,1998-09-24) -26/04/01 05:59:51 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_shipdate#26),(l_shipdate#26 <= 1998-09-24) -26/04/01 05:59:51 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:58235 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 05:59:51 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.5 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:58235 (size: 35.5 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:59:51 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 05:59:51 INFO DAGScheduler: Registering RDD 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 -26/04/01 05:59:51 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions -26/04/01 05:59:51 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:59:51 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:59:51 INFO DAGScheduler: Missing parents: List() -26/04/01 05:59:51 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 46.8 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 10.1 KiB, free 8.6 GiB) -26/04/01 05:59:51 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:58235 (size: 10.1 KiB, free: 8.6 GiB) -26/04/01 05:59:51 INFO SparkContext: Created broadcast 9 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:59:51 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 05:59:51 INFO TaskSchedulerImpl: Adding task set 8.0 with 208 tasks resource profile 0 -26/04/01 05:59:51 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) -26/04/01 05:59:51 INFO TaskSetManager: Starting task 1.0 in stage 8.0 (TID 9) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9274 bytes) -26/04/01 05:59:51 INFO TaskSetManager: Starting task 2.0 in stage 8.0 (TID 10) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9275 bytes) -26/04/01 05:59:51 INFO TaskSetManager: Starting task 3.0 in stage 8.0 (TID 11) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9275 bytes) -26/04/01 05:59:51 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) -26/04/01 05:59:51 INFO Executor: Running task 1.0 in stage 8.0 (TID 9) -26/04/01 05:59:51 INFO Executor: Running task 3.0 in stage 8.0 (TID 11) -26/04/01 05:59:51 INFO Executor: Running task 2.0 in stage 8.0 (TID 10) -26/04/01 05:59:51 INFO BaseAllocator: Debug mode disabled. Enable with the VM option -Darrow.memory.debug.allocator=true. -26/04/01 05:59:51 INFO DefaultAllocationManagerOption: allocation manager type not specified, using netty as the default type -26/04/01 05:59:51 INFO CheckAllocator: Using DefaultAllocationManager at memory/unsafe/DefaultAllocationManagerFactory.class -26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 05:59:51 INFO core/src/execution/jni_api.rs: Comet tokio runtime: using spark.executor.cores=4 worker threads -26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 05:59:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 05:59:58 INFO Executor: Finished task 1.0 in stage 8.0 (TID 9). 6830 bytes result sent to driver -26/04/01 05:59:58 INFO Executor: Finished task 2.0 in stage 8.0 (TID 10). 6830 bytes result sent to driver -26/04/01 05:59:58 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 6830 bytes result sent to driver -26/04/01 05:59:58 INFO Executor: Finished task 3.0 in stage 8.0 (TID 11). 6830 bytes result sent to driver -26/04/01 05:59:58 INFO TaskSetManager: Starting task 4.0 in stage 8.0 (TID 12) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9275 bytes) -26/04/01 05:59:58 INFO Executor: Running task 4.0 in stage 8.0 (TID 12) -26/04/01 05:59:58 INFO TaskSetManager: Starting task 5.0 in stage 8.0 (TID 13) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9275 bytes) -26/04/01 05:59:58 INFO TaskSetManager: Finished task 1.0 in stage 8.0 (TID 9) in 6754 ms on 10.0.0.133 (executor driver) (1/208) -26/04/01 05:59:58 INFO TaskSetManager: Finished task 2.0 in stage 8.0 (TID 10) in 6754 ms on 10.0.0.133 (executor driver) (2/208) -26/04/01 05:59:58 INFO Executor: Running task 5.0 in stage 8.0 (TID 13) -26/04/01 05:59:58 INFO TaskSetManager: Starting task 6.0 in stage 8.0 (TID 14) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9269 bytes) -26/04/01 05:59:58 INFO Executor: Running task 6.0 in stage 8.0 (TID 14) -26/04/01 05:59:58 INFO TaskSetManager: Starting task 7.0 in stage 8.0 (TID 15) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9274 bytes) -26/04/01 05:59:58 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 6756 ms on 10.0.0.133 (executor driver) (3/208) -26/04/01 05:59:58 INFO Executor: Running task 7.0 in stage 8.0 (TID 15) -26/04/01 05:59:58 INFO TaskSetManager: Finished task 3.0 in stage 8.0 (TID 11) in 6756 ms on 10.0.0.133 (executor driver) (4/208) -26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 05:59:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:05 INFO Executor: Finished task 4.0 in stage 8.0 (TID 12). 6830 bytes result sent to driver -26/04/01 06:00:05 INFO Executor: Finished task 6.0 in stage 8.0 (TID 14). 6830 bytes result sent to driver -26/04/01 06:00:05 INFO Executor: Finished task 7.0 in stage 8.0 (TID 15). 6830 bytes result sent to driver -26/04/01 06:00:05 INFO TaskSetManager: Starting task 8.0 in stage 8.0 (TID 16) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:05 INFO Executor: Finished task 5.0 in stage 8.0 (TID 13). 6830 bytes result sent to driver -26/04/01 06:00:05 INFO Executor: Running task 8.0 in stage 8.0 (TID 16) -26/04/01 06:00:05 INFO TaskSetManager: Starting task 9.0 in stage 8.0 (TID 17) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:05 INFO Executor: Running task 9.0 in stage 8.0 (TID 17) -26/04/01 06:00:05 INFO TaskSetManager: Starting task 10.0 in stage 8.0 (TID 18) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:05 INFO TaskSetManager: Finished task 4.0 in stage 8.0 (TID 12) in 6489 ms on 10.0.0.133 (executor driver) (5/208) -26/04/01 06:00:05 INFO Executor: Running task 10.0 in stage 8.0 (TID 18) -26/04/01 06:00:05 INFO TaskSetManager: Starting task 11.0 in stage 8.0 (TID 19) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:05 INFO TaskSetManager: Finished task 6.0 in stage 8.0 (TID 14) in 6489 ms on 10.0.0.133 (executor driver) (6/208) -26/04/01 06:00:05 INFO Executor: Running task 11.0 in stage 8.0 (TID 19) -26/04/01 06:00:05 INFO TaskSetManager: Finished task 7.0 in stage 8.0 (TID 15) in 6488 ms on 10.0.0.133 (executor driver) (7/208) -26/04/01 06:00:05 INFO TaskSetManager: Finished task 5.0 in stage 8.0 (TID 13) in 6490 ms on 10.0.0.133 (executor driver) (8/208) -26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:11 INFO Executor: Finished task 9.0 in stage 8.0 (TID 17). 6830 bytes result sent to driver -26/04/01 06:00:11 INFO TaskSetManager: Starting task 12.0 in stage 8.0 (TID 20) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:00:11 INFO Executor: Running task 12.0 in stage 8.0 (TID 20) -26/04/01 06:00:11 INFO TaskSetManager: Finished task 9.0 in stage 8.0 (TID 17) in 6468 ms on 10.0.0.133 (executor driver) (9/208) -26/04/01 06:00:11 INFO Executor: Finished task 8.0 in stage 8.0 (TID 16). 6830 bytes result sent to driver -26/04/01 06:00:11 INFO TaskSetManager: Starting task 13.0 in stage 8.0 (TID 21) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:00:11 INFO TaskSetManager: Finished task 8.0 in stage 8.0 (TID 16) in 6470 ms on 10.0.0.133 (executor driver) (10/208) -26/04/01 06:00:11 INFO Executor: Running task 13.0 in stage 8.0 (TID 21) -26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:11 INFO Executor: Finished task 10.0 in stage 8.0 (TID 18). 6830 bytes result sent to driver -26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:11 INFO TaskSetManager: Starting task 14.0 in stage 8.0 (TID 22) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:11 INFO Executor: Running task 14.0 in stage 8.0 (TID 22) -26/04/01 06:00:11 INFO TaskSetManager: Finished task 10.0 in stage 8.0 (TID 18) in 6472 ms on 10.0.0.133 (executor driver) (11/208) -26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:11 INFO Executor: Finished task 11.0 in stage 8.0 (TID 19). 6830 bytes result sent to driver -26/04/01 06:00:11 INFO TaskSetManager: Starting task 15.0 in stage 8.0 (TID 23) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:11 INFO Executor: Running task 15.0 in stage 8.0 (TID 23) -26/04/01 06:00:11 INFO TaskSetManager: Finished task 11.0 in stage 8.0 (TID 19) in 6473 ms on 10.0.0.133 (executor driver) (12/208) -26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:17 INFO Executor: Finished task 14.0 in stage 8.0 (TID 22). 6787 bytes result sent to driver -26/04/01 06:00:17 INFO TaskSetManager: Starting task 16.0 in stage 8.0 (TID 24) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:17 INFO Executor: Running task 16.0 in stage 8.0 (TID 24) -26/04/01 06:00:17 INFO TaskSetManager: Finished task 14.0 in stage 8.0 (TID 22) in 6440 ms on 10.0.0.133 (executor driver) (13/208) -26/04/01 06:00:17 INFO Executor: Finished task 15.0 in stage 8.0 (TID 23). 6830 bytes result sent to driver -26/04/01 06:00:17 INFO TaskSetManager: Starting task 17.0 in stage 8.0 (TID 25) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:17 INFO Executor: Running task 17.0 in stage 8.0 (TID 25) -26/04/01 06:00:17 INFO TaskSetManager: Finished task 15.0 in stage 8.0 (TID 23) in 6439 ms on 10.0.0.133 (executor driver) (14/208) -26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:17 INFO Executor: Finished task 13.0 in stage 8.0 (TID 21). 6787 bytes result sent to driver -26/04/01 06:00:17 INFO TaskSetManager: Starting task 18.0 in stage 8.0 (TID 26) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:17 INFO TaskSetManager: Finished task 13.0 in stage 8.0 (TID 21) in 6447 ms on 10.0.0.133 (executor driver) (15/208) -26/04/01 06:00:17 INFO Executor: Running task 18.0 in stage 8.0 (TID 26) -26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:17 INFO Executor: Finished task 12.0 in stage 8.0 (TID 20). 6787 bytes result sent to driver -26/04/01 06:00:17 INFO TaskSetManager: Starting task 19.0 in stage 8.0 (TID 27) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:00:17 INFO Executor: Running task 19.0 in stage 8.0 (TID 27) -26/04/01 06:00:17 INFO TaskSetManager: Finished task 12.0 in stage 8.0 (TID 20) in 6454 ms on 10.0.0.133 (executor driver) (16/208) -26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:24 INFO Executor: Finished task 18.0 in stage 8.0 (TID 26). 6787 bytes result sent to driver -26/04/01 06:00:24 INFO TaskSetManager: Starting task 20.0 in stage 8.0 (TID 28) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:00:24 INFO Executor: Running task 20.0 in stage 8.0 (TID 28) -26/04/01 06:00:24 INFO TaskSetManager: Finished task 18.0 in stage 8.0 (TID 26) in 6430 ms on 10.0.0.133 (executor driver) (17/208) -26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:24 INFO Executor: Finished task 19.0 in stage 8.0 (TID 27). 6787 bytes result sent to driver -26/04/01 06:00:24 INFO TaskSetManager: Starting task 21.0 in stage 8.0 (TID 29) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:00:24 INFO Executor: Running task 21.0 in stage 8.0 (TID 29) -26/04/01 06:00:24 INFO TaskSetManager: Finished task 19.0 in stage 8.0 (TID 27) in 6431 ms on 10.0.0.133 (executor driver) (18/208) -26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:24 INFO Executor: Finished task 17.0 in stage 8.0 (TID 25). 6787 bytes result sent to driver -26/04/01 06:00:24 INFO TaskSetManager: Starting task 22.0 in stage 8.0 (TID 30) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:00:24 INFO TaskSetManager: Finished task 17.0 in stage 8.0 (TID 25) in 6442 ms on 10.0.0.133 (executor driver) (19/208) -26/04/01 06:00:24 INFO Executor: Running task 22.0 in stage 8.0 (TID 30) -26/04/01 06:00:24 INFO Executor: Finished task 16.0 in stage 8.0 (TID 24). 6787 bytes result sent to driver -26/04/01 06:00:24 INFO TaskSetManager: Starting task 23.0 in stage 8.0 (TID 31) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:00:24 INFO Executor: Running task 23.0 in stage 8.0 (TID 31) -26/04/01 06:00:24 INFO TaskSetManager: Finished task 16.0 in stage 8.0 (TID 24) in 6446 ms on 10.0.0.133 (executor driver) (20/208) -26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:30 INFO Executor: Finished task 23.0 in stage 8.0 (TID 31). 6787 bytes result sent to driver -26/04/01 06:00:30 INFO TaskSetManager: Starting task 24.0 in stage 8.0 (TID 32) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:00:30 INFO Executor: Running task 24.0 in stage 8.0 (TID 32) -26/04/01 06:00:30 INFO TaskSetManager: Finished task 23.0 in stage 8.0 (TID 31) in 6471 ms on 10.0.0.133 (executor driver) (21/208) -26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:30 INFO Executor: Finished task 22.0 in stage 8.0 (TID 30). 6873 bytes result sent to driver -26/04/01 06:00:30 INFO TaskSetManager: Starting task 25.0 in stage 8.0 (TID 33) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:00:30 INFO Executor: Running task 25.0 in stage 8.0 (TID 33) -26/04/01 06:00:30 INFO Executor: Finished task 21.0 in stage 8.0 (TID 29). 6830 bytes result sent to driver -26/04/01 06:00:30 INFO TaskSetManager: Finished task 22.0 in stage 8.0 (TID 30) in 6478 ms on 10.0.0.133 (executor driver) (22/208) -26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:30 INFO Executor: Finished task 20.0 in stage 8.0 (TID 28). 6830 bytes result sent to driver -26/04/01 06:00:30 INFO TaskSetManager: Starting task 26.0 in stage 8.0 (TID 34) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:30 INFO Executor: Running task 26.0 in stage 8.0 (TID 34) -26/04/01 06:00:30 INFO TaskSetManager: Starting task 27.0 in stage 8.0 (TID 35) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:30 INFO TaskSetManager: Finished task 21.0 in stage 8.0 (TID 29) in 6486 ms on 10.0.0.133 (executor driver) (23/208) -26/04/01 06:00:30 INFO Executor: Running task 27.0 in stage 8.0 (TID 35) -26/04/01 06:00:30 INFO TaskSetManager: Finished task 20.0 in stage 8.0 (TID 28) in 6494 ms on 10.0.0.133 (executor driver) (24/208) -26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:37 INFO Executor: Finished task 24.0 in stage 8.0 (TID 32). 6830 bytes result sent to driver -26/04/01 06:00:37 INFO TaskSetManager: Starting task 28.0 in stage 8.0 (TID 36) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:37 INFO Executor: Running task 28.0 in stage 8.0 (TID 36) -26/04/01 06:00:37 INFO TaskSetManager: Finished task 24.0 in stage 8.0 (TID 32) in 6421 ms on 10.0.0.133 (executor driver) (25/208) -26/04/01 06:00:37 INFO Executor: Finished task 27.0 in stage 8.0 (TID 35). 6787 bytes result sent to driver -26/04/01 06:00:37 INFO TaskSetManager: Starting task 29.0 in stage 8.0 (TID 37) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:37 INFO Executor: Running task 29.0 in stage 8.0 (TID 37) -26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:37 INFO TaskSetManager: Finished task 27.0 in stage 8.0 (TID 35) in 6417 ms on 10.0.0.133 (executor driver) (26/208) -26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:37 INFO Executor: Finished task 26.0 in stage 8.0 (TID 34). 6873 bytes result sent to driver -26/04/01 06:00:37 INFO TaskSetManager: Starting task 30.0 in stage 8.0 (TID 38) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:00:37 INFO Executor: Running task 30.0 in stage 8.0 (TID 38) -26/04/01 06:00:37 INFO TaskSetManager: Finished task 26.0 in stage 8.0 (TID 34) in 6424 ms on 10.0.0.133 (executor driver) (27/208) -26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:37 INFO Executor: Finished task 25.0 in stage 8.0 (TID 33). 6873 bytes result sent to driver -26/04/01 06:00:37 INFO TaskSetManager: Starting task 31.0 in stage 8.0 (TID 39) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:00:37 INFO TaskSetManager: Finished task 25.0 in stage 8.0 (TID 33) in 6429 ms on 10.0.0.133 (executor driver) (28/208) -26/04/01 06:00:37 INFO Executor: Running task 31.0 in stage 8.0 (TID 39) -26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:43 INFO Executor: Finished task 28.0 in stage 8.0 (TID 36). 6787 bytes result sent to driver -26/04/01 06:00:43 INFO TaskSetManager: Starting task 32.0 in stage 8.0 (TID 40) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:43 INFO Executor: Running task 32.0 in stage 8.0 (TID 40) -26/04/01 06:00:43 INFO TaskSetManager: Finished task 28.0 in stage 8.0 (TID 36) in 6372 ms on 10.0.0.133 (executor driver) (29/208) -26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:43 INFO Executor: Finished task 30.0 in stage 8.0 (TID 38). 6787 bytes result sent to driver -26/04/01 06:00:43 INFO TaskSetManager: Starting task 33.0 in stage 8.0 (TID 41) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:43 INFO Executor: Running task 33.0 in stage 8.0 (TID 41) -26/04/01 06:00:43 INFO TaskSetManager: Finished task 30.0 in stage 8.0 (TID 38) in 6367 ms on 10.0.0.133 (executor driver) (30/208) -26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:43 INFO Executor: Finished task 31.0 in stage 8.0 (TID 39). 6787 bytes result sent to driver -26/04/01 06:00:43 INFO TaskSetManager: Starting task 34.0 in stage 8.0 (TID 42) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:43 INFO TaskSetManager: Finished task 31.0 in stage 8.0 (TID 39) in 6372 ms on 10.0.0.133 (executor driver) (31/208) -26/04/01 06:00:43 INFO Executor: Running task 34.0 in stage 8.0 (TID 42) -26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:43 INFO Executor: Finished task 29.0 in stage 8.0 (TID 37). 6787 bytes result sent to driver -26/04/01 06:00:43 INFO TaskSetManager: Starting task 35.0 in stage 8.0 (TID 43) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:43 INFO Executor: Running task 35.0 in stage 8.0 (TID 43) -26/04/01 06:00:43 INFO TaskSetManager: Finished task 29.0 in stage 8.0 (TID 37) in 6388 ms on 10.0.0.133 (executor driver) (32/208) -26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:50 INFO Executor: Finished task 33.0 in stage 8.0 (TID 41). 6787 bytes result sent to driver -26/04/01 06:00:50 INFO TaskSetManager: Starting task 36.0 in stage 8.0 (TID 44) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:00:50 INFO TaskSetManager: Finished task 33.0 in stage 8.0 (TID 41) in 6375 ms on 10.0.0.133 (executor driver) (33/208) -26/04/01 06:00:50 INFO Executor: Running task 36.0 in stage 8.0 (TID 44) -26/04/01 06:00:50 INFO Executor: Finished task 32.0 in stage 8.0 (TID 40). 6787 bytes result sent to driver -26/04/01 06:00:50 INFO TaskSetManager: Starting task 37.0 in stage 8.0 (TID 45) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:00:50 INFO Executor: Running task 37.0 in stage 8.0 (TID 45) -26/04/01 06:00:50 INFO TaskSetManager: Finished task 32.0 in stage 8.0 (TID 40) in 6379 ms on 10.0.0.133 (executor driver) (34/208) -26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:50 INFO Executor: Finished task 34.0 in stage 8.0 (TID 42). 6787 bytes result sent to driver -26/04/01 06:00:50 INFO TaskSetManager: Starting task 38.0 in stage 8.0 (TID 46) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:50 INFO Executor: Running task 38.0 in stage 8.0 (TID 46) -26/04/01 06:00:50 INFO TaskSetManager: Finished task 34.0 in stage 8.0 (TID 42) in 6380 ms on 10.0.0.133 (executor driver) (35/208) -26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:50 INFO Executor: Finished task 35.0 in stage 8.0 (TID 43). 6787 bytes result sent to driver -26/04/01 06:00:50 INFO TaskSetManager: Starting task 39.0 in stage 8.0 (TID 47) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:50 INFO TaskSetManager: Finished task 35.0 in stage 8.0 (TID 43) in 6377 ms on 10.0.0.133 (executor driver) (36/208) -26/04/01 06:00:50 INFO Executor: Running task 39.0 in stage 8.0 (TID 47) -26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:56 INFO Executor: Finished task 36.0 in stage 8.0 (TID 44). 6787 bytes result sent to driver -26/04/01 06:00:56 INFO TaskSetManager: Starting task 40.0 in stage 8.0 (TID 48) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:56 INFO Executor: Running task 40.0 in stage 8.0 (TID 48) -26/04/01 06:00:56 INFO TaskSetManager: Finished task 36.0 in stage 8.0 (TID 44) in 6459 ms on 10.0.0.133 (executor driver) (37/208) -26/04/01 06:00:56 INFO Executor: Finished task 37.0 in stage 8.0 (TID 45). 6787 bytes result sent to driver -26/04/01 06:00:56 INFO TaskSetManager: Starting task 41.0 in stage 8.0 (TID 49) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:00:56 INFO TaskSetManager: Finished task 37.0 in stage 8.0 (TID 45) in 6459 ms on 10.0.0.133 (executor driver) (38/208) -26/04/01 06:00:56 INFO Executor: Running task 41.0 in stage 8.0 (TID 49) -26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:56 INFO Executor: Finished task 38.0 in stage 8.0 (TID 46). 6787 bytes result sent to driver -26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:56 INFO TaskSetManager: Starting task 42.0 in stage 8.0 (TID 50) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:00:56 INFO TaskSetManager: Finished task 38.0 in stage 8.0 (TID 46) in 6447 ms on 10.0.0.133 (executor driver) (39/208) -26/04/01 06:00:56 INFO Executor: Running task 42.0 in stage 8.0 (TID 50) -26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:56 INFO Executor: Finished task 39.0 in stage 8.0 (TID 47). 6787 bytes result sent to driver -26/04/01 06:00:56 INFO TaskSetManager: Starting task 43.0 in stage 8.0 (TID 51) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:00:56 INFO TaskSetManager: Finished task 39.0 in stage 8.0 (TID 47) in 6447 ms on 10.0.0.133 (executor driver) (40/208) -26/04/01 06:00:56 INFO Executor: Running task 43.0 in stage 8.0 (TID 51) -26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:00:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:02 INFO Executor: Finished task 42.0 in stage 8.0 (TID 50). 6787 bytes result sent to driver -26/04/01 06:01:02 INFO TaskSetManager: Starting task 44.0 in stage 8.0 (TID 52) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:02 INFO Executor: Running task 44.0 in stage 8.0 (TID 52) -26/04/01 06:01:02 INFO TaskSetManager: Finished task 42.0 in stage 8.0 (TID 50) in 6390 ms on 10.0.0.133 (executor driver) (41/208) -26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:02 INFO Executor: Finished task 43.0 in stage 8.0 (TID 51). 6787 bytes result sent to driver -26/04/01 06:01:02 INFO TaskSetManager: Starting task 45.0 in stage 8.0 (TID 53) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:02 INFO Executor: Running task 45.0 in stage 8.0 (TID 53) -26/04/01 06:01:02 INFO TaskSetManager: Finished task 43.0 in stage 8.0 (TID 51) in 6390 ms on 10.0.0.133 (executor driver) (42/208) -26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:02 INFO Executor: Finished task 41.0 in stage 8.0 (TID 49). 6787 bytes result sent to driver -26/04/01 06:01:02 INFO Executor: Finished task 40.0 in stage 8.0 (TID 48). 6787 bytes result sent to driver -26/04/01 06:01:02 INFO TaskSetManager: Starting task 46.0 in stage 8.0 (TID 54) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:02 INFO TaskSetManager: Finished task 41.0 in stage 8.0 (TID 49) in 6439 ms on 10.0.0.133 (executor driver) (43/208) -26/04/01 06:01:02 INFO Executor: Running task 46.0 in stage 8.0 (TID 54) -26/04/01 06:01:02 INFO TaskSetManager: Starting task 47.0 in stage 8.0 (TID 55) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:02 INFO Executor: Running task 47.0 in stage 8.0 (TID 55) -26/04/01 06:01:02 INFO TaskSetManager: Finished task 40.0 in stage 8.0 (TID 48) in 6440 ms on 10.0.0.133 (executor driver) (44/208) -26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:09 INFO Executor: Finished task 45.0 in stage 8.0 (TID 53). 6787 bytes result sent to driver -26/04/01 06:01:09 INFO TaskSetManager: Starting task 48.0 in stage 8.0 (TID 56) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:01:09 INFO TaskSetManager: Finished task 45.0 in stage 8.0 (TID 53) in 6151 ms on 10.0.0.133 (executor driver) (45/208) -26/04/01 06:01:09 INFO Executor: Running task 48.0 in stage 8.0 (TID 56) -26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:09 INFO Executor: Finished task 47.0 in stage 8.0 (TID 55). 6787 bytes result sent to driver -26/04/01 06:01:09 INFO TaskSetManager: Starting task 49.0 in stage 8.0 (TID 57) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:09 INFO TaskSetManager: Finished task 47.0 in stage 8.0 (TID 55) in 6395 ms on 10.0.0.133 (executor driver) (46/208) -26/04/01 06:01:09 INFO Executor: Running task 49.0 in stage 8.0 (TID 57) -26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:09 INFO Executor: Finished task 46.0 in stage 8.0 (TID 54). 6787 bytes result sent to driver -26/04/01 06:01:09 INFO TaskSetManager: Starting task 50.0 in stage 8.0 (TID 58) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:01:09 INFO TaskSetManager: Finished task 46.0 in stage 8.0 (TID 54) in 6403 ms on 10.0.0.133 (executor driver) (47/208) -26/04/01 06:01:09 INFO Executor: Running task 50.0 in stage 8.0 (TID 58) -26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:09 INFO Executor: Finished task 44.0 in stage 8.0 (TID 52). 6787 bytes result sent to driver -26/04/01 06:01:09 INFO TaskSetManager: Starting task 51.0 in stage 8.0 (TID 59) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:01:09 INFO Executor: Running task 51.0 in stage 8.0 (TID 59) -26/04/01 06:01:09 INFO TaskSetManager: Finished task 44.0 in stage 8.0 (TID 52) in 6648 ms on 10.0.0.133 (executor driver) (48/208) -26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:15 INFO Executor: Finished task 48.0 in stage 8.0 (TID 56). 6787 bytes result sent to driver -26/04/01 06:01:15 INFO TaskSetManager: Starting task 52.0 in stage 8.0 (TID 60) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:01:15 INFO TaskSetManager: Finished task 48.0 in stage 8.0 (TID 56) in 6440 ms on 10.0.0.133 (executor driver) (49/208) -26/04/01 06:01:15 INFO Executor: Running task 52.0 in stage 8.0 (TID 60) -26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:15 INFO Executor: Finished task 49.0 in stage 8.0 (TID 57). 6787 bytes result sent to driver -26/04/01 06:01:15 INFO TaskSetManager: Starting task 53.0 in stage 8.0 (TID 61) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:01:15 INFO Executor: Running task 53.0 in stage 8.0 (TID 61) -26/04/01 06:01:15 INFO TaskSetManager: Finished task 49.0 in stage 8.0 (TID 57) in 6413 ms on 10.0.0.133 (executor driver) (50/208) -26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:15 INFO Executor: Finished task 50.0 in stage 8.0 (TID 58). 6787 bytes result sent to driver -26/04/01 06:01:15 INFO TaskSetManager: Starting task 54.0 in stage 8.0 (TID 62) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:01:15 INFO TaskSetManager: Finished task 50.0 in stage 8.0 (TID 58) in 6422 ms on 10.0.0.133 (executor driver) (51/208) -26/04/01 06:01:15 INFO Executor: Running task 54.0 in stage 8.0 (TID 62) -26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:15 INFO Executor: Finished task 51.0 in stage 8.0 (TID 59). 6787 bytes result sent to driver -26/04/01 06:01:15 INFO TaskSetManager: Starting task 55.0 in stage 8.0 (TID 63) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:15 INFO Executor: Running task 55.0 in stage 8.0 (TID 63) -26/04/01 06:01:15 INFO TaskSetManager: Finished task 51.0 in stage 8.0 (TID 59) in 6444 ms on 10.0.0.133 (executor driver) (52/208) -26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:21 INFO Executor: Finished task 52.0 in stage 8.0 (TID 60). 6787 bytes result sent to driver -26/04/01 06:01:21 INFO TaskSetManager: Starting task 56.0 in stage 8.0 (TID 64) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:01:21 INFO TaskSetManager: Finished task 52.0 in stage 8.0 (TID 60) in 6472 ms on 10.0.0.133 (executor driver) (53/208) -26/04/01 06:01:21 INFO Executor: Running task 56.0 in stage 8.0 (TID 64) -26/04/01 06:01:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:22 INFO Executor: Finished task 53.0 in stage 8.0 (TID 61). 6787 bytes result sent to driver -26/04/01 06:01:22 INFO TaskSetManager: Starting task 57.0 in stage 8.0 (TID 65) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:01:22 INFO TaskSetManager: Finished task 53.0 in stage 8.0 (TID 61) in 6457 ms on 10.0.0.133 (executor driver) (54/208) -26/04/01 06:01:22 INFO Executor: Running task 57.0 in stage 8.0 (TID 65) -26/04/01 06:01:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:22 INFO Executor: Finished task 54.0 in stage 8.0 (TID 62). 6787 bytes result sent to driver -26/04/01 06:01:22 INFO TaskSetManager: Starting task 58.0 in stage 8.0 (TID 66) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:01:22 INFO TaskSetManager: Finished task 54.0 in stage 8.0 (TID 62) in 6461 ms on 10.0.0.133 (executor driver) (55/208) -26/04/01 06:01:22 INFO Executor: Running task 58.0 in stage 8.0 (TID 66) -26/04/01 06:01:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:22 INFO Executor: Finished task 55.0 in stage 8.0 (TID 63). 6787 bytes result sent to driver -26/04/01 06:01:22 INFO TaskSetManager: Starting task 59.0 in stage 8.0 (TID 67) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:01:22 INFO Executor: Running task 59.0 in stage 8.0 (TID 67) -26/04/01 06:01:22 INFO TaskSetManager: Finished task 55.0 in stage 8.0 (TID 63) in 6469 ms on 10.0.0.133 (executor driver) (56/208) -26/04/01 06:01:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:28 INFO Executor: Finished task 56.0 in stage 8.0 (TID 64). 6787 bytes result sent to driver -26/04/01 06:01:28 INFO TaskSetManager: Starting task 60.0 in stage 8.0 (TID 68) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:01:28 INFO Executor: Running task 60.0 in stage 8.0 (TID 68) -26/04/01 06:01:28 INFO TaskSetManager: Finished task 56.0 in stage 8.0 (TID 64) in 6478 ms on 10.0.0.133 (executor driver) (57/208) -26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:28 INFO Executor: Finished task 57.0 in stage 8.0 (TID 65). 6787 bytes result sent to driver -26/04/01 06:01:28 INFO TaskSetManager: Starting task 61.0 in stage 8.0 (TID 69) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:28 INFO TaskSetManager: Finished task 57.0 in stage 8.0 (TID 65) in 6477 ms on 10.0.0.133 (executor driver) (58/208) -26/04/01 06:01:28 INFO Executor: Running task 61.0 in stage 8.0 (TID 69) -26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:28 INFO Executor: Finished task 58.0 in stage 8.0 (TID 66). 6787 bytes result sent to driver -26/04/01 06:01:28 INFO TaskSetManager: Starting task 62.0 in stage 8.0 (TID 70) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:01:28 INFO Executor: Running task 62.0 in stage 8.0 (TID 70) -26/04/01 06:01:28 INFO TaskSetManager: Finished task 58.0 in stage 8.0 (TID 66) in 6458 ms on 10.0.0.133 (executor driver) (59/208) -26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:28 INFO Executor: Finished task 59.0 in stage 8.0 (TID 67). 6787 bytes result sent to driver -26/04/01 06:01:28 INFO TaskSetManager: Starting task 63.0 in stage 8.0 (TID 71) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:01:28 INFO TaskSetManager: Finished task 59.0 in stage 8.0 (TID 67) in 6477 ms on 10.0.0.133 (executor driver) (60/208) -26/04/01 06:01:28 INFO Executor: Running task 63.0 in stage 8.0 (TID 71) -26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:34 INFO Executor: Finished task 60.0 in stage 8.0 (TID 68). 6787 bytes result sent to driver -26/04/01 06:01:34 INFO TaskSetManager: Starting task 64.0 in stage 8.0 (TID 72) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:01:34 INFO TaskSetManager: Finished task 60.0 in stage 8.0 (TID 68) in 6400 ms on 10.0.0.133 (executor driver) (61/208) -26/04/01 06:01:34 INFO Executor: Running task 64.0 in stage 8.0 (TID 72) -26/04/01 06:01:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:35 INFO Executor: Finished task 61.0 in stage 8.0 (TID 69). 6787 bytes result sent to driver -26/04/01 06:01:35 INFO TaskSetManager: Starting task 65.0 in stage 8.0 (TID 73) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:01:35 INFO Executor: Running task 65.0 in stage 8.0 (TID 73) -26/04/01 06:01:35 INFO TaskSetManager: Finished task 61.0 in stage 8.0 (TID 69) in 6392 ms on 10.0.0.133 (executor driver) (62/208) -26/04/01 06:01:35 INFO Executor: Finished task 62.0 in stage 8.0 (TID 70). 6787 bytes result sent to driver -26/04/01 06:01:35 INFO TaskSetManager: Starting task 66.0 in stage 8.0 (TID 74) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:01:35 INFO Executor: Running task 66.0 in stage 8.0 (TID 74) -26/04/01 06:01:35 INFO TaskSetManager: Finished task 62.0 in stage 8.0 (TID 70) in 6391 ms on 10.0.0.133 (executor driver) (63/208) -26/04/01 06:01:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:35 INFO Executor: Finished task 63.0 in stage 8.0 (TID 71). 6787 bytes result sent to driver -26/04/01 06:01:35 INFO TaskSetManager: Starting task 67.0 in stage 8.0 (TID 75) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:01:35 INFO TaskSetManager: Finished task 63.0 in stage 8.0 (TID 71) in 6392 ms on 10.0.0.133 (executor driver) (64/208) -26/04/01 06:01:35 INFO Executor: Running task 67.0 in stage 8.0 (TID 75) -26/04/01 06:01:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:41 INFO Executor: Finished task 64.0 in stage 8.0 (TID 72). 6787 bytes result sent to driver -26/04/01 06:01:41 INFO TaskSetManager: Starting task 68.0 in stage 8.0 (TID 76) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:41 INFO Executor: Running task 68.0 in stage 8.0 (TID 76) -26/04/01 06:01:41 INFO TaskSetManager: Finished task 64.0 in stage 8.0 (TID 72) in 6387 ms on 10.0.0.133 (executor driver) (65/208) -26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:41 INFO Executor: Finished task 65.0 in stage 8.0 (TID 73). 6787 bytes result sent to driver -26/04/01 06:01:41 INFO Executor: Finished task 66.0 in stage 8.0 (TID 74). 6787 bytes result sent to driver -26/04/01 06:01:41 INFO TaskSetManager: Starting task 69.0 in stage 8.0 (TID 77) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:41 INFO Executor: Running task 69.0 in stage 8.0 (TID 77) -26/04/01 06:01:41 INFO TaskSetManager: Starting task 70.0 in stage 8.0 (TID 78) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:41 INFO TaskSetManager: Finished task 65.0 in stage 8.0 (TID 73) in 6410 ms on 10.0.0.133 (executor driver) (66/208) -26/04/01 06:01:41 INFO Executor: Running task 70.0 in stage 8.0 (TID 78) -26/04/01 06:01:41 INFO TaskSetManager: Finished task 66.0 in stage 8.0 (TID 74) in 6411 ms on 10.0.0.133 (executor driver) (67/208) -26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:41 INFO Executor: Finished task 67.0 in stage 8.0 (TID 75). 6787 bytes result sent to driver -26/04/01 06:01:41 INFO TaskSetManager: Starting task 71.0 in stage 8.0 (TID 79) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:41 INFO Executor: Running task 71.0 in stage 8.0 (TID 79) -26/04/01 06:01:41 INFO TaskSetManager: Finished task 67.0 in stage 8.0 (TID 75) in 6389 ms on 10.0.0.133 (executor driver) (68/208) -26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:47 INFO Executor: Finished task 68.0 in stage 8.0 (TID 76). 6787 bytes result sent to driver -26/04/01 06:01:47 INFO TaskSetManager: Starting task 72.0 in stage 8.0 (TID 80) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:01:47 INFO Executor: Running task 72.0 in stage 8.0 (TID 80) -26/04/01 06:01:47 INFO TaskSetManager: Finished task 68.0 in stage 8.0 (TID 76) in 6396 ms on 10.0.0.133 (executor driver) (69/208) -26/04/01 06:01:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:47 INFO Executor: Finished task 69.0 in stage 8.0 (TID 77). 6787 bytes result sent to driver -26/04/01 06:01:47 INFO TaskSetManager: Starting task 73.0 in stage 8.0 (TID 81) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:01:47 INFO TaskSetManager: Finished task 69.0 in stage 8.0 (TID 77) in 6392 ms on 10.0.0.133 (executor driver) (70/208) -26/04/01 06:01:47 INFO Executor: Running task 73.0 in stage 8.0 (TID 81) -26/04/01 06:01:47 INFO Executor: Finished task 70.0 in stage 8.0 (TID 78). 6787 bytes result sent to driver -26/04/01 06:01:47 INFO TaskSetManager: Starting task 74.0 in stage 8.0 (TID 82) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:47 INFO Executor: Running task 74.0 in stage 8.0 (TID 82) -26/04/01 06:01:47 INFO TaskSetManager: Finished task 70.0 in stage 8.0 (TID 78) in 6392 ms on 10.0.0.133 (executor driver) (71/208) -26/04/01 06:01:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:48 INFO Executor: Finished task 71.0 in stage 8.0 (TID 79). 6830 bytes result sent to driver -26/04/01 06:01:48 INFO TaskSetManager: Starting task 75.0 in stage 8.0 (TID 83) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:48 INFO TaskSetManager: Finished task 71.0 in stage 8.0 (TID 79) in 6390 ms on 10.0.0.133 (executor driver) (72/208) -26/04/01 06:01:48 INFO Executor: Running task 75.0 in stage 8.0 (TID 83) -26/04/01 06:01:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:54 INFO Executor: Finished task 72.0 in stage 8.0 (TID 80). 6787 bytes result sent to driver -26/04/01 06:01:54 INFO TaskSetManager: Starting task 76.0 in stage 8.0 (TID 84) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:54 INFO Executor: Running task 76.0 in stage 8.0 (TID 84) -26/04/01 06:01:54 INFO TaskSetManager: Finished task 72.0 in stage 8.0 (TID 80) in 6449 ms on 10.0.0.133 (executor driver) (73/208) -26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:54 INFO Executor: Finished task 73.0 in stage 8.0 (TID 81). 6787 bytes result sent to driver -26/04/01 06:01:54 INFO TaskSetManager: Starting task 77.0 in stage 8.0 (TID 85) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:54 INFO TaskSetManager: Finished task 73.0 in stage 8.0 (TID 81) in 6437 ms on 10.0.0.133 (executor driver) (74/208) -26/04/01 06:01:54 INFO Executor: Running task 77.0 in stage 8.0 (TID 85) -26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:54 INFO Executor: Finished task 74.0 in stage 8.0 (TID 82). 6787 bytes result sent to driver -26/04/01 06:01:54 INFO TaskSetManager: Starting task 78.0 in stage 8.0 (TID 86) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:01:54 INFO TaskSetManager: Finished task 74.0 in stage 8.0 (TID 82) in 6446 ms on 10.0.0.133 (executor driver) (75/208) -26/04/01 06:01:54 INFO Executor: Running task 78.0 in stage 8.0 (TID 86) -26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:54 INFO Executor: Finished task 75.0 in stage 8.0 (TID 83). 6787 bytes result sent to driver -26/04/01 06:01:54 INFO TaskSetManager: Starting task 79.0 in stage 8.0 (TID 87) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:01:54 INFO TaskSetManager: Finished task 75.0 in stage 8.0 (TID 83) in 6464 ms on 10.0.0.133 (executor driver) (76/208) -26/04/01 06:01:54 INFO Executor: Running task 79.0 in stage 8.0 (TID 87) -26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:01:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:00 INFO Executor: Finished task 76.0 in stage 8.0 (TID 84). 6787 bytes result sent to driver -26/04/01 06:02:00 INFO TaskSetManager: Starting task 80.0 in stage 8.0 (TID 88) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:00 INFO TaskSetManager: Finished task 76.0 in stage 8.0 (TID 84) in 6447 ms on 10.0.0.133 (executor driver) (77/208) -26/04/01 06:02:00 INFO Executor: Running task 80.0 in stage 8.0 (TID 88) -26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:00 INFO Executor: Finished task 77.0 in stage 8.0 (TID 85). 6787 bytes result sent to driver -26/04/01 06:02:00 INFO TaskSetManager: Starting task 81.0 in stage 8.0 (TID 89) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:00 INFO TaskSetManager: Finished task 77.0 in stage 8.0 (TID 85) in 6447 ms on 10.0.0.133 (executor driver) (78/208) -26/04/01 06:02:00 INFO Executor: Running task 81.0 in stage 8.0 (TID 89) -26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:00 INFO Executor: Finished task 78.0 in stage 8.0 (TID 86). 6787 bytes result sent to driver -26/04/01 06:02:00 INFO TaskSetManager: Starting task 82.0 in stage 8.0 (TID 90) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:00 INFO Executor: Running task 82.0 in stage 8.0 (TID 90) -26/04/01 06:02:00 INFO TaskSetManager: Finished task 78.0 in stage 8.0 (TID 86) in 6440 ms on 10.0.0.133 (executor driver) (79/208) -26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:00 INFO Executor: Finished task 79.0 in stage 8.0 (TID 87). 6787 bytes result sent to driver -26/04/01 06:02:00 INFO TaskSetManager: Starting task 83.0 in stage 8.0 (TID 91) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:00 INFO TaskSetManager: Finished task 79.0 in stage 8.0 (TID 87) in 6444 ms on 10.0.0.133 (executor driver) (80/208) -26/04/01 06:02:00 INFO Executor: Running task 83.0 in stage 8.0 (TID 91) -26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:06 INFO Executor: Finished task 80.0 in stage 8.0 (TID 88). 6787 bytes result sent to driver -26/04/01 06:02:06 INFO TaskSetManager: Starting task 84.0 in stage 8.0 (TID 92) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:02:06 INFO TaskSetManager: Finished task 80.0 in stage 8.0 (TID 88) in 6451 ms on 10.0.0.133 (executor driver) (81/208) -26/04/01 06:02:06 INFO Executor: Running task 84.0 in stage 8.0 (TID 92) -26/04/01 06:02:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:07 INFO Executor: Finished task 81.0 in stage 8.0 (TID 89). 6787 bytes result sent to driver -26/04/01 06:02:07 INFO Executor: Finished task 82.0 in stage 8.0 (TID 90). 6787 bytes result sent to driver -26/04/01 06:02:07 INFO TaskSetManager: Starting task 85.0 in stage 8.0 (TID 93) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:02:07 INFO Executor: Running task 85.0 in stage 8.0 (TID 93) -26/04/01 06:02:07 INFO TaskSetManager: Starting task 86.0 in stage 8.0 (TID 94) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:07 INFO TaskSetManager: Finished task 81.0 in stage 8.0 (TID 89) in 6451 ms on 10.0.0.133 (executor driver) (82/208) -26/04/01 06:02:07 INFO Executor: Running task 86.0 in stage 8.0 (TID 94) -26/04/01 06:02:07 INFO TaskSetManager: Finished task 82.0 in stage 8.0 (TID 90) in 6447 ms on 10.0.0.133 (executor driver) (83/208) -26/04/01 06:02:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:07 INFO Executor: Finished task 83.0 in stage 8.0 (TID 91). 6787 bytes result sent to driver -26/04/01 06:02:07 INFO TaskSetManager: Starting task 87.0 in stage 8.0 (TID 95) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:07 INFO TaskSetManager: Finished task 83.0 in stage 8.0 (TID 91) in 6452 ms on 10.0.0.133 (executor driver) (84/208) -26/04/01 06:02:07 INFO Executor: Running task 87.0 in stage 8.0 (TID 95) -26/04/01 06:02:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:13 INFO Executor: Finished task 84.0 in stage 8.0 (TID 92). 6787 bytes result sent to driver -26/04/01 06:02:13 INFO TaskSetManager: Starting task 88.0 in stage 8.0 (TID 96) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:13 INFO TaskSetManager: Finished task 84.0 in stage 8.0 (TID 92) in 6455 ms on 10.0.0.133 (executor driver) (85/208) -26/04/01 06:02:13 INFO Executor: Running task 88.0 in stage 8.0 (TID 96) -26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:13 INFO Executor: Finished task 86.0 in stage 8.0 (TID 94). 6787 bytes result sent to driver -26/04/01 06:02:13 INFO TaskSetManager: Starting task 89.0 in stage 8.0 (TID 97) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:13 INFO Executor: Running task 89.0 in stage 8.0 (TID 97) -26/04/01 06:02:13 INFO TaskSetManager: Finished task 86.0 in stage 8.0 (TID 94) in 6453 ms on 10.0.0.133 (executor driver) (86/208) -26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:13 INFO Executor: Finished task 85.0 in stage 8.0 (TID 93). 6787 bytes result sent to driver -26/04/01 06:02:13 INFO TaskSetManager: Starting task 90.0 in stage 8.0 (TID 98) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:02:13 INFO TaskSetManager: Finished task 85.0 in stage 8.0 (TID 93) in 6460 ms on 10.0.0.133 (executor driver) (87/208) -26/04/01 06:02:13 INFO Executor: Running task 90.0 in stage 8.0 (TID 98) -26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:13 INFO Executor: Finished task 87.0 in stage 8.0 (TID 95). 6787 bytes result sent to driver -26/04/01 06:02:13 INFO TaskSetManager: Starting task 91.0 in stage 8.0 (TID 99) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:02:13 INFO Executor: Running task 91.0 in stage 8.0 (TID 99) -26/04/01 06:02:13 INFO TaskSetManager: Finished task 87.0 in stage 8.0 (TID 95) in 6456 ms on 10.0.0.133 (executor driver) (88/208) -26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:19 INFO Executor: Finished task 88.0 in stage 8.0 (TID 96). 6787 bytes result sent to driver -26/04/01 06:02:19 INFO TaskSetManager: Starting task 92.0 in stage 8.0 (TID 100) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:19 INFO TaskSetManager: Finished task 88.0 in stage 8.0 (TID 96) in 6448 ms on 10.0.0.133 (executor driver) (89/208) -26/04/01 06:02:19 INFO Executor: Running task 92.0 in stage 8.0 (TID 100) -26/04/01 06:02:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:20 INFO Executor: Finished task 91.0 in stage 8.0 (TID 99). 6787 bytes result sent to driver -26/04/01 06:02:20 INFO TaskSetManager: Starting task 93.0 in stage 8.0 (TID 101) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:20 INFO TaskSetManager: Finished task 91.0 in stage 8.0 (TID 99) in 6190 ms on 10.0.0.133 (executor driver) (90/208) -26/04/01 06:02:20 INFO Executor: Running task 93.0 in stage 8.0 (TID 101) -26/04/01 06:02:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:20 INFO Executor: Finished task 89.0 in stage 8.0 (TID 97). 6787 bytes result sent to driver -26/04/01 06:02:20 INFO TaskSetManager: Starting task 94.0 in stage 8.0 (TID 102) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:20 INFO TaskSetManager: Finished task 89.0 in stage 8.0 (TID 97) in 6446 ms on 10.0.0.133 (executor driver) (91/208) -26/04/01 06:02:20 INFO Executor: Running task 94.0 in stage 8.0 (TID 102) -26/04/01 06:02:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:20 INFO Executor: Finished task 90.0 in stage 8.0 (TID 98). 6787 bytes result sent to driver -26/04/01 06:02:20 INFO TaskSetManager: Starting task 95.0 in stage 8.0 (TID 103) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:20 INFO Executor: Running task 95.0 in stage 8.0 (TID 103) -26/04/01 06:02:20 INFO TaskSetManager: Finished task 90.0 in stage 8.0 (TID 98) in 6689 ms on 10.0.0.133 (executor driver) (92/208) -26/04/01 06:02:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:26 INFO Executor: Finished task 92.0 in stage 8.0 (TID 100). 6787 bytes result sent to driver -26/04/01 06:02:26 INFO TaskSetManager: Starting task 96.0 in stage 8.0 (TID 104) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:02:26 INFO TaskSetManager: Finished task 92.0 in stage 8.0 (TID 100) in 6427 ms on 10.0.0.133 (executor driver) (93/208) -26/04/01 06:02:26 INFO Executor: Running task 96.0 in stage 8.0 (TID 104) -26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:26 INFO Executor: Finished task 93.0 in stage 8.0 (TID 101). 6787 bytes result sent to driver -26/04/01 06:02:26 INFO TaskSetManager: Starting task 97.0 in stage 8.0 (TID 105) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:02:26 INFO TaskSetManager: Finished task 93.0 in stage 8.0 (TID 101) in 6430 ms on 10.0.0.133 (executor driver) (94/208) -26/04/01 06:02:26 INFO Executor: Running task 97.0 in stage 8.0 (TID 105) -26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:26 INFO Executor: Finished task 94.0 in stage 8.0 (TID 102). 6787 bytes result sent to driver -26/04/01 06:02:26 INFO TaskSetManager: Starting task 98.0 in stage 8.0 (TID 106) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:02:26 INFO Executor: Running task 98.0 in stage 8.0 (TID 106) -26/04/01 06:02:26 INFO TaskSetManager: Finished task 94.0 in stage 8.0 (TID 102) in 6433 ms on 10.0.0.133 (executor driver) (95/208) -26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:26 INFO Executor: Finished task 95.0 in stage 8.0 (TID 103). 6787 bytes result sent to driver -26/04/01 06:02:26 INFO TaskSetManager: Starting task 99.0 in stage 8.0 (TID 107) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:02:26 INFO TaskSetManager: Finished task 95.0 in stage 8.0 (TID 103) in 6428 ms on 10.0.0.133 (executor driver) (96/208) -26/04/01 06:02:26 INFO Executor: Running task 99.0 in stage 8.0 (TID 107) -26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:32 INFO Executor: Finished task 96.0 in stage 8.0 (TID 104). 6787 bytes result sent to driver -26/04/01 06:02:32 INFO TaskSetManager: Starting task 100.0 in stage 8.0 (TID 108) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:02:32 INFO TaskSetManager: Finished task 96.0 in stage 8.0 (TID 104) in 6449 ms on 10.0.0.133 (executor driver) (97/208) -26/04/01 06:02:32 INFO Executor: Running task 100.0 in stage 8.0 (TID 108) -26/04/01 06:02:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:32 INFO Executor: Finished task 97.0 in stage 8.0 (TID 105). 6787 bytes result sent to driver -26/04/01 06:02:32 INFO TaskSetManager: Starting task 101.0 in stage 8.0 (TID 109) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:02:32 INFO TaskSetManager: Finished task 97.0 in stage 8.0 (TID 105) in 6449 ms on 10.0.0.133 (executor driver) (98/208) -26/04/01 06:02:32 INFO Executor: Running task 101.0 in stage 8.0 (TID 109) -26/04/01 06:02:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:32 INFO Executor: Finished task 98.0 in stage 8.0 (TID 106). 6787 bytes result sent to driver -26/04/01 06:02:32 INFO TaskSetManager: Starting task 102.0 in stage 8.0 (TID 110) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:02:32 INFO TaskSetManager: Finished task 98.0 in stage 8.0 (TID 106) in 6446 ms on 10.0.0.133 (executor driver) (99/208) -26/04/01 06:02:32 INFO Executor: Running task 102.0 in stage 8.0 (TID 110) -26/04/01 06:02:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:33 INFO Executor: Finished task 99.0 in stage 8.0 (TID 107). 6787 bytes result sent to driver -26/04/01 06:02:33 INFO TaskSetManager: Starting task 103.0 in stage 8.0 (TID 111) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:02:33 INFO Executor: Running task 103.0 in stage 8.0 (TID 111) -26/04/01 06:02:33 INFO TaskSetManager: Finished task 99.0 in stage 8.0 (TID 107) in 6450 ms on 10.0.0.133 (executor driver) (100/208) -26/04/01 06:02:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:39 INFO Executor: Finished task 100.0 in stage 8.0 (TID 108). 6787 bytes result sent to driver -26/04/01 06:02:39 INFO TaskSetManager: Starting task 104.0 in stage 8.0 (TID 112) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:39 INFO TaskSetManager: Finished task 100.0 in stage 8.0 (TID 108) in 6450 ms on 10.0.0.133 (executor driver) (101/208) -26/04/01 06:02:39 INFO Executor: Running task 104.0 in stage 8.0 (TID 112) -26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:39 INFO Executor: Finished task 102.0 in stage 8.0 (TID 110). 6787 bytes result sent to driver -26/04/01 06:02:39 INFO Executor: Finished task 101.0 in stage 8.0 (TID 109). 6787 bytes result sent to driver -26/04/01 06:02:39 INFO TaskSetManager: Starting task 105.0 in stage 8.0 (TID 113) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:39 INFO TaskSetManager: Starting task 106.0 in stage 8.0 (TID 114) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:39 INFO Executor: Running task 105.0 in stage 8.0 (TID 113) -26/04/01 06:02:39 INFO TaskSetManager: Finished task 102.0 in stage 8.0 (TID 110) in 6451 ms on 10.0.0.133 (executor driver) (102/208) -26/04/01 06:02:39 INFO Executor: Running task 106.0 in stage 8.0 (TID 114) -26/04/01 06:02:39 INFO TaskSetManager: Finished task 101.0 in stage 8.0 (TID 109) in 6454 ms on 10.0.0.133 (executor driver) (103/208) -26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:39 INFO Executor: Finished task 103.0 in stage 8.0 (TID 111). 6787 bytes result sent to driver -26/04/01 06:02:39 INFO TaskSetManager: Starting task 107.0 in stage 8.0 (TID 115) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:39 INFO TaskSetManager: Finished task 103.0 in stage 8.0 (TID 111) in 6454 ms on 10.0.0.133 (executor driver) (104/208) -26/04/01 06:02:39 INFO Executor: Running task 107.0 in stage 8.0 (TID 115) -26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:45 INFO Executor: Finished task 104.0 in stage 8.0 (TID 112). 6787 bytes result sent to driver -26/04/01 06:02:45 INFO TaskSetManager: Starting task 108.0 in stage 8.0 (TID 116) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:02:45 INFO TaskSetManager: Finished task 104.0 in stage 8.0 (TID 112) in 6464 ms on 10.0.0.133 (executor driver) (105/208) -26/04/01 06:02:45 INFO Executor: Running task 108.0 in stage 8.0 (TID 116) -26/04/01 06:02:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:45 INFO Executor: Finished task 105.0 in stage 8.0 (TID 113). 6787 bytes result sent to driver -26/04/01 06:02:45 INFO Executor: Finished task 106.0 in stage 8.0 (TID 114). 6787 bytes result sent to driver -26/04/01 06:02:45 INFO TaskSetManager: Starting task 109.0 in stage 8.0 (TID 117) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:02:45 INFO TaskSetManager: Starting task 110.0 in stage 8.0 (TID 118) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:02:45 INFO Executor: Running task 109.0 in stage 8.0 (TID 117) -26/04/01 06:02:45 INFO TaskSetManager: Finished task 105.0 in stage 8.0 (TID 113) in 6467 ms on 10.0.0.133 (executor driver) (106/208) -26/04/01 06:02:45 INFO Executor: Running task 110.0 in stage 8.0 (TID 118) -26/04/01 06:02:45 INFO TaskSetManager: Finished task 106.0 in stage 8.0 (TID 114) in 6466 ms on 10.0.0.133 (executor driver) (107/208) -26/04/01 06:02:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:46 INFO Executor: Finished task 107.0 in stage 8.0 (TID 115). 6787 bytes result sent to driver -26/04/01 06:02:46 INFO TaskSetManager: Starting task 111.0 in stage 8.0 (TID 119) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:02:46 INFO TaskSetManager: Finished task 107.0 in stage 8.0 (TID 115) in 6461 ms on 10.0.0.133 (executor driver) (108/208) -26/04/01 06:02:46 INFO Executor: Running task 111.0 in stage 8.0 (TID 119) -26/04/01 06:02:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:52 INFO Executor: Finished task 108.0 in stage 8.0 (TID 116). 6787 bytes result sent to driver -26/04/01 06:02:52 INFO TaskSetManager: Starting task 112.0 in stage 8.0 (TID 120) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:02:52 INFO Executor: Running task 112.0 in stage 8.0 (TID 120) -26/04/01 06:02:52 INFO TaskSetManager: Finished task 108.0 in stage 8.0 (TID 116) in 6471 ms on 10.0.0.133 (executor driver) (109/208) -26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:52 INFO Executor: Finished task 109.0 in stage 8.0 (TID 117). 6787 bytes result sent to driver -26/04/01 06:02:52 INFO TaskSetManager: Starting task 113.0 in stage 8.0 (TID 121) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:02:52 INFO Executor: Running task 113.0 in stage 8.0 (TID 121) -26/04/01 06:02:52 INFO TaskSetManager: Finished task 109.0 in stage 8.0 (TID 117) in 6464 ms on 10.0.0.133 (executor driver) (110/208) -26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:52 INFO Executor: Finished task 110.0 in stage 8.0 (TID 118). 6787 bytes result sent to driver -26/04/01 06:02:52 INFO TaskSetManager: Starting task 114.0 in stage 8.0 (TID 122) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:02:52 INFO TaskSetManager: Finished task 110.0 in stage 8.0 (TID 118) in 6468 ms on 10.0.0.133 (executor driver) (111/208) -26/04/01 06:02:52 INFO Executor: Running task 114.0 in stage 8.0 (TID 122) -26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:52 INFO Executor: Finished task 111.0 in stage 8.0 (TID 119). 6787 bytes result sent to driver -26/04/01 06:02:52 INFO TaskSetManager: Starting task 115.0 in stage 8.0 (TID 123) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:02:52 INFO TaskSetManager: Finished task 111.0 in stage 8.0 (TID 119) in 6473 ms on 10.0.0.133 (executor driver) (112/208) -26/04/01 06:02:52 INFO Executor: Running task 115.0 in stage 8.0 (TID 123) -26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:58 INFO Executor: Finished task 112.0 in stage 8.0 (TID 120). 6787 bytes result sent to driver -26/04/01 06:02:58 INFO TaskSetManager: Starting task 116.0 in stage 8.0 (TID 124) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:58 INFO TaskSetManager: Finished task 112.0 in stage 8.0 (TID 120) in 6456 ms on 10.0.0.133 (executor driver) (113/208) -26/04/01 06:02:58 INFO Executor: Running task 116.0 in stage 8.0 (TID 124) -26/04/01 06:02:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:58 INFO Executor: Finished task 113.0 in stage 8.0 (TID 121). 6787 bytes result sent to driver -26/04/01 06:02:58 INFO TaskSetManager: Starting task 117.0 in stage 8.0 (TID 125) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:58 INFO Executor: Running task 117.0 in stage 8.0 (TID 125) -26/04/01 06:02:58 INFO TaskSetManager: Finished task 113.0 in stage 8.0 (TID 121) in 6457 ms on 10.0.0.133 (executor driver) (114/208) -26/04/01 06:02:58 INFO Executor: Finished task 114.0 in stage 8.0 (TID 122). 6787 bytes result sent to driver -26/04/01 06:02:58 INFO TaskSetManager: Starting task 118.0 in stage 8.0 (TID 126) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:58 INFO TaskSetManager: Finished task 114.0 in stage 8.0 (TID 122) in 6454 ms on 10.0.0.133 (executor driver) (115/208) -26/04/01 06:02:58 INFO Executor: Running task 118.0 in stage 8.0 (TID 126) -26/04/01 06:02:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:59 INFO Executor: Finished task 115.0 in stage 8.0 (TID 123). 6830 bytes result sent to driver -26/04/01 06:02:59 INFO TaskSetManager: Starting task 119.0 in stage 8.0 (TID 127) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:02:59 INFO TaskSetManager: Finished task 115.0 in stage 8.0 (TID 123) in 6447 ms on 10.0.0.133 (executor driver) (116/208) -26/04/01 06:02:59 INFO Executor: Running task 119.0 in stage 8.0 (TID 127) -26/04/01 06:02:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:02:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:05 INFO Executor: Finished task 116.0 in stage 8.0 (TID 124). 6830 bytes result sent to driver -26/04/01 06:03:05 INFO TaskSetManager: Starting task 120.0 in stage 8.0 (TID 128) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:03:05 INFO TaskSetManager: Finished task 116.0 in stage 8.0 (TID 124) in 6456 ms on 10.0.0.133 (executor driver) (117/208) -26/04/01 06:03:05 INFO Executor: Running task 120.0 in stage 8.0 (TID 128) -26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:05 INFO Executor: Finished task 118.0 in stage 8.0 (TID 126). 6873 bytes result sent to driver -26/04/01 06:03:05 INFO TaskSetManager: Starting task 121.0 in stage 8.0 (TID 129) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:03:05 INFO Executor: Running task 121.0 in stage 8.0 (TID 129) -26/04/01 06:03:05 INFO TaskSetManager: Finished task 118.0 in stage 8.0 (TID 126) in 6462 ms on 10.0.0.133 (executor driver) (118/208) -26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:05 INFO Executor: Finished task 117.0 in stage 8.0 (TID 125). 6830 bytes result sent to driver -26/04/01 06:03:05 INFO TaskSetManager: Starting task 122.0 in stage 8.0 (TID 130) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:05 INFO Executor: Running task 122.0 in stage 8.0 (TID 130) -26/04/01 06:03:05 INFO TaskSetManager: Finished task 117.0 in stage 8.0 (TID 125) in 6466 ms on 10.0.0.133 (executor driver) (119/208) -26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:05 INFO Executor: Finished task 119.0 in stage 8.0 (TID 127). 6787 bytes result sent to driver -26/04/01 06:03:05 INFO TaskSetManager: Starting task 123.0 in stage 8.0 (TID 131) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:05 INFO Executor: Running task 123.0 in stage 8.0 (TID 131) -26/04/01 06:03:05 INFO TaskSetManager: Finished task 119.0 in stage 8.0 (TID 127) in 6469 ms on 10.0.0.133 (executor driver) (120/208) -26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:11 INFO Executor: Finished task 120.0 in stage 8.0 (TID 128). 6787 bytes result sent to driver -26/04/01 06:03:11 INFO TaskSetManager: Starting task 124.0 in stage 8.0 (TID 132) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:11 INFO TaskSetManager: Finished task 120.0 in stage 8.0 (TID 128) in 6474 ms on 10.0.0.133 (executor driver) (121/208) -26/04/01 06:03:11 INFO Executor: Running task 124.0 in stage 8.0 (TID 132) -26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:11 INFO Executor: Finished task 122.0 in stage 8.0 (TID 130). 6787 bytes result sent to driver -26/04/01 06:03:11 INFO TaskSetManager: Starting task 125.0 in stage 8.0 (TID 133) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:11 INFO TaskSetManager: Finished task 122.0 in stage 8.0 (TID 130) in 6453 ms on 10.0.0.133 (executor driver) (122/208) -26/04/01 06:03:11 INFO Executor: Running task 125.0 in stage 8.0 (TID 133) -26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:11 INFO Executor: Finished task 121.0 in stage 8.0 (TID 129). 6787 bytes result sent to driver -26/04/01 06:03:11 INFO TaskSetManager: Starting task 126.0 in stage 8.0 (TID 134) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:03:11 INFO TaskSetManager: Finished task 121.0 in stage 8.0 (TID 129) in 6466 ms on 10.0.0.133 (executor driver) (123/208) -26/04/01 06:03:11 INFO Executor: Running task 126.0 in stage 8.0 (TID 134) -26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:11 INFO Executor: Finished task 123.0 in stage 8.0 (TID 131). 6787 bytes result sent to driver -26/04/01 06:03:11 INFO TaskSetManager: Starting task 127.0 in stage 8.0 (TID 135) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:03:11 INFO TaskSetManager: Finished task 123.0 in stage 8.0 (TID 131) in 6453 ms on 10.0.0.133 (executor driver) (124/208) -26/04/01 06:03:11 INFO Executor: Running task 127.0 in stage 8.0 (TID 135) -26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:17 INFO Executor: Finished task 124.0 in stage 8.0 (TID 132). 6787 bytes result sent to driver -26/04/01 06:03:17 INFO TaskSetManager: Starting task 128.0 in stage 8.0 (TID 136) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:17 INFO Executor: Running task 128.0 in stage 8.0 (TID 136) -26/04/01 06:03:17 INFO TaskSetManager: Finished task 124.0 in stage 8.0 (TID 132) in 6445 ms on 10.0.0.133 (executor driver) (125/208) -26/04/01 06:03:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:18 INFO Executor: Finished task 126.0 in stage 8.0 (TID 134). 6787 bytes result sent to driver -26/04/01 06:03:18 INFO Executor: Finished task 125.0 in stage 8.0 (TID 133). 6787 bytes result sent to driver -26/04/01 06:03:18 INFO TaskSetManager: Starting task 129.0 in stage 8.0 (TID 137) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:18 INFO TaskSetManager: Starting task 130.0 in stage 8.0 (TID 138) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:18 INFO Executor: Running task 129.0 in stage 8.0 (TID 137) -26/04/01 06:03:18 INFO TaskSetManager: Finished task 126.0 in stage 8.0 (TID 134) in 6448 ms on 10.0.0.133 (executor driver) (126/208) -26/04/01 06:03:18 INFO Executor: Running task 130.0 in stage 8.0 (TID 138) -26/04/01 06:03:18 INFO TaskSetManager: Finished task 125.0 in stage 8.0 (TID 133) in 6458 ms on 10.0.0.133 (executor driver) (127/208) -26/04/01 06:03:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:18 INFO Executor: Finished task 127.0 in stage 8.0 (TID 135). 6787 bytes result sent to driver -26/04/01 06:03:18 INFO TaskSetManager: Starting task 131.0 in stage 8.0 (TID 139) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:18 INFO TaskSetManager: Finished task 127.0 in stage 8.0 (TID 135) in 6458 ms on 10.0.0.133 (executor driver) (128/208) -26/04/01 06:03:18 INFO Executor: Running task 131.0 in stage 8.0 (TID 139) -26/04/01 06:03:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:24 INFO Executor: Finished task 128.0 in stage 8.0 (TID 136). 6787 bytes result sent to driver -26/04/01 06:03:24 INFO TaskSetManager: Starting task 132.0 in stage 8.0 (TID 140) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:03:24 INFO TaskSetManager: Finished task 128.0 in stage 8.0 (TID 136) in 6468 ms on 10.0.0.133 (executor driver) (129/208) -26/04/01 06:03:24 INFO Executor: Running task 132.0 in stage 8.0 (TID 140) -26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:24 INFO Executor: Finished task 130.0 in stage 8.0 (TID 138). 6787 bytes result sent to driver -26/04/01 06:03:24 INFO Executor: Finished task 129.0 in stage 8.0 (TID 137). 6787 bytes result sent to driver -26/04/01 06:03:24 INFO TaskSetManager: Starting task 133.0 in stage 8.0 (TID 141) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:03:24 INFO Executor: Running task 133.0 in stage 8.0 (TID 141) -26/04/01 06:03:24 INFO TaskSetManager: Starting task 134.0 in stage 8.0 (TID 142) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:24 INFO TaskSetManager: Finished task 130.0 in stage 8.0 (TID 138) in 6469 ms on 10.0.0.133 (executor driver) (130/208) -26/04/01 06:03:24 INFO Executor: Running task 134.0 in stage 8.0 (TID 142) -26/04/01 06:03:24 INFO TaskSetManager: Finished task 129.0 in stage 8.0 (TID 137) in 6471 ms on 10.0.0.133 (executor driver) (131/208) -26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:24 INFO Executor: Finished task 131.0 in stage 8.0 (TID 139). 6787 bytes result sent to driver -26/04/01 06:03:24 INFO TaskSetManager: Starting task 135.0 in stage 8.0 (TID 143) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:24 INFO TaskSetManager: Finished task 131.0 in stage 8.0 (TID 139) in 6465 ms on 10.0.0.133 (executor driver) (132/208) -26/04/01 06:03:24 INFO Executor: Running task 135.0 in stage 8.0 (TID 143) -26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:30 INFO Executor: Finished task 132.0 in stage 8.0 (TID 140). 6787 bytes result sent to driver -26/04/01 06:03:30 INFO TaskSetManager: Starting task 136.0 in stage 8.0 (TID 144) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:30 INFO Executor: Running task 136.0 in stage 8.0 (TID 144) -26/04/01 06:03:30 INFO TaskSetManager: Finished task 132.0 in stage 8.0 (TID 140) in 6406 ms on 10.0.0.133 (executor driver) (133/208) -26/04/01 06:03:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:31 INFO Executor: Finished task 134.0 in stage 8.0 (TID 142). 6787 bytes result sent to driver -26/04/01 06:03:31 INFO Executor: Finished task 133.0 in stage 8.0 (TID 141). 6787 bytes result sent to driver -26/04/01 06:03:31 INFO TaskSetManager: Starting task 137.0 in stage 8.0 (TID 145) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:31 INFO TaskSetManager: Finished task 134.0 in stage 8.0 (TID 142) in 6396 ms on 10.0.0.133 (executor driver) (134/208) -26/04/01 06:03:31 INFO Executor: Running task 137.0 in stage 8.0 (TID 145) -26/04/01 06:03:31 INFO TaskSetManager: Starting task 138.0 in stage 8.0 (TID 146) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:03:31 INFO TaskSetManager: Finished task 133.0 in stage 8.0 (TID 141) in 6396 ms on 10.0.0.133 (executor driver) (135/208) -26/04/01 06:03:31 INFO Executor: Running task 138.0 in stage 8.0 (TID 146) -26/04/01 06:03:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:31 INFO Executor: Finished task 135.0 in stage 8.0 (TID 143). 6787 bytes result sent to driver -26/04/01 06:03:31 INFO TaskSetManager: Starting task 139.0 in stage 8.0 (TID 147) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:03:31 INFO TaskSetManager: Finished task 135.0 in stage 8.0 (TID 143) in 6403 ms on 10.0.0.133 (executor driver) (136/208) -26/04/01 06:03:31 INFO Executor: Running task 139.0 in stage 8.0 (TID 147) -26/04/01 06:03:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:37 INFO Executor: Finished task 136.0 in stage 8.0 (TID 144). 6787 bytes result sent to driver -26/04/01 06:03:37 INFO TaskSetManager: Starting task 140.0 in stage 8.0 (TID 148) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:37 INFO TaskSetManager: Finished task 136.0 in stage 8.0 (TID 144) in 6393 ms on 10.0.0.133 (executor driver) (137/208) -26/04/01 06:03:37 INFO Executor: Running task 140.0 in stage 8.0 (TID 148) -26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:37 INFO Executor: Finished task 137.0 in stage 8.0 (TID 145). 6787 bytes result sent to driver -26/04/01 06:03:37 INFO TaskSetManager: Starting task 141.0 in stage 8.0 (TID 149) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:37 INFO Executor: Running task 141.0 in stage 8.0 (TID 149) -26/04/01 06:03:37 INFO TaskSetManager: Finished task 137.0 in stage 8.0 (TID 145) in 6396 ms on 10.0.0.133 (executor driver) (138/208) -26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:37 INFO Executor: Finished task 138.0 in stage 8.0 (TID 146). 6787 bytes result sent to driver -26/04/01 06:03:37 INFO TaskSetManager: Starting task 142.0 in stage 8.0 (TID 150) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:37 INFO TaskSetManager: Finished task 138.0 in stage 8.0 (TID 146) in 6411 ms on 10.0.0.133 (executor driver) (139/208) -26/04/01 06:03:37 INFO Executor: Running task 142.0 in stage 8.0 (TID 150) -26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:37 INFO Executor: Finished task 139.0 in stage 8.0 (TID 147). 6787 bytes result sent to driver -26/04/01 06:03:37 INFO TaskSetManager: Starting task 143.0 in stage 8.0 (TID 151) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:37 INFO TaskSetManager: Finished task 139.0 in stage 8.0 (TID 147) in 6390 ms on 10.0.0.133 (executor driver) (140/208) -26/04/01 06:03:37 INFO Executor: Running task 143.0 in stage 8.0 (TID 151) -26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:43 INFO Executor: Finished task 140.0 in stage 8.0 (TID 148). 6787 bytes result sent to driver -26/04/01 06:03:43 INFO TaskSetManager: Starting task 144.0 in stage 8.0 (TID 152) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:03:43 INFO TaskSetManager: Finished task 140.0 in stage 8.0 (TID 148) in 6385 ms on 10.0.0.133 (executor driver) (141/208) -26/04/01 06:03:43 INFO Executor: Running task 144.0 in stage 8.0 (TID 152) -26/04/01 06:03:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:43 INFO Executor: Finished task 141.0 in stage 8.0 (TID 149). 6787 bytes result sent to driver -26/04/01 06:03:43 INFO TaskSetManager: Starting task 145.0 in stage 8.0 (TID 153) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:03:43 INFO Executor: Running task 145.0 in stage 8.0 (TID 153) -26/04/01 06:03:43 INFO TaskSetManager: Finished task 141.0 in stage 8.0 (TID 149) in 6382 ms on 10.0.0.133 (executor driver) (142/208) -26/04/01 06:03:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:43 INFO Executor: Finished task 142.0 in stage 8.0 (TID 150). 6787 bytes result sent to driver -26/04/01 06:03:43 INFO TaskSetManager: Starting task 146.0 in stage 8.0 (TID 154) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:03:43 INFO Executor: Running task 146.0 in stage 8.0 (TID 154) -26/04/01 06:03:43 INFO TaskSetManager: Finished task 142.0 in stage 8.0 (TID 150) in 6388 ms on 10.0.0.133 (executor driver) (143/208) -26/04/01 06:03:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:44 INFO Executor: Finished task 143.0 in stage 8.0 (TID 151). 6787 bytes result sent to driver -26/04/01 06:03:44 INFO TaskSetManager: Starting task 147.0 in stage 8.0 (TID 155) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:03:44 INFO Executor: Running task 147.0 in stage 8.0 (TID 155) -26/04/01 06:03:44 INFO TaskSetManager: Finished task 143.0 in stage 8.0 (TID 151) in 6383 ms on 10.0.0.133 (executor driver) (144/208) -26/04/01 06:03:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:50 INFO Executor: Finished task 144.0 in stage 8.0 (TID 152). 6787 bytes result sent to driver -26/04/01 06:03:50 INFO TaskSetManager: Starting task 148.0 in stage 8.0 (TID 156) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:03:50 INFO Executor: Running task 148.0 in stage 8.0 (TID 156) -26/04/01 06:03:50 INFO TaskSetManager: Finished task 144.0 in stage 8.0 (TID 152) in 6448 ms on 10.0.0.133 (executor driver) (145/208) -26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:50 INFO Executor: Finished task 146.0 in stage 8.0 (TID 154). 6787 bytes result sent to driver -26/04/01 06:03:50 INFO TaskSetManager: Starting task 149.0 in stage 8.0 (TID 157) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:03:50 INFO TaskSetManager: Finished task 146.0 in stage 8.0 (TID 154) in 6436 ms on 10.0.0.133 (executor driver) (146/208) -26/04/01 06:03:50 INFO Executor: Running task 149.0 in stage 8.0 (TID 157) -26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:50 INFO Executor: Finished task 145.0 in stage 8.0 (TID 153). 6787 bytes result sent to driver -26/04/01 06:03:50 INFO TaskSetManager: Starting task 150.0 in stage 8.0 (TID 158) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:03:50 INFO TaskSetManager: Finished task 145.0 in stage 8.0 (TID 153) in 6459 ms on 10.0.0.133 (executor driver) (147/208) -26/04/01 06:03:50 INFO Executor: Running task 150.0 in stage 8.0 (TID 158) -26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:50 INFO Executor: Finished task 147.0 in stage 8.0 (TID 155). 6787 bytes result sent to driver -26/04/01 06:03:50 INFO TaskSetManager: Starting task 151.0 in stage 8.0 (TID 159) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:03:50 INFO TaskSetManager: Finished task 147.0 in stage 8.0 (TID 155) in 6460 ms on 10.0.0.133 (executor driver) (148/208) -26/04/01 06:03:50 INFO Executor: Running task 151.0 in stage 8.0 (TID 159) -26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:56 INFO Executor: Finished task 148.0 in stage 8.0 (TID 156). 6787 bytes result sent to driver -26/04/01 06:03:56 INFO TaskSetManager: Starting task 152.0 in stage 8.0 (TID 160) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:56 INFO TaskSetManager: Finished task 148.0 in stage 8.0 (TID 156) in 6463 ms on 10.0.0.133 (executor driver) (149/208) -26/04/01 06:03:56 INFO Executor: Running task 152.0 in stage 8.0 (TID 160) -26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:56 INFO Executor: Finished task 149.0 in stage 8.0 (TID 157). 6787 bytes result sent to driver -26/04/01 06:03:56 INFO TaskSetManager: Starting task 153.0 in stage 8.0 (TID 161) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:56 INFO TaskSetManager: Finished task 149.0 in stage 8.0 (TID 157) in 6462 ms on 10.0.0.133 (executor driver) (150/208) -26/04/01 06:03:56 INFO Executor: Running task 153.0 in stage 8.0 (TID 161) -26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:56 INFO Executor: Finished task 150.0 in stage 8.0 (TID 158). 6787 bytes result sent to driver -26/04/01 06:03:56 INFO TaskSetManager: Starting task 154.0 in stage 8.0 (TID 162) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:56 INFO TaskSetManager: Finished task 150.0 in stage 8.0 (TID 158) in 6464 ms on 10.0.0.133 (executor driver) (151/208) -26/04/01 06:03:56 INFO Executor: Running task 154.0 in stage 8.0 (TID 162) -26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:56 INFO Executor: Finished task 151.0 in stage 8.0 (TID 159). 6787 bytes result sent to driver -26/04/01 06:03:56 INFO TaskSetManager: Starting task 155.0 in stage 8.0 (TID 163) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:03:56 INFO TaskSetManager: Finished task 151.0 in stage 8.0 (TID 159) in 6462 ms on 10.0.0.133 (executor driver) (152/208) -26/04/01 06:03:56 INFO Executor: Running task 155.0 in stage 8.0 (TID 163) -26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:03:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:02 INFO Executor: Finished task 152.0 in stage 8.0 (TID 160). 6787 bytes result sent to driver -26/04/01 06:04:02 INFO TaskSetManager: Starting task 156.0 in stage 8.0 (TID 164) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:04:02 INFO TaskSetManager: Finished task 152.0 in stage 8.0 (TID 160) in 6447 ms on 10.0.0.133 (executor driver) (153/208) -26/04/01 06:04:02 INFO Executor: Running task 156.0 in stage 8.0 (TID 164) -26/04/01 06:04:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:03 INFO Executor: Finished task 153.0 in stage 8.0 (TID 161). 6787 bytes result sent to driver -26/04/01 06:04:03 INFO TaskSetManager: Starting task 157.0 in stage 8.0 (TID 165) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:04:03 INFO Executor: Running task 157.0 in stage 8.0 (TID 165) -26/04/01 06:04:03 INFO TaskSetManager: Finished task 153.0 in stage 8.0 (TID 161) in 6447 ms on 10.0.0.133 (executor driver) (154/208) -26/04/01 06:04:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:03 INFO Executor: Finished task 154.0 in stage 8.0 (TID 162). 6787 bytes result sent to driver -26/04/01 06:04:03 INFO TaskSetManager: Starting task 158.0 in stage 8.0 (TID 166) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:03 INFO Executor: Running task 158.0 in stage 8.0 (TID 166) -26/04/01 06:04:03 INFO TaskSetManager: Finished task 154.0 in stage 8.0 (TID 162) in 6447 ms on 10.0.0.133 (executor driver) (155/208) -26/04/01 06:04:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:03 INFO Executor: Finished task 155.0 in stage 8.0 (TID 163). 6787 bytes result sent to driver -26/04/01 06:04:03 INFO TaskSetManager: Starting task 159.0 in stage 8.0 (TID 167) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:03 INFO Executor: Running task 159.0 in stage 8.0 (TID 167) -26/04/01 06:04:03 INFO TaskSetManager: Finished task 155.0 in stage 8.0 (TID 163) in 6450 ms on 10.0.0.133 (executor driver) (156/208) -26/04/01 06:04:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:09 INFO Executor: Finished task 156.0 in stage 8.0 (TID 164). 6787 bytes result sent to driver -26/04/01 06:04:09 INFO TaskSetManager: Starting task 160.0 in stage 8.0 (TID 168) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:09 INFO TaskSetManager: Finished task 156.0 in stage 8.0 (TID 164) in 6408 ms on 10.0.0.133 (executor driver) (157/208) -26/04/01 06:04:09 INFO Executor: Running task 160.0 in stage 8.0 (TID 168) -26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:09 INFO Executor: Finished task 158.0 in stage 8.0 (TID 166). 6787 bytes result sent to driver -26/04/01 06:04:09 INFO Executor: Finished task 157.0 in stage 8.0 (TID 165). 6787 bytes result sent to driver -26/04/01 06:04:09 INFO TaskSetManager: Starting task 161.0 in stage 8.0 (TID 169) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:09 INFO Executor: Running task 161.0 in stage 8.0 (TID 169) -26/04/01 06:04:09 INFO TaskSetManager: Starting task 162.0 in stage 8.0 (TID 170) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:04:09 INFO TaskSetManager: Finished task 158.0 in stage 8.0 (TID 166) in 6406 ms on 10.0.0.133 (executor driver) (158/208) -26/04/01 06:04:09 INFO Executor: Running task 162.0 in stage 8.0 (TID 170) -26/04/01 06:04:09 INFO TaskSetManager: Finished task 157.0 in stage 8.0 (TID 165) in 6410 ms on 10.0.0.133 (executor driver) (159/208) -26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:09 INFO Executor: Finished task 159.0 in stage 8.0 (TID 167). 6787 bytes result sent to driver -26/04/01 06:04:09 INFO TaskSetManager: Starting task 163.0 in stage 8.0 (TID 171) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:04:09 INFO TaskSetManager: Finished task 159.0 in stage 8.0 (TID 167) in 6410 ms on 10.0.0.133 (executor driver) (160/208) -26/04/01 06:04:09 INFO Executor: Running task 163.0 in stage 8.0 (TID 171) -26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:15 INFO Executor: Finished task 160.0 in stage 8.0 (TID 168). 6787 bytes result sent to driver -26/04/01 06:04:15 INFO TaskSetManager: Starting task 164.0 in stage 8.0 (TID 172) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:04:15 INFO Executor: Running task 164.0 in stage 8.0 (TID 172) -26/04/01 06:04:15 INFO TaskSetManager: Finished task 160.0 in stage 8.0 (TID 168) in 6422 ms on 10.0.0.133 (executor driver) (161/208) -26/04/01 06:04:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:16 INFO Executor: Finished task 162.0 in stage 8.0 (TID 170). 6787 bytes result sent to driver -26/04/01 06:04:16 INFO TaskSetManager: Starting task 165.0 in stage 8.0 (TID 173) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:04:16 INFO Executor: Finished task 161.0 in stage 8.0 (TID 169). 6787 bytes result sent to driver -26/04/01 06:04:16 INFO TaskSetManager: Finished task 162.0 in stage 8.0 (TID 170) in 6422 ms on 10.0.0.133 (executor driver) (162/208) -26/04/01 06:04:16 INFO Executor: Running task 165.0 in stage 8.0 (TID 173) -26/04/01 06:04:16 INFO TaskSetManager: Starting task 166.0 in stage 8.0 (TID 174) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:04:16 INFO Executor: Running task 166.0 in stage 8.0 (TID 174) -26/04/01 06:04:16 INFO TaskSetManager: Finished task 161.0 in stage 8.0 (TID 169) in 6423 ms on 10.0.0.133 (executor driver) (163/208) -26/04/01 06:04:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:16 INFO Executor: Finished task 163.0 in stage 8.0 (TID 171). 6830 bytes result sent to driver -26/04/01 06:04:16 INFO TaskSetManager: Starting task 167.0 in stage 8.0 (TID 175) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:04:16 INFO Executor: Running task 167.0 in stage 8.0 (TID 175) -26/04/01 06:04:16 INFO TaskSetManager: Finished task 163.0 in stage 8.0 (TID 171) in 6417 ms on 10.0.0.133 (executor driver) (164/208) -26/04/01 06:04:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:22 INFO Executor: Finished task 164.0 in stage 8.0 (TID 172). 6830 bytes result sent to driver -26/04/01 06:04:22 INFO TaskSetManager: Starting task 168.0 in stage 8.0 (TID 176) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:04:22 INFO Executor: Running task 168.0 in stage 8.0 (TID 176) -26/04/01 06:04:22 INFO TaskSetManager: Finished task 164.0 in stage 8.0 (TID 172) in 6422 ms on 10.0.0.133 (executor driver) (165/208) -26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:22 INFO Executor: Finished task 165.0 in stage 8.0 (TID 173). 6830 bytes result sent to driver -26/04/01 06:04:22 INFO TaskSetManager: Starting task 169.0 in stage 8.0 (TID 177) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:04:22 INFO Executor: Running task 169.0 in stage 8.0 (TID 177) -26/04/01 06:04:22 INFO TaskSetManager: Finished task 165.0 in stage 8.0 (TID 173) in 6418 ms on 10.0.0.133 (executor driver) (166/208) -26/04/01 06:04:22 INFO Executor: Finished task 166.0 in stage 8.0 (TID 174). 6830 bytes result sent to driver -26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:22 INFO TaskSetManager: Starting task 170.0 in stage 8.0 (TID 178) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:22 INFO TaskSetManager: Finished task 166.0 in stage 8.0 (TID 174) in 6420 ms on 10.0.0.133 (executor driver) (167/208) -26/04/01 06:04:22 INFO Executor: Running task 170.0 in stage 8.0 (TID 178) -26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:22 INFO Executor: Finished task 167.0 in stage 8.0 (TID 175). 6787 bytes result sent to driver -26/04/01 06:04:22 INFO TaskSetManager: Starting task 171.0 in stage 8.0 (TID 179) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:22 INFO TaskSetManager: Finished task 167.0 in stage 8.0 (TID 175) in 6428 ms on 10.0.0.133 (executor driver) (168/208) -26/04/01 06:04:22 INFO Executor: Running task 171.0 in stage 8.0 (TID 179) -26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:28 INFO Executor: Finished task 168.0 in stage 8.0 (TID 176). 6787 bytes result sent to driver -26/04/01 06:04:28 INFO TaskSetManager: Starting task 172.0 in stage 8.0 (TID 180) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:28 INFO TaskSetManager: Finished task 168.0 in stage 8.0 (TID 176) in 6424 ms on 10.0.0.133 (executor driver) (169/208) -26/04/01 06:04:28 INFO Executor: Running task 172.0 in stage 8.0 (TID 180) -26/04/01 06:04:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:28 INFO Executor: Finished task 169.0 in stage 8.0 (TID 177). 6787 bytes result sent to driver -26/04/01 06:04:28 INFO Executor: Finished task 170.0 in stage 8.0 (TID 178). 6787 bytes result sent to driver -26/04/01 06:04:28 INFO TaskSetManager: Starting task 173.0 in stage 8.0 (TID 181) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:28 INFO TaskSetManager: Finished task 169.0 in stage 8.0 (TID 177) in 6426 ms on 10.0.0.133 (executor driver) (170/208) -26/04/01 06:04:28 INFO Executor: Running task 173.0 in stage 8.0 (TID 181) -26/04/01 06:04:28 INFO TaskSetManager: Starting task 174.0 in stage 8.0 (TID 182) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:04:28 INFO TaskSetManager: Finished task 170.0 in stage 8.0 (TID 178) in 6424 ms on 10.0.0.133 (executor driver) (171/208) -26/04/01 06:04:28 INFO Executor: Running task 174.0 in stage 8.0 (TID 182) -26/04/01 06:04:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:29 INFO Executor: Finished task 171.0 in stage 8.0 (TID 179). 6787 bytes result sent to driver -26/04/01 06:04:29 INFO TaskSetManager: Starting task 175.0 in stage 8.0 (TID 183) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:04:29 INFO TaskSetManager: Finished task 171.0 in stage 8.0 (TID 179) in 6416 ms on 10.0.0.133 (executor driver) (172/208) -26/04/01 06:04:29 INFO Executor: Running task 175.0 in stage 8.0 (TID 183) -26/04/01 06:04:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:35 INFO Executor: Finished task 172.0 in stage 8.0 (TID 180). 6787 bytes result sent to driver -26/04/01 06:04:35 INFO TaskSetManager: Starting task 176.0 in stage 8.0 (TID 184) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:04:35 INFO Executor: Running task 176.0 in stage 8.0 (TID 184) -26/04/01 06:04:35 INFO TaskSetManager: Finished task 172.0 in stage 8.0 (TID 180) in 6420 ms on 10.0.0.133 (executor driver) (173/208) -26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:35 INFO Executor: Finished task 173.0 in stage 8.0 (TID 181). 6787 bytes result sent to driver -26/04/01 06:04:35 INFO TaskSetManager: Starting task 177.0 in stage 8.0 (TID 185) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:04:35 INFO TaskSetManager: Finished task 173.0 in stage 8.0 (TID 181) in 6419 ms on 10.0.0.133 (executor driver) (174/208) -26/04/01 06:04:35 INFO Executor: Running task 177.0 in stage 8.0 (TID 185) -26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:35 INFO Executor: Finished task 174.0 in stage 8.0 (TID 182). 6787 bytes result sent to driver -26/04/01 06:04:35 INFO TaskSetManager: Starting task 178.0 in stage 8.0 (TID 186) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:04:35 INFO Executor: Running task 178.0 in stage 8.0 (TID 186) -26/04/01 06:04:35 INFO TaskSetManager: Finished task 174.0 in stage 8.0 (TID 182) in 6434 ms on 10.0.0.133 (executor driver) (175/208) -26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:35 INFO Executor: Finished task 175.0 in stage 8.0 (TID 183). 6787 bytes result sent to driver -26/04/01 06:04:35 INFO TaskSetManager: Starting task 179.0 in stage 8.0 (TID 187) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:04:35 INFO TaskSetManager: Finished task 175.0 in stage 8.0 (TID 183) in 6430 ms on 10.0.0.133 (executor driver) (176/208) -26/04/01 06:04:35 INFO Executor: Running task 179.0 in stage 8.0 (TID 187) -26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:41 INFO Executor: Finished task 176.0 in stage 8.0 (TID 184). 6787 bytes result sent to driver -26/04/01 06:04:41 INFO TaskSetManager: Starting task 180.0 in stage 8.0 (TID 188) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:04:41 INFO Executor: Running task 180.0 in stage 8.0 (TID 188) -26/04/01 06:04:41 INFO TaskSetManager: Finished task 176.0 in stage 8.0 (TID 184) in 6454 ms on 10.0.0.133 (executor driver) (177/208) -26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:41 INFO Executor: Finished task 177.0 in stage 8.0 (TID 185). 6787 bytes result sent to driver -26/04/01 06:04:41 INFO Executor: Finished task 178.0 in stage 8.0 (TID 186). 6787 bytes result sent to driver -26/04/01 06:04:41 INFO TaskSetManager: Starting task 181.0 in stage 8.0 (TID 189) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:04:41 INFO TaskSetManager: Finished task 177.0 in stage 8.0 (TID 185) in 6452 ms on 10.0.0.133 (executor driver) (178/208) -26/04/01 06:04:41 INFO Executor: Running task 181.0 in stage 8.0 (TID 189) -26/04/01 06:04:41 INFO TaskSetManager: Starting task 182.0 in stage 8.0 (TID 190) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:41 INFO TaskSetManager: Finished task 178.0 in stage 8.0 (TID 186) in 6438 ms on 10.0.0.133 (executor driver) (179/208) -26/04/01 06:04:41 INFO Executor: Running task 182.0 in stage 8.0 (TID 190) -26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:41 INFO Executor: Finished task 179.0 in stage 8.0 (TID 187). 6787 bytes result sent to driver -26/04/01 06:04:41 INFO TaskSetManager: Starting task 183.0 in stage 8.0 (TID 191) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:41 INFO TaskSetManager: Finished task 179.0 in stage 8.0 (TID 187) in 6446 ms on 10.0.0.133 (executor driver) (180/208) -26/04/01 06:04:41 INFO Executor: Running task 183.0 in stage 8.0 (TID 191) -26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:47 INFO Executor: Finished task 180.0 in stage 8.0 (TID 188). 6787 bytes result sent to driver -26/04/01 06:04:47 INFO TaskSetManager: Starting task 184.0 in stage 8.0 (TID 192) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:47 INFO TaskSetManager: Finished task 180.0 in stage 8.0 (TID 188) in 6482 ms on 10.0.0.133 (executor driver) (181/208) -26/04/01 06:04:47 INFO Executor: Running task 184.0 in stage 8.0 (TID 192) -26/04/01 06:04:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:48 INFO Executor: Finished task 181.0 in stage 8.0 (TID 189). 6787 bytes result sent to driver -26/04/01 06:04:48 INFO Executor: Finished task 182.0 in stage 8.0 (TID 190). 6787 bytes result sent to driver -26/04/01 06:04:48 INFO TaskSetManager: Starting task 185.0 in stage 8.0 (TID 193) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:48 INFO TaskSetManager: Starting task 186.0 in stage 8.0 (TID 194) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:04:48 INFO Executor: Running task 185.0 in stage 8.0 (TID 193) -26/04/01 06:04:48 INFO TaskSetManager: Finished task 181.0 in stage 8.0 (TID 189) in 6489 ms on 10.0.0.133 (executor driver) (182/208) -26/04/01 06:04:48 INFO Executor: Running task 186.0 in stage 8.0 (TID 194) -26/04/01 06:04:48 INFO TaskSetManager: Finished task 182.0 in stage 8.0 (TID 190) in 6488 ms on 10.0.0.133 (executor driver) (183/208) -26/04/01 06:04:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:48 INFO Executor: Finished task 183.0 in stage 8.0 (TID 191). 6787 bytes result sent to driver -26/04/01 06:04:48 INFO TaskSetManager: Starting task 187.0 in stage 8.0 (TID 195) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:04:48 INFO TaskSetManager: Finished task 183.0 in stage 8.0 (TID 191) in 6484 ms on 10.0.0.133 (executor driver) (184/208) -26/04/01 06:04:48 INFO Executor: Running task 187.0 in stage 8.0 (TID 195) -26/04/01 06:04:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:54 INFO Executor: Finished task 184.0 in stage 8.0 (TID 192). 6787 bytes result sent to driver -26/04/01 06:04:54 INFO TaskSetManager: Starting task 188.0 in stage 8.0 (TID 196) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:54 INFO Executor: Running task 188.0 in stage 8.0 (TID 196) -26/04/01 06:04:54 INFO TaskSetManager: Finished task 184.0 in stage 8.0 (TID 192) in 6490 ms on 10.0.0.133 (executor driver) (185/208) -26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:54 INFO Executor: Finished task 186.0 in stage 8.0 (TID 194). 6787 bytes result sent to driver -26/04/01 06:04:54 INFO TaskSetManager: Starting task 189.0 in stage 8.0 (TID 197) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:54 INFO TaskSetManager: Finished task 186.0 in stage 8.0 (TID 194) in 6482 ms on 10.0.0.133 (executor driver) (186/208) -26/04/01 06:04:54 INFO Executor: Running task 189.0 in stage 8.0 (TID 197) -26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:54 INFO Executor: Finished task 185.0 in stage 8.0 (TID 193). 6787 bytes result sent to driver -26/04/01 06:04:54 INFO TaskSetManager: Starting task 190.0 in stage 8.0 (TID 198) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:54 INFO Executor: Running task 190.0 in stage 8.0 (TID 198) -26/04/01 06:04:54 INFO TaskSetManager: Finished task 185.0 in stage 8.0 (TID 193) in 6485 ms on 10.0.0.133 (executor driver) (187/208) -26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:54 INFO Executor: Finished task 187.0 in stage 8.0 (TID 195). 6787 bytes result sent to driver -26/04/01 06:04:54 INFO TaskSetManager: Starting task 191.0 in stage 8.0 (TID 199) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:04:54 INFO Executor: Running task 191.0 in stage 8.0 (TID 199) -26/04/01 06:04:54 INFO TaskSetManager: Finished task 187.0 in stage 8.0 (TID 195) in 6484 ms on 10.0.0.133 (executor driver) (188/208) -26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:04:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:00 INFO Executor: Finished task 188.0 in stage 8.0 (TID 196). 6787 bytes result sent to driver -26/04/01 06:05:00 INFO TaskSetManager: Starting task 192.0 in stage 8.0 (TID 200) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:05:00 INFO TaskSetManager: Finished task 188.0 in stage 8.0 (TID 196) in 6477 ms on 10.0.0.133 (executor driver) (189/208) -26/04/01 06:05:00 INFO Executor: Running task 192.0 in stage 8.0 (TID 200) -26/04/01 06:05:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:01 INFO Executor: Finished task 189.0 in stage 8.0 (TID 197). 6787 bytes result sent to driver -26/04/01 06:05:01 INFO Executor: Finished task 190.0 in stage 8.0 (TID 198). 6787 bytes result sent to driver -26/04/01 06:05:01 INFO TaskSetManager: Starting task 193.0 in stage 8.0 (TID 201) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:05:01 INFO Executor: Running task 193.0 in stage 8.0 (TID 201) -26/04/01 06:05:01 INFO TaskSetManager: Starting task 194.0 in stage 8.0 (TID 202) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:05:01 INFO TaskSetManager: Finished task 189.0 in stage 8.0 (TID 197) in 6481 ms on 10.0.0.133 (executor driver) (190/208) -26/04/01 06:05:01 INFO Executor: Running task 194.0 in stage 8.0 (TID 202) -26/04/01 06:05:01 INFO TaskSetManager: Finished task 190.0 in stage 8.0 (TID 198) in 6478 ms on 10.0.0.133 (executor driver) (191/208) -26/04/01 06:05:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:01 INFO Executor: Finished task 191.0 in stage 8.0 (TID 199). 6787 bytes result sent to driver -26/04/01 06:05:01 INFO TaskSetManager: Starting task 195.0 in stage 8.0 (TID 203) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:05:01 INFO TaskSetManager: Finished task 191.0 in stage 8.0 (TID 199) in 6478 ms on 10.0.0.133 (executor driver) (192/208) -26/04/01 06:05:01 INFO Executor: Running task 195.0 in stage 8.0 (TID 203) -26/04/01 06:05:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:04 INFO Executor: Finished task 192.0 in stage 8.0 (TID 200). 6787 bytes result sent to driver -26/04/01 06:05:04 INFO TaskSetManager: Starting task 196.0 in stage 8.0 (TID 204) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:05:04 INFO Executor: Running task 196.0 in stage 8.0 (TID 204) -26/04/01 06:05:04 INFO TaskSetManager: Finished task 192.0 in stage 8.0 (TID 200) in 4019 ms on 10.0.0.133 (executor driver) (193/208) -26/04/01 06:05:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:05 INFO Executor: Finished task 193.0 in stage 8.0 (TID 201). 6787 bytes result sent to driver -26/04/01 06:05:05 INFO TaskSetManager: Starting task 197.0 in stage 8.0 (TID 205) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:05:05 INFO TaskSetManager: Finished task 193.0 in stage 8.0 (TID 201) in 4016 ms on 10.0.0.133 (executor driver) (194/208) -26/04/01 06:05:05 INFO Executor: Running task 197.0 in stage 8.0 (TID 205) -26/04/01 06:05:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:05 INFO Executor: Finished task 194.0 in stage 8.0 (TID 202). 6787 bytes result sent to driver -26/04/01 06:05:05 INFO TaskSetManager: Starting task 198.0 in stage 8.0 (TID 206) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:05:05 INFO TaskSetManager: Finished task 194.0 in stage 8.0 (TID 202) in 4016 ms on 10.0.0.133 (executor driver) (195/208) -26/04/01 06:05:05 INFO Executor: Running task 198.0 in stage 8.0 (TID 206) -26/04/01 06:05:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:05 INFO Executor: Finished task 195.0 in stage 8.0 (TID 203). 6787 bytes result sent to driver -26/04/01 06:05:05 INFO TaskSetManager: Starting task 199.0 in stage 8.0 (TID 207) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:05:05 INFO TaskSetManager: Finished task 195.0 in stage 8.0 (TID 203) in 4022 ms on 10.0.0.133 (executor driver) (196/208) -26/04/01 06:05:05 INFO Executor: Running task 199.0 in stage 8.0 (TID 207) -26/04/01 06:05:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:08 INFO Executor: Finished task 196.0 in stage 8.0 (TID 204). 6787 bytes result sent to driver -26/04/01 06:05:08 INFO TaskSetManager: Starting task 200.0 in stage 8.0 (TID 208) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9346 bytes) -26/04/01 06:05:08 INFO Executor: Running task 200.0 in stage 8.0 (TID 208) -26/04/01 06:05:08 INFO TaskSetManager: Finished task 196.0 in stage 8.0 (TID 204) in 4014 ms on 10.0.0.133 (executor driver) (197/208) -26/04/01 06:05:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:09 INFO Executor: Finished task 198.0 in stage 8.0 (TID 206). 6787 bytes result sent to driver -26/04/01 06:05:09 INFO Executor: Finished task 197.0 in stage 8.0 (TID 205). 6787 bytes result sent to driver -26/04/01 06:05:09 INFO TaskSetManager: Starting task 201.0 in stage 8.0 (TID 209) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:05:09 INFO Executor: Running task 201.0 in stage 8.0 (TID 209) -26/04/01 06:05:09 INFO TaskSetManager: Starting task 202.0 in stage 8.0 (TID 210) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:05:09 INFO TaskSetManager: Finished task 198.0 in stage 8.0 (TID 206) in 4016 ms on 10.0.0.133 (executor driver) (198/208) -26/04/01 06:05:09 INFO Executor: Running task 202.0 in stage 8.0 (TID 210) -26/04/01 06:05:09 INFO TaskSetManager: Finished task 197.0 in stage 8.0 (TID 205) in 4017 ms on 10.0.0.133 (executor driver) (199/208) -26/04/01 06:05:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:09 INFO Executor: Finished task 199.0 in stage 8.0 (TID 207). 6787 bytes result sent to driver -26/04/01 06:05:09 INFO TaskSetManager: Starting task 203.0 in stage 8.0 (TID 211) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:05:09 INFO TaskSetManager: Finished task 199.0 in stage 8.0 (TID 207) in 4016 ms on 10.0.0.133 (executor driver) (200/208) -26/04/01 06:05:09 INFO Executor: Running task 203.0 in stage 8.0 (TID 211) -26/04/01 06:05:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:12 INFO Executor: Finished task 200.0 in stage 8.0 (TID 208). 6787 bytes result sent to driver -26/04/01 06:05:12 INFO TaskSetManager: Starting task 204.0 in stage 8.0 (TID 212) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:05:12 INFO TaskSetManager: Finished task 200.0 in stage 8.0 (TID 208) in 3996 ms on 10.0.0.133 (executor driver) (201/208) -26/04/01 06:05:12 INFO Executor: Running task 204.0 in stage 8.0 (TID 212) -26/04/01 06:05:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:13 INFO Executor: Finished task 202.0 in stage 8.0 (TID 210). 6787 bytes result sent to driver -26/04/01 06:05:13 INFO TaskSetManager: Starting task 205.0 in stage 8.0 (TID 213) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:05:13 INFO TaskSetManager: Finished task 202.0 in stage 8.0 (TID 210) in 3997 ms on 10.0.0.133 (executor driver) (202/208) -26/04/01 06:05:13 INFO Executor: Running task 205.0 in stage 8.0 (TID 213) -26/04/01 06:05:13 INFO Executor: Finished task 201.0 in stage 8.0 (TID 209). 6787 bytes result sent to driver -26/04/01 06:05:13 INFO TaskSetManager: Starting task 206.0 in stage 8.0 (TID 214) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:05:13 INFO Executor: Running task 206.0 in stage 8.0 (TID 214) -26/04/01 06:05:13 INFO TaskSetManager: Finished task 201.0 in stage 8.0 (TID 209) in 3998 ms on 10.0.0.133 (executor driver) (203/208) -26/04/01 06:05:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:13 INFO Executor: Finished task 203.0 in stage 8.0 (TID 211). 6787 bytes result sent to driver -26/04/01 06:05:13 INFO TaskSetManager: Starting task 207.0 in stage 8.0 (TID 215) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:05:13 INFO TaskSetManager: Finished task 203.0 in stage 8.0 (TID 211) in 3996 ms on 10.0.0.133 (executor driver) (204/208) -26/04/01 06:05:13 INFO Executor: Running task 207.0 in stage 8.0 (TID 215) -26/04/01 06:05:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:16 INFO Executor: Finished task 204.0 in stage 8.0 (TID 212). 6787 bytes result sent to driver -26/04/01 06:05:16 INFO TaskSetManager: Finished task 204.0 in stage 8.0 (TID 212) in 3997 ms on 10.0.0.133 (executor driver) (205/208) -26/04/01 06:05:17 INFO Executor: Finished task 205.0 in stage 8.0 (TID 213). 6787 bytes result sent to driver -26/04/01 06:05:17 INFO Executor: Finished task 206.0 in stage 8.0 (TID 214). 6787 bytes result sent to driver -26/04/01 06:05:17 INFO TaskSetManager: Finished task 205.0 in stage 8.0 (TID 213) in 3994 ms on 10.0.0.133 (executor driver) (206/208) -26/04/01 06:05:17 INFO TaskSetManager: Finished task 206.0 in stage 8.0 (TID 214) in 3994 ms on 10.0.0.133 (executor driver) (207/208) -26/04/01 06:05:17 INFO Executor: Finished task 207.0 in stage 8.0 (TID 215). 6787 bytes result sent to driver -26/04/01 06:05:17 INFO TaskSetManager: Finished task 207.0 in stage 8.0 (TID 215) in 3967 ms on 10.0.0.133 (executor driver) (208/208) -26/04/01 06:05:17 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool -26/04/01 06:05:17 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 325.669 s -26/04/01 06:05:17 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:05:17 INFO DAGScheduler: running: Set() -26/04/01 06:05:17 INFO DAGScheduler: waiting: Set() -26/04/01 06:05:17 INFO DAGScheduler: failed: Set() -26/04/01 06:05:17 INFO ShufflePartitionsUtil: For shuffle(0), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 06:05:17 INFO CodeGenerator: Code generated in 61.275208 ms -26/04/01 06:05:17 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:05:17 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:05:17 INFO DAGScheduler: Final stage: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:05:17 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 9) -26/04/01 06:05:17 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:17 INFO DAGScheduler: Submitting ResultStage 10 (MapPartitionsRDD[23] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:05:17 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 49.9 KiB, free 8.6 GiB) -26/04/01 06:05:17 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 10.4 KiB, free 8.6 GiB) -26/04/01 06:05:17 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:58235 (size: 10.4 KiB, free: 8.6 GiB) -26/04/01 06:05:17 INFO SparkContext: Created broadcast 10 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:17 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 10 (MapPartitionsRDD[23] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:05:17 INFO TaskSchedulerImpl: Adding task set 10.0 with 1 tasks resource profile 0 -26/04/01 06:05:17 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 216) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) -26/04/01 06:05:17 INFO Executor: Running task 0.0 in stage 10.0 (TID 216) -26/04/01 06:05:17 INFO ShuffleBlockFetcherIterator: Getting 208 (853.9 KiB) non-empty blocks including 208 (853.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:05:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms -26/04/01 06:05:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:17 INFO CodeGenerator: Code generated in 5.76175 ms -26/04/01 06:05:17 INFO Executor: Finished task 0.0 in stage 10.0 (TID 216). 5160 bytes result sent to driver -26/04/01 06:05:17 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 216) in 342 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:05:17 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool -26/04/01 06:05:17 INFO DAGScheduler: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.345 s -26/04/01 06:05:17 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:05:17 INFO TaskSchedulerImpl: Killing all running tasks in stage 10: Stage finished -26/04/01 06:05:17 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.349708 s -26/04/01 06:05:17 INFO DAGScheduler: Registering RDD 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 -26/04/01 06:05:17 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:05:17 INFO DAGScheduler: Final stage: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:05:17 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 11) -26/04/01 06:05:17 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:17 INFO DAGScheduler: Submitting ShuffleMapStage 12 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:05:17 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 51.7 KiB, free 8.6 GiB) -26/04/01 06:05:17 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 11.2 KiB, free 8.6 GiB) -26/04/01 06:05:17 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:58235 (size: 11.2 KiB, free: 8.6 GiB) -26/04/01 06:05:17 INFO SparkContext: Created broadcast 11 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:17 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 12 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:05:17 INFO TaskSchedulerImpl: Adding task set 12.0 with 1 tasks resource profile 0 -26/04/01 06:05:17 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 217) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9404 bytes) -26/04/01 06:05:17 INFO Executor: Running task 0.0 in stage 12.0 (TID 217) -26/04/01 06:05:17 INFO CodeGenerator: Code generated in 3.546 ms -26/04/01 06:05:17 INFO ShuffleBlockFetcherIterator: Getting 208 (853.9 KiB) non-empty blocks including 208 (853.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:05:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:05:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:18 INFO BlockManagerInfo: Removed broadcast_10_piece0 on 10.0.0.133:58235 in memory (size: 10.4 KiB, free: 8.6 GiB) -26/04/01 06:05:18 INFO Executor: Finished task 0.0 in stage 12.0 (TID 217). 6609 bytes result sent to driver -26/04/01 06:05:18 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 217) in 313 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:05:18 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool -26/04/01 06:05:18 INFO DAGScheduler: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.316 s -26/04/01 06:05:18 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:05:18 INFO DAGScheduler: running: Set() -26/04/01 06:05:18 INFO DAGScheduler: waiting: Set() -26/04/01 06:05:18 INFO DAGScheduler: failed: Set() -26/04/01 06:05:18 INFO ShufflePartitionsUtil: For shuffle(1), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 06:05:18 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:05:18 INFO DAGScheduler: Got job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:05:18 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:05:18 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 14) -26/04/01 06:05:18 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:18 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:05:18 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 20.4 KiB, free 8.6 GiB) -26/04/01 06:05:18 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 9.0 KiB, free 8.6 GiB) -26/04/01 06:05:18 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:58235 (size: 9.0 KiB, free: 8.6 GiB) -26/04/01 06:05:18 INFO SparkContext: Created broadcast 12 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:18 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:05:18 INFO TaskSchedulerImpl: Adding task set 15.0 with 1 tasks resource profile 0 -26/04/01 06:05:18 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 218) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) -26/04/01 06:05:18 INFO Executor: Running task 0.0 in stage 15.0 (TID 218) -26/04/01 06:05:18 INFO ShuffleBlockFetcherIterator: Getting 1 (2.6 KiB) non-empty blocks including 1 (2.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:05:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:05:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:18 INFO Executor: Finished task 0.0 in stage 15.0 (TID 218). 5560 bytes result sent to driver -26/04/01 06:05:18 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 218) in 9 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:05:18 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool -26/04/01 06:05:18 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.012 s -26/04/01 06:05:18 INFO DAGScheduler: Job 11 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:05:18 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished -26/04/01 06:05:18 INFO DAGScheduler: Job 11 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.012342 s -26/04/01 06:05:18 INFO SparkContext: SparkContext is stopping with exitCode 0. -26/04/01 06:05:18 INFO CometDriverPlugin: CometDriverPlugin shutdown -26/04/01 06:05:18 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! -26/04/01 06:05:18 INFO MemoryStore: MemoryStore cleared -26/04/01 06:05:18 INFO BlockManager: BlockManager stopped -26/04/01 06:05:18 INFO BlockManagerMaster: BlockManagerMaster stopped -26/04/01 06:05:18 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! -26/04/01 06:05:18 INFO SparkContext: Successfully stopped SparkContext -26/04/01 06:05:18 INFO ShutdownHookManager: Shutdown hook called -26/04/01 06:05:18 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-8b7018bf-f101-4273-93fa-9cf7eb6d820a/pyspark-b3568a12-6421-40dc-b3fc-0213601717c5 -26/04/01 06:05:18 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-960962b7-b8cb-4781-a172-17193aefe80f -26/04/01 06:05:18 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-8b7018bf-f101-4273-93fa-9cf7eb6d820a - 332.80 real 1302.40 user 5.62 sys - 712359936 maximum resident set size - 0 average shared memory size - 0 average unshared data size - 0 average unshared stack size - 71057 page reclaims - 63 page faults - 0 swaps - 0 block input operations - 0 block output operations - 1275 messages sent - 2361 messages received - 17 signals received - 31134 voluntary context switches - 203718 involuntary context switches - 20342884625324 instructions retired - 4776772444264 cycles elapsed - 597542592 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.log b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.log deleted file mode 100644 index bea807a74f..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.log +++ /dev/null @@ -1,317 +0,0 @@ -Registering table customer from /opt/tpch/sf100/customer -Registering table lineitem from /opt/tpch/sf100/lineitem -Registering table nation from /opt/tpch/sf100/nation -Registering table orders from /opt/tpch/sf100/orders -Registering table part from /opt/tpch/sf100/part -Registering table partsupp from /opt/tpch/sf100/partsupp -Registering table region from /opt/tpch/sf100/region -Registering table supplier from /opt/tpch/sf100/supplier - -============================================================ -Starting iteration 1 of 1 -============================================================ - -Running query 5 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q5.sql -Executing: -- CometBench-H query 5 derived from TPC-H query 5 under the terms of the TPC Fair Use Policy. --- TP... -== Physical Plan == -AdaptiveSparkPlan (44) -+- CometSort (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometExchange (40) - +- CometHashAggregate (39) - +- CometProject (38) - +- CometBroadcastHashJoin (37) - :- CometProject (32) - : +- CometBroadcastHashJoin (31) - : :- CometProject (27) - : : +- CometSortMergeJoin (26) - : : :- CometSort (21) - : : : +- CometExchange (20) - : : : +- CometProject (19) - : : : +- CometSortMergeJoin (18) - : : : :- CometSort (13) - : : : : +- CometExchange (12) - : : : : +- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (4) - : : : : : +- CometExchange (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometFilter (6) - : : : : +- CometNativeScan parquet (5) - : : : +- CometSort (17) - : : : +- CometExchange (16) - : : : +- CometFilter (15) - : : : +- CometNativeScan parquet (14) - : : +- CometSort (25) - : : +- CometExchange (24) - : : +- CometFilter (23) - : : +- CometNativeScan parquet (22) - : +- CometBroadcastExchange (30) - : +- CometFilter (29) - : +- CometNativeScan parquet (28) - +- CometBroadcastExchange (36) - +- CometProject (35) - +- CometFilter (34) - +- CometNativeScan parquet (33) - - -(1) CometNativeScan parquet -Output [2]: [c_custkey#0L, c_nationkey#3L] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/customer] -PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [c_custkey#0L, c_nationkey#3L] -Condition : (isnotnull(c_custkey#0L) AND isnotnull(c_nationkey#3L)) - -(3) CometExchange -Input [2]: [c_custkey#0L, c_nationkey#3L] -Arguments: hashpartitioning(c_custkey#0L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=412] - -(4) CometSort -Input [2]: [c_custkey#0L, c_nationkey#3L] -Arguments: [c_custkey#0L, c_nationkey#3L], [c_custkey#0L ASC NULLS FIRST] - -(5) CometNativeScan parquet -Output [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/orders] -PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1994-01-01), LessThan(o_orderdate,1995-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] -Condition : ((((isnotnull(o_orderdate#60) AND (o_orderdate#60 >= 1994-01-01)) AND (o_orderdate#60 < 1995-01-01)) AND isnotnull(o_custkey#57L)) AND isnotnull(o_orderkey#56L)) - -(7) CometProject -Input [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] -Arguments: [o_orderkey#56L, o_custkey#57L], [o_orderkey#56L, o_custkey#57L] - -(8) CometExchange -Input [2]: [o_orderkey#56L, o_custkey#57L] -Arguments: hashpartitioning(o_custkey#57L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=416] - -(9) CometSort -Input [2]: [o_orderkey#56L, o_custkey#57L] -Arguments: [o_orderkey#56L, o_custkey#57L], [o_custkey#57L ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [2]: [c_custkey#0L, c_nationkey#3L] -Right output [2]: [o_orderkey#56L, o_custkey#57L] -Arguments: [c_custkey#0L], [o_custkey#57L], Inner - -(11) CometProject -Input [4]: [c_custkey#0L, c_nationkey#3L, o_orderkey#56L, o_custkey#57L] -Arguments: [c_nationkey#3L, o_orderkey#56L], [c_nationkey#3L, o_orderkey#56L] - -(12) CometExchange -Input [2]: [c_nationkey#3L, o_orderkey#56L] -Arguments: hashpartitioning(o_orderkey#56L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=420] - -(13) CometSort -Input [2]: [c_nationkey#3L, o_orderkey#56L] -Arguments: [c_nationkey#3L, o_orderkey#56L], [o_orderkey#56L ASC NULLS FIRST] - -(14) CometNativeScan parquet -Output [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] -PushedFilters: [IsNotNull(l_orderkey), IsNotNull(l_suppkey)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Condition : (isnotnull(l_orderkey#16L) AND isnotnull(l_suppkey#18L)) - -(16) CometExchange -Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: hashpartitioning(l_orderkey#16L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=424] - -(17) CometSort -Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22], [l_orderkey#16L ASC NULLS FIRST] - -(18) CometSortMergeJoin -Left output [2]: [c_nationkey#3L, o_orderkey#56L] -Right output [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: [o_orderkey#56L], [l_orderkey#16L], Inner - -(19) CometProject -Input [6]: [c_nationkey#3L, o_orderkey#56L, l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22], [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] - -(20) CometExchange -Input [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: hashpartitioning(l_suppkey#18L, c_nationkey#3L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=428] - -(21) CometSort -Input [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22], [l_suppkey#18L ASC NULLS FIRST, c_nationkey#3L ASC NULLS FIRST] - -(22) CometNativeScan parquet -Output [2]: [s_suppkey#108L, s_nationkey#111L] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/supplier] -PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] -ReadSchema: struct - -(23) CometFilter -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Condition : (isnotnull(s_suppkey#108L) AND isnotnull(s_nationkey#111L)) - -(24) CometExchange -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: hashpartitioning(s_suppkey#108L, s_nationkey#111L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=432] - -(25) CometSort -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: [s_suppkey#108L, s_nationkey#111L], [s_suppkey#108L ASC NULLS FIRST, s_nationkey#111L ASC NULLS FIRST] - -(26) CometSortMergeJoin -Left output [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Right output [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: [l_suppkey#18L, c_nationkey#3L], [s_suppkey#108L, s_nationkey#111L], Inner - -(27) CometProject -Input [6]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22, s_suppkey#108L, s_nationkey#111L] -Arguments: [l_extendedprice#21, l_discount#22, s_nationkey#111L], [l_extendedprice#21, l_discount#22, s_nationkey#111L] - -(28) CometNativeScan parquet -Output [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/nation] -PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] -Condition : ((isnotnull(n_nationkey#48L) AND isnotnull(n_regionkey#50L)) AND might_contain(Subquery subquery#128, [id=#161], xxhash64(n_regionkey#50L, 42))) - -(30) CometBroadcastExchange -Input [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] -Arguments: [n_nationkey#48L, n_name#49, n_regionkey#50L] - -(31) CometBroadcastHashJoin -Left output [3]: [l_extendedprice#21, l_discount#22, s_nationkey#111L] -Right output [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] -Arguments: [s_nationkey#111L], [n_nationkey#48L], Inner, BuildRight - -(32) CometProject -Input [6]: [l_extendedprice#21, l_discount#22, s_nationkey#111L, n_nationkey#48L, n_name#49, n_regionkey#50L] -Arguments: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L], [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L] - -(33) CometNativeScan parquet -Output [2]: [r_regionkey#102L, r_name#103] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/region] -PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AFRICA), IsNotNull(r_regionkey)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [r_regionkey#102L, r_name#103] -Condition : ((isnotnull(r_name#103) AND (r_name#103 = AFRICA)) AND isnotnull(r_regionkey#102L)) - -(35) CometProject -Input [2]: [r_regionkey#102L, r_name#103] -Arguments: [r_regionkey#102L], [r_regionkey#102L] - -(36) CometBroadcastExchange -Input [1]: [r_regionkey#102L] -Arguments: [r_regionkey#102L] - -(37) CometBroadcastHashJoin -Left output [4]: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L] -Right output [1]: [r_regionkey#102L] -Arguments: [n_regionkey#50L], [r_regionkey#102L], Inner, BuildRight - -(38) CometProject -Input [5]: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L, r_regionkey#102L] -Arguments: [l_extendedprice#21, l_discount#22, n_name#49], [l_extendedprice#21, l_discount#22, n_name#49] - -(39) CometHashAggregate -Input [3]: [l_extendedprice#21, l_discount#22, n_name#49] -Keys [1]: [n_name#49] -Functions [1]: [partial_sum((l_extendedprice#21 * (1 - l_discount#22)))] - -(40) CometExchange -Input [3]: [n_name#49, sum#131, isEmpty#132] -Arguments: hashpartitioning(n_name#49, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=445] - -(41) CometHashAggregate -Input [3]: [n_name#49, sum#131, isEmpty#132] -Keys [1]: [n_name#49] -Functions [1]: [sum((l_extendedprice#21 * (1 - l_discount#22)))] - -(42) CometExchange -Input [2]: [n_name#49, revenue#122] -Arguments: rangepartitioning(revenue#122 DESC NULLS LAST, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=447] - -(43) CometSort -Input [2]: [n_name#49, revenue#122] -Arguments: [n_name#49, revenue#122], [revenue#122 DESC NULLS LAST] - -(44) AdaptiveSparkPlan -Output [2]: [n_name#49, revenue#122] -Arguments: isFinalPlan=false - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 29 Hosting Expression = Subquery subquery#128, [id=#161] -AdaptiveSparkPlan (51) -+- CometHashAggregate (50) - +- CometExchange (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometFilter (46) - +- CometNativeScan parquet (45) - - -(45) CometNativeScan parquet -Output [2]: [r_regionkey#102L, r_name#103] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/region] -PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AFRICA), IsNotNull(r_regionkey)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [r_regionkey#102L, r_name#103] -Condition : ((isnotnull(r_name#103) AND (r_name#103 = AFRICA)) AND isnotnull(r_regionkey#102L)) - -(47) CometProject -Input [2]: [r_regionkey#102L, r_name#103] -Arguments: [r_regionkey#102L], [r_regionkey#102L] - -(48) CometHashAggregate -Input [1]: [r_regionkey#102L] -Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)] - -(49) CometExchange -Input [1]: [buf#134] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=159] - -(50) CometHashAggregate -Input [1]: [buf#134] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)] - -(51) AdaptiveSparkPlan -Output [1]: [bloomFilter#127] -Arguments: isFinalPlan=false - - - -Query 5 returned 5 rows, hash=f1e68aab4aa9d0988709357fc210d775 -Query 5 took 482.58 seconds - -Iteration 1 took 482.58 seconds - -Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/comet-offheap4g-q5-tpch-1775045606478.json diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.time b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.time deleted file mode 100644 index e5c7eae40a..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q5.time +++ /dev/null @@ -1,5639 +0,0 @@ -26/04/01 06:05:19 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) -26/04/01 06:05:19 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address -26/04/01 06:05:19 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable -26/04/01 06:05:19 INFO SparkContext: Running Spark version 3.5.8 -26/04/01 06:05:19 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 06:05:19 INFO SparkContext: Java version 17.0.17 -26/04/01 06:05:19 INFO ResourceUtils: ============================================================== -26/04/01 06:05:19 INFO ResourceUtils: No custom resources configured for spark.driver. -26/04/01 06:05:19 INFO ResourceUtils: ============================================================== -26/04/01 06:05:19 INFO SparkContext: Submitted application: comet-offheap4g-q5 benchmark derived from tpch -26/04/01 06:05:19 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 4096, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) -26/04/01 06:05:19 INFO ResourceProfile: Limiting resource is cpu -26/04/01 06:05:19 INFO ResourceProfileManager: Added ResourceProfile id: 0 -26/04/01 06:05:19 INFO SecurityManager: Changing view acls to: andy -26/04/01 06:05:19 INFO SecurityManager: Changing modify acls to: andy -26/04/01 06:05:19 INFO SecurityManager: Changing view acls groups to: -26/04/01 06:05:19 INFO SecurityManager: Changing modify acls groups to: -26/04/01 06:05:19 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY -26/04/01 06:05:19 INFO Utils: Successfully started service 'sparkDriver' on port 58257. -26/04/01 06:05:19 INFO SparkEnv: Registering MapOutputTracker -26/04/01 06:05:19 INFO SparkEnv: Registering BlockManagerMaster -26/04/01 06:05:19 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information -26/04/01 06:05:19 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up -26/04/01 06:05:19 INFO SparkEnv: Registering BlockManagerMasterHeartbeat -26/04/01 06:05:19 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-833f884d-78cd-4dcb-b53f-b68f98c95068 -26/04/01 06:05:19 INFO MemoryStore: MemoryStore started with capacity 8.6 GiB -26/04/01 06:05:19 INFO SparkEnv: Registering OutputCommitCoordinator -26/04/01 06:05:19 INFO SparkContext: Added JAR file:///Users/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar at spark://10.0.0.133:58257/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775045119743 -26/04/01 06:05:19 INFO CometDriverPlugin: CometDriverPlugin init -26/04/01 06:05:19 INFO CometDriverPlugin: Setting spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions -26/04/01 06:05:19 INFO CometDriverPlugin: Comet metrics reporting is disabled. Set spark.comet.metrics.enabled=true to enable. -26/04/01 06:05:19 INFO CometDriverPlugin: Comet is running in unified memory mode and sharing off-heap memory with Spark -26/04/01 06:05:19 INFO DriverPluginContainer: Initialized driver component for plugin org.apache.spark.CometPlugin. -26/04/01 06:05:19 INFO Executor: Starting executor ID driver on host 10.0.0.133 -26/04/01 06:05:19 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 06:05:19 INFO Executor: Java version 17.0.17 -26/04/01 06:05:19 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' -26/04/01 06:05:19 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@656aa134 for default. -26/04/01 06:05:19 INFO Executor: Fetching spark://10.0.0.133:58257/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775045119743 -26/04/01 06:05:19 INFO TransportClientFactory: Successfully created connection to /10.0.0.133:58257 after 8 ms (0 ms spent in bootstraps) -26/04/01 06:05:19 INFO Utils: Fetching spark://10.0.0.133:58257/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-73a4ef89-b8ea-48f7-92a1-8593182866f3/userFiles-f8334189-3236-49b1-b1d8-d36506822e30/fetchFileTemp14484242616185578193.tmp -26/04/01 06:05:20 INFO Executor: Adding file:/private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-73a4ef89-b8ea-48f7-92a1-8593182866f3/userFiles-f8334189-3236-49b1-b1d8-d36506822e30/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to class loader default -26/04/01 06:05:20 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 58259. -26/04/01 06:05:20 INFO NettyBlockTransferService: Server created on 10.0.0.133:58259 -26/04/01 06:05:20 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy -26/04/01 06:05:20 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 58259, None) -26/04/01 06:05:20 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:58259 with 8.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 58259, None) -26/04/01 06:05:20 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 58259, None) -26/04/01 06:05:20 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 58259, None) -26/04/01 06:05:20 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. -26/04/01 06:05:20 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. -26/04/01 06:05:20 INFO InMemoryFileIndex: It took 17 ms to list leaf files for 1 paths. -26/04/01 06:05:20 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:05:20 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:05:20 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:05:20 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:05:20 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:20 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:05:20 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 06:05:20 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) -26/04/01 06:05:20 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:20 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:20 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:05:20 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 -26/04/01 06:05:20 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 06:05:20 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) -26/04/01 06:05:20 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 2041 bytes result sent to driver -26/04/01 06:05:20 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 112 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:05:20 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool -26/04/01 06:05:20 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.390 s -26/04/01 06:05:20 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:05:20 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished -26/04/01 06:05:20 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.401075 s -26/04/01 06:05:21 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:23 INFO NativeBase: Couldn't locate log file from either COMET_CONF_DIR or comet.log.file.path. Using default log configuration with INFO log level which emits to stderr -26/04/01 06:05:23 INFO core/src/lib.rs: Comet native library version 0.15.0 initialized -26/04/01 06:05:23 INFO NativeBase: Setting system property arrow.enable_unsafe_memory_access to true -26/04/01 06:05:23 INFO NativeBase: Setting system property arrow.enable_null_check_for_get to false -26/04/01 06:05:23 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. -26/04/01 06:05:23 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:05:23 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:05:23 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:05:23 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:05:23 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:23 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:05:23 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 06:05:23 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) -26/04/01 06:05:23 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:23 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:23 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:05:23 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 -26/04/01 06:05:23 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 06:05:23 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) -26/04/01 06:05:23 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2430 bytes result sent to driver -26/04/01 06:05:23 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 15 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:05:23 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool -26/04/01 06:05:23 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.024 s -26/04/01 06:05:23 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:05:23 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished -26/04/01 06:05:23 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.024869 s -26/04/01 06:05:23 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. -26/04/01 06:05:23 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:05:23 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:05:23 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:05:23 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:05:23 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:23 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:05:23 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 06:05:23 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) -26/04/01 06:05:23 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:23 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:23 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:05:23 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 -26/04/01 06:05:23 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 06:05:23 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) -26/04/01 06:05:23 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver -26/04/01 06:05:23 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:05:23 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool -26/04/01 06:05:23 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 06:05:23 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:05:23 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished -26/04/01 06:05:23 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.014269 s -26/04/01 06:05:23 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 06:05:23 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:05:23 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:05:23 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:05:23 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:05:23 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:23 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:05:23 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 06:05:23 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) -26/04/01 06:05:23 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:23 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:23 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:05:23 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 -26/04/01 06:05:23 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 06:05:23 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) -26/04/01 06:05:23 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver -26/04/01 06:05:23 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:05:23 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool -26/04/01 06:05:23 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 06:05:23 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:05:23 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished -26/04/01 06:05:23 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.014762 s -26/04/01 06:05:23 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 06:05:23 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:05:23 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:05:23 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:05:23 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:05:23 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:23 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:05:23 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 06:05:23 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) -26/04/01 06:05:23 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:23 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:23 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:23 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:05:23 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 -26/04/01 06:05:23 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9299 bytes) -26/04/01 06:05:23 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:23 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) -26/04/01 06:05:23 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:23 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver -26/04/01 06:05:23 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:05:23 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool -26/04/01 06:05:23 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 06:05:23 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:05:23 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished -26/04/01 06:05:23 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.014726 s -26/04/01 06:05:23 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 06:05:23 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:05:23 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:05:23 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:05:23 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:05:23 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:23 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:05:23 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 06:05:23 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) -26/04/01 06:05:23 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:23 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:23 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:23 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:05:23 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 -26/04/01 06:05:23 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 06:05:23 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) -26/04/01 06:05:23 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver -26/04/01 06:05:23 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:05:23 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool -26/04/01 06:05:23 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 06:05:23 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:05:23 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished -26/04/01 06:05:23 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.014210 s -26/04/01 06:05:23 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. -26/04/01 06:05:23 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:05:23 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:05:23 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:05:23 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:05:23 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:23 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:05:23 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 06:05:23 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) -26/04/01 06:05:23 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:23 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:23 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:05:23 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 -26/04/01 06:05:23 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 06:05:23 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) -26/04/01 06:05:23 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver -26/04/01 06:05:23 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 4 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:05:23 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool -26/04/01 06:05:23 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.010 s -26/04/01 06:05:23 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:05:23 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished -26/04/01 06:05:23 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.011783 s -26/04/01 06:05:23 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 06:05:23 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:05:23 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:05:23 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:05:23 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:05:23 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:23 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:05:23 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 06:05:23 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 8.6 GiB) -26/04/01 06:05:23 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:58259 (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:23 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:23 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:05:23 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 -26/04/01 06:05:23 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 06:05:23 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) -26/04/01 06:05:23 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver -26/04/01 06:05:23 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:05:23 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool -26/04/01 06:05:23 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.011 s -26/04/01 06:05:23 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:05:23 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished -26/04/01 06:05:23 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.012717 s -26/04/01 06:05:24 INFO FileSourceStrategy: Pushed Filters: IsNotNull(c_custkey),IsNotNull(c_nationkey) -26/04/01 06:05:24 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(c_custkey#0L),isnotnull(c_nationkey#3L) -26/04/01 06:05:24 INFO FileSourceStrategy: Pushed Filters: IsNotNull(o_orderdate),GreaterThanOrEqual(o_orderdate,1994-01-01),LessThan(o_orderdate,1995-01-01),IsNotNull(o_custkey),IsNotNull(o_orderkey) -26/04/01 06:05:24 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(o_orderdate#60),(o_orderdate#60 >= 1994-01-01),(o_orderdate#60 < 1995-01-01),isnotnull(o_custkey#57L),isnotnull(o_orderkey#56L) -26/04/01 06:05:24 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_orderkey),IsNotNull(l_suppkey) -26/04/01 06:05:24 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_orderkey#16L),isnotnull(l_suppkey#18L) -26/04/01 06:05:24 INFO FileSourceStrategy: Pushed Filters: IsNotNull(s_suppkey),IsNotNull(s_nationkey) -26/04/01 06:05:24 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(s_suppkey#108L),isnotnull(s_nationkey#111L) -26/04/01 06:05:24 INFO FileSourceStrategy: Pushed Filters: IsNotNull(n_nationkey),IsNotNull(n_regionkey) -26/04/01 06:05:24 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(n_nationkey#48L),isnotnull(n_regionkey#50L),might_contain(scalar-subquery#128 [], xxhash64(n_regionkey#50L, 42)) -26/04/01 06:05:24 INFO FileSourceStrategy: Pushed Filters: IsNotNull(r_name),EqualTo(r_name,AFRICA),IsNotNull(r_regionkey) -26/04/01 06:05:24 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(r_name#103),(r_name#103 = AFRICA),isnotnull(r_regionkey#102L) -26/04/01 06:05:24 INFO FileSourceStrategy: Pushed Filters: IsNotNull(r_name),EqualTo(r_name,AFRICA),IsNotNull(r_regionkey) -26/04/01 06:05:24 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(r_name#103),(r_name#103 = AFRICA),isnotnull(r_regionkey#102L) -26/04/01 06:05:24 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:58259 in memory (size: 37.9 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 55920126 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:58259 (size: 35.4 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:05:24 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO DAGScheduler: Registering RDD 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 -26/04/01 06:05:24 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:05:24 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:24 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 23.0 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 8.0 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:58259 (size: 8.0 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO SparkContext: Created broadcast 9 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:24 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 8.0 with 1 tasks resource profile 0 -26/04/01 06:05:24 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9260 bytes) -26/04/01 06:05:24 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) -26/04/01 06:05:24 INFO BaseAllocator: Debug mode disabled. Enable with the VM option -Darrow.memory.debug.allocator=true. -26/04/01 06:05:24 INFO DefaultAllocationManagerOption: allocation manager type not specified, using netty as the default type -26/04/01 06:05:24 INFO CheckAllocator: Using DefaultAllocationManager at memory/unsafe/DefaultAllocationManagerFactory.class -26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:24 INFO core/src/execution/jni_api.rs: Comet tokio runtime: using spark.executor.cores=4 worker threads -26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:24 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 6628 bytes result sent to driver -26/04/01 06:05:24 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 292 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:05:24 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool -26/04/01 06:05:24 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.306 s -26/04/01 06:05:24 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:05:24 INFO DAGScheduler: running: Set() -26/04/01 06:05:24 INFO DAGScheduler: waiting: Set() -26/04/01 06:05:24 INFO DAGScheduler: failed: Set() -26/04/01 06:05:24 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:05:24 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:05:24 INFO DAGScheduler: Final stage: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 9) -26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:24 INFO DAGScheduler: Submitting ResultStage 10 (MapPartitionsRDD[22] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 17.6 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 7.5 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:58259 (size: 7.5 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO SparkContext: Created broadcast 10 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:24 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 10 (MapPartitionsRDD[22] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 10.0 with 1 tasks resource profile 0 -26/04/01 06:05:24 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 9) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) -26/04/01 06:05:24 INFO Executor: Running task 0.0 in stage 10.0 (TID 9) -26/04/01 06:05:24 INFO ShuffleBlockFetcherIterator: Getting 1 (4.7 KiB) non-empty blocks including 1 (4.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:05:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 3 ms -26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:24 INFO Executor: Finished task 0.0 in stage 10.0 (TID 9). 9966 bytes result sent to driver -26/04/01 06:05:24 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 9) in 39 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:05:24 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool -26/04/01 06:05:24 INFO DAGScheduler: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.045 s -26/04/01 06:05:24 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:05:24 INFO TaskSchedulerImpl: Killing all running tasks in stage 10: Stage finished -26/04/01 06:05:24 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.049825 s -26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_13 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:58259 (size: 35.4 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO SparkContext: Created broadcast 11 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_13_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:58259 (size: 35.4 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO SparkContext: Created broadcast 12 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_13_piece0 in memory on 10.0.0.133:58259 (size: 35.4 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO SparkContext: Created broadcast 13 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:05:24 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO DAGScheduler: Registering RDD 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 -26/04/01 06:05:24 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 16 output partitions -26/04/01 06:05:24 INFO DAGScheduler: Final stage: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:24 INFO DAGScheduler: Submitting ShuffleMapStage 11 (MapPartitionsRDD[31] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_14 stored as values in memory (estimated size 17.6 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_14_piece0 stored as bytes in memory (estimated size 7.8 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_14_piece0 in memory on 10.0.0.133:58259 (size: 7.8 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO SparkContext: Created broadcast 14 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:24 INFO DAGScheduler: Submitting 16 missing tasks from ShuffleMapStage 11 (MapPartitionsRDD[31] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 11.0 with 16 tasks resource profile 0 -26/04/01 06:05:24 INFO DAGScheduler: Got job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:05:24 INFO DAGScheduler: Final stage: ResultStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:24 INFO TaskSetManager: Starting task 0.0 in stage 11.0 (TID 10) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:05:24 INFO DAGScheduler: Submitting ResultStage 12 (MapPartitionsRDD[26] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:05:24 INFO TaskSetManager: Starting task 1.0 in stage 11.0 (TID 11) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9334 bytes) -26/04/01 06:05:24 INFO TaskSetManager: Starting task 2.0 in stage 11.0 (TID 12) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:05:24 INFO TaskSetManager: Starting task 3.0 in stage 11.0 (TID 13) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9334 bytes) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_15 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO Executor: Running task 0.0 in stage 11.0 (TID 10) -26/04/01 06:05:24 INFO Executor: Running task 1.0 in stage 11.0 (TID 11) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_16 stored as values in memory (estimated size 14.0 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO Executor: Running task 2.0 in stage 11.0 (TID 12) -26/04/01 06:05:24 INFO Executor: Running task 3.0 in stage 11.0 (TID 13) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_16_piece0 stored as bytes in memory (estimated size 5.4 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_16_piece0 in memory on 10.0.0.133:58259 (size: 5.4 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO SparkContext: Created broadcast 16 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:24 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 12 (MapPartitionsRDD[26] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 12.0 with 1 tasks resource profile 0 -26/04/01 06:05:24 INFO BlockManagerInfo: Removed broadcast_10_piece0 on 10.0.0.133:58259 in memory (size: 7.5 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Removed broadcast_9_piece0 on 10.0.0.133:58259 in memory (size: 8.0 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_15_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_15_piece0 in memory on 10.0.0.133:58259 (size: 35.4 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO SparkContext: Created broadcast 15 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:05:24 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:24 INFO DAGScheduler: Registering RDD 34 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 2 -26/04/01 06:05:24 INFO DAGScheduler: Got map stage job 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 64 output partitions -26/04/01 06:05:24 INFO DAGScheduler: Final stage: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:24 INFO DAGScheduler: Submitting ShuffleMapStage 13 (MapPartitionsRDD[34] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_17 stored as values in memory (estimated size 23.3 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_17_piece0 stored as bytes in memory (estimated size 8.1 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_17_piece0 in memory on 10.0.0.133:58259 (size: 8.1 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO SparkContext: Created broadcast 17 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:24 INFO DAGScheduler: Submitting 64 missing tasks from ShuffleMapStage 13 (MapPartitionsRDD[34] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 13.0 with 64 tasks resource profile 0 -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_18 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_18_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_18_piece0 in memory on 10.0.0.133:58259 (size: 35.4 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO SparkContext: Created broadcast 18 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:05:24 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO DAGScheduler: Registering RDD 37 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 3 -26/04/01 06:05:24 INFO DAGScheduler: Got map stage job 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions -26/04/01 06:05:24 INFO DAGScheduler: Final stage: ShuffleMapStage 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:24 INFO DAGScheduler: Submitting ShuffleMapStage 14 (MapPartitionsRDD[37] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:05:24 INFO CometScanExec: Planning scan with bin packing, max size: 55920126 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_19 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_20 stored as values in memory (estimated size 18.4 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_20_piece0 stored as bytes in memory (estimated size 8.0 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_20_piece0 in memory on 10.0.0.133:58259 (size: 8.0 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO SparkContext: Created broadcast 20 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:24 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 14 (MapPartitionsRDD[37] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 14.0 with 208 tasks resource profile 0 -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_19_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_19_piece0 in memory on 10.0.0.133:58259 (size: 35.4 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO SparkContext: Created broadcast 19 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:05:24 INFO FileSourceScanExec: Planning scan with bin packing, max size: 55920126 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:05:24 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:05:24 INFO DAGScheduler: Got job 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:05:24 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:24 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:05:24 WARN DAGScheduler: Broadcasting large task binary with size 1113.7 KiB -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_21 stored as values in memory (estimated size 1113.7 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_21_piece0 stored as bytes in memory (estimated size 30.7 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_21_piece0 in memory on 10.0.0.133:58259 (size: 30.7 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO SparkContext: Created broadcast 21 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:24 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 15.0 with 1 tasks resource profile 0 -26/04/01 06:05:24 INFO DAGScheduler: Registering RDD 40 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 4 -26/04/01 06:05:24 INFO DAGScheduler: Got map stage job 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 4 output partitions -26/04/01 06:05:24 INFO DAGScheduler: Final stage: ShuffleMapStage 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:05:24 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:05:24 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:24 INFO DAGScheduler: Submitting ShuffleMapStage 16 (MapPartitionsRDD[40] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_22 stored as values in memory (estimated size 17.6 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO MemoryStore: Block broadcast_22_piece0 stored as bytes in memory (estimated size 7.8 KiB, free 8.6 GiB) -26/04/01 06:05:24 INFO BlockManagerInfo: Added broadcast_22_piece0 in memory on 10.0.0.133:58259 (size: 7.8 KiB, free: 8.6 GiB) -26/04/01 06:05:24 INFO SparkContext: Created broadcast 22 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:24 INFO DAGScheduler: Submitting 4 missing tasks from ShuffleMapStage 16 (MapPartitionsRDD[40] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3)) -26/04/01 06:05:24 INFO TaskSchedulerImpl: Adding task set 16.0 with 4 tasks resource profile 0 -26/04/01 06:05:25 INFO Executor: Finished task 3.0 in stage 11.0 (TID 13). 6649 bytes result sent to driver -26/04/01 06:05:25 INFO Executor: Finished task 1.0 in stage 11.0 (TID 11). 6649 bytes result sent to driver -26/04/01 06:05:25 INFO TaskSetManager: Starting task 4.0 in stage 11.0 (TID 14) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:05:25 INFO TaskSetManager: Finished task 3.0 in stage 11.0 (TID 13) in 746 ms on 10.0.0.133 (executor driver) (1/16) -26/04/01 06:05:25 INFO Executor: Running task 4.0 in stage 11.0 (TID 14) -26/04/01 06:05:25 INFO Executor: Finished task 2.0 in stage 11.0 (TID 12). 6649 bytes result sent to driver -26/04/01 06:05:25 INFO TaskSetManager: Starting task 5.0 in stage 11.0 (TID 15) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:05:25 INFO Executor: Running task 5.0 in stage 11.0 (TID 15) -26/04/01 06:05:25 INFO TaskSetManager: Starting task 6.0 in stage 11.0 (TID 16) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:05:25 INFO Executor: Running task 6.0 in stage 11.0 (TID 16) -26/04/01 06:05:25 INFO Executor: Finished task 0.0 in stage 11.0 (TID 10). 6649 bytes result sent to driver -26/04/01 06:05:25 INFO TaskSetManager: Starting task 7.0 in stage 11.0 (TID 17) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9334 bytes) -26/04/01 06:05:25 INFO TaskSetManager: Finished task 1.0 in stage 11.0 (TID 11) in 747 ms on 10.0.0.133 (executor driver) (2/16) -26/04/01 06:05:25 INFO Executor: Running task 7.0 in stage 11.0 (TID 17) -26/04/01 06:05:25 INFO TaskSetManager: Finished task 2.0 in stage 11.0 (TID 12) in 748 ms on 10.0.0.133 (executor driver) (3/16) -26/04/01 06:05:25 INFO TaskSetManager: Finished task 0.0 in stage 11.0 (TID 10) in 748 ms on 10.0.0.133 (executor driver) (4/16) -26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO Executor: Finished task 6.0 in stage 11.0 (TID 16). 6606 bytes result sent to driver -26/04/01 06:05:26 INFO TaskSetManager: Starting task 8.0 in stage 11.0 (TID 18) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:05:26 INFO Executor: Running task 8.0 in stage 11.0 (TID 18) -26/04/01 06:05:26 INFO TaskSetManager: Finished task 6.0 in stage 11.0 (TID 16) in 717 ms on 10.0.0.133 (executor driver) (5/16) -26/04/01 06:05:26 INFO Executor: Finished task 4.0 in stage 11.0 (TID 14). 6606 bytes result sent to driver -26/04/01 06:05:26 INFO TaskSetManager: Starting task 9.0 in stage 11.0 (TID 19) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:05:26 INFO Executor: Finished task 5.0 in stage 11.0 (TID 15). 6606 bytes result sent to driver -26/04/01 06:05:26 INFO Executor: Running task 9.0 in stage 11.0 (TID 19) -26/04/01 06:05:26 INFO TaskSetManager: Starting task 10.0 in stage 11.0 (TID 20) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9334 bytes) -26/04/01 06:05:26 INFO Executor: Running task 10.0 in stage 11.0 (TID 20) -26/04/01 06:05:26 INFO Executor: Finished task 7.0 in stage 11.0 (TID 17). 6606 bytes result sent to driver -26/04/01 06:05:26 INFO TaskSetManager: Finished task 5.0 in stage 11.0 (TID 15) in 718 ms on 10.0.0.133 (executor driver) (6/16) -26/04/01 06:05:26 INFO TaskSetManager: Finished task 4.0 in stage 11.0 (TID 14) in 719 ms on 10.0.0.133 (executor driver) (7/16) -26/04/01 06:05:26 INFO TaskSetManager: Starting task 11.0 in stage 11.0 (TID 21) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO Executor: Running task 11.0 in stage 11.0 (TID 21) -26/04/01 06:05:26 INFO TaskSetManager: Finished task 7.0 in stage 11.0 (TID 17) in 718 ms on 10.0.0.133 (executor driver) (8/16) -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO Executor: Finished task 11.0 in stage 11.0 (TID 21). 6563 bytes result sent to driver -26/04/01 06:05:26 INFO Executor: Finished task 9.0 in stage 11.0 (TID 19). 6606 bytes result sent to driver -26/04/01 06:05:26 INFO Executor: Finished task 8.0 in stage 11.0 (TID 18). 6606 bytes result sent to driver -26/04/01 06:05:26 INFO TaskSetManager: Starting task 12.0 in stage 11.0 (TID 22) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9334 bytes) -26/04/01 06:05:26 INFO Executor: Finished task 10.0 in stage 11.0 (TID 20). 6606 bytes result sent to driver -26/04/01 06:05:26 INFO Executor: Running task 12.0 in stage 11.0 (TID 22) -26/04/01 06:05:26 INFO TaskSetManager: Starting task 13.0 in stage 11.0 (TID 23) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:05:26 INFO Executor: Running task 13.0 in stage 11.0 (TID 23) -26/04/01 06:05:26 INFO TaskSetManager: Starting task 14.0 in stage 11.0 (TID 24) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9334 bytes) -26/04/01 06:05:26 INFO TaskSetManager: Finished task 11.0 in stage 11.0 (TID 21) in 717 ms on 10.0.0.133 (executor driver) (9/16) -26/04/01 06:05:26 INFO Executor: Running task 14.0 in stage 11.0 (TID 24) -26/04/01 06:05:26 INFO TaskSetManager: Finished task 8.0 in stage 11.0 (TID 18) in 719 ms on 10.0.0.133 (executor driver) (10/16) -26/04/01 06:05:26 INFO TaskSetManager: Finished task 9.0 in stage 11.0 (TID 19) in 718 ms on 10.0.0.133 (executor driver) (11/16) -26/04/01 06:05:26 INFO TaskSetManager: Starting task 15.0 in stage 11.0 (TID 25) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9334 bytes) -26/04/01 06:05:26 INFO TaskSetManager: Finished task 10.0 in stage 11.0 (TID 20) in 718 ms on 10.0.0.133 (executor driver) (12/16) -26/04/01 06:05:26 INFO Executor: Running task 15.0 in stage 11.0 (TID 25) -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:27 INFO Executor: Finished task 15.0 in stage 11.0 (TID 25). 6563 bytes result sent to driver -26/04/01 06:05:27 INFO Executor: Finished task 13.0 in stage 11.0 (TID 23). 6606 bytes result sent to driver -26/04/01 06:05:27 INFO Executor: Finished task 14.0 in stage 11.0 (TID 24). 6606 bytes result sent to driver -26/04/01 06:05:27 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 26) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9271 bytes) -26/04/01 06:05:27 INFO Executor: Running task 0.0 in stage 12.0 (TID 26) -26/04/01 06:05:27 INFO TaskSetManager: Starting task 0.0 in stage 13.0 (TID 27) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:27 INFO TaskSetManager: Finished task 15.0 in stage 11.0 (TID 25) in 712 ms on 10.0.0.133 (executor driver) (13/16) -26/04/01 06:05:27 INFO Executor: Running task 0.0 in stage 13.0 (TID 27) -26/04/01 06:05:27 INFO TaskSetManager: Finished task 13.0 in stage 11.0 (TID 23) in 713 ms on 10.0.0.133 (executor driver) (14/16) -26/04/01 06:05:27 INFO TaskSetManager: Starting task 1.0 in stage 13.0 (TID 28) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:05:27 INFO Executor: Running task 1.0 in stage 13.0 (TID 28) -26/04/01 06:05:27 INFO TaskSetManager: Finished task 14.0 in stage 11.0 (TID 24) in 713 ms on 10.0.0.133 (executor driver) (15/16) -26/04/01 06:05:27 INFO Executor: Finished task 12.0 in stage 11.0 (TID 22). 6606 bytes result sent to driver -26/04/01 06:05:27 INFO TaskSetManager: Starting task 2.0 in stage 13.0 (TID 29) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:27 INFO Executor: Running task 2.0 in stage 13.0 (TID 29) -26/04/01 06:05:27 INFO TaskSetManager: Finished task 12.0 in stage 11.0 (TID 22) in 715 ms on 10.0.0.133 (executor driver) (16/16) -26/04/01 06:05:27 INFO TaskSchedulerImpl: Removed TaskSet 11.0, whose tasks have all completed, from pool -26/04/01 06:05:27 INFO DAGScheduler: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.897 s -26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:27 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:05:27 INFO DAGScheduler: running: Set(ResultStage 15, ResultStage 12, ShuffleMapStage 16, ShuffleMapStage 13, ShuffleMapStage 14) -26/04/01 06:05:27 INFO DAGScheduler: waiting: Set() -26/04/01 06:05:27 INFO DAGScheduler: failed: Set() -26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:27 INFO Executor: Finished task 0.0 in stage 12.0 (TID 26). 4405 bytes result sent to driver -26/04/01 06:05:27 INFO TaskSetManager: Starting task 3.0 in stage 13.0 (TID 30) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:27 INFO Executor: Running task 3.0 in stage 13.0 (TID 30) -26/04/01 06:05:27 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 26) in 32 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:05:27 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool -26/04/01 06:05:27 INFO DAGScheduler: ResultStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.924 s -26/04/01 06:05:27 INFO DAGScheduler: Job 11 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:05:27 INFO TaskSchedulerImpl: Killing all running tasks in stage 12: Stage finished -26/04/01 06:05:27 INFO DAGScheduler: Job 11 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 2.925689 s -26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:27 INFO Utils: Coalesced 1 broadcast batches into 1 (1 rows) -26/04/01 06:05:27 INFO MemoryStore: Block broadcast_23 stored as values in memory (estimated size 416.0 B, free 8.6 GiB) -26/04/01 06:05:27 INFO MemoryStore: Block broadcast_23_piece0 stored as bytes in memory (estimated size 367.0 B, free 8.6 GiB) -26/04/01 06:05:27 INFO BlockManagerInfo: Added broadcast_23_piece0 in memory on 10.0.0.133:58259 (size: 367.0 B, free: 8.6 GiB) -26/04/01 06:05:27 INFO SparkContext: Created broadcast 23 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:05:28 INFO Executor: Finished task 1.0 in stage 13.0 (TID 28). 6675 bytes result sent to driver -26/04/01 06:05:28 INFO TaskSetManager: Starting task 4.0 in stage 13.0 (TID 31) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:28 INFO TaskSetManager: Finished task 1.0 in stage 13.0 (TID 28) in 1289 ms on 10.0.0.133 (executor driver) (1/64) -26/04/01 06:05:28 INFO Executor: Running task 4.0 in stage 13.0 (TID 31) -26/04/01 06:05:28 INFO Executor: Finished task 0.0 in stage 13.0 (TID 27). 6675 bytes result sent to driver -26/04/01 06:05:28 INFO TaskSetManager: Starting task 5.0 in stage 13.0 (TID 32) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:05:28 INFO Executor: Finished task 2.0 in stage 13.0 (TID 29). 6675 bytes result sent to driver -26/04/01 06:05:28 INFO Executor: Running task 5.0 in stage 13.0 (TID 32) -26/04/01 06:05:28 INFO TaskSetManager: Starting task 6.0 in stage 13.0 (TID 33) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:28 INFO TaskSetManager: Finished task 0.0 in stage 13.0 (TID 27) in 1291 ms on 10.0.0.133 (executor driver) (2/64) -26/04/01 06:05:28 INFO Executor: Running task 6.0 in stage 13.0 (TID 33) -26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:28 INFO TaskSetManager: Finished task 2.0 in stage 13.0 (TID 29) in 1290 ms on 10.0.0.133 (executor driver) (3/64) -26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:28 INFO Executor: Finished task 3.0 in stage 13.0 (TID 30). 6675 bytes result sent to driver -26/04/01 06:05:28 INFO TaskSetManager: Starting task 7.0 in stage 13.0 (TID 34) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:28 INFO TaskSetManager: Finished task 3.0 in stage 13.0 (TID 30) in 1292 ms on 10.0.0.133 (executor driver) (4/64) -26/04/01 06:05:28 INFO Executor: Running task 7.0 in stage 13.0 (TID 34) -26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:30 INFO Executor: Finished task 5.0 in stage 13.0 (TID 32). 6675 bytes result sent to driver -26/04/01 06:05:30 INFO Executor: Finished task 4.0 in stage 13.0 (TID 31). 6675 bytes result sent to driver -26/04/01 06:05:30 INFO TaskSetManager: Starting task 8.0 in stage 13.0 (TID 35) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:05:30 INFO Executor: Running task 8.0 in stage 13.0 (TID 35) -26/04/01 06:05:30 INFO TaskSetManager: Starting task 9.0 in stage 13.0 (TID 36) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:30 INFO TaskSetManager: Finished task 5.0 in stage 13.0 (TID 32) in 1288 ms on 10.0.0.133 (executor driver) (5/64) -26/04/01 06:05:30 INFO TaskSetManager: Finished task 4.0 in stage 13.0 (TID 31) in 1289 ms on 10.0.0.133 (executor driver) (6/64) -26/04/01 06:05:30 INFO Executor: Running task 9.0 in stage 13.0 (TID 36) -26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:30 INFO Executor: Finished task 6.0 in stage 13.0 (TID 33). 6675 bytes result sent to driver -26/04/01 06:05:30 INFO TaskSetManager: Starting task 10.0 in stage 13.0 (TID 37) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:05:30 INFO TaskSetManager: Finished task 6.0 in stage 13.0 (TID 33) in 1289 ms on 10.0.0.133 (executor driver) (7/64) -26/04/01 06:05:30 INFO Executor: Running task 10.0 in stage 13.0 (TID 37) -26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:30 INFO Executor: Finished task 7.0 in stage 13.0 (TID 34). 6675 bytes result sent to driver -26/04/01 06:05:30 INFO TaskSetManager: Starting task 11.0 in stage 13.0 (TID 38) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:30 INFO TaskSetManager: Finished task 7.0 in stage 13.0 (TID 34) in 1291 ms on 10.0.0.133 (executor driver) (8/64) -26/04/01 06:05:30 INFO Executor: Running task 11.0 in stage 13.0 (TID 38) -26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:31 INFO Executor: Finished task 8.0 in stage 13.0 (TID 35). 6675 bytes result sent to driver -26/04/01 06:05:31 INFO Executor: Finished task 10.0 in stage 13.0 (TID 37). 6675 bytes result sent to driver -26/04/01 06:05:31 INFO TaskSetManager: Starting task 12.0 in stage 13.0 (TID 39) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:31 INFO TaskSetManager: Starting task 13.0 in stage 13.0 (TID 40) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:31 INFO Executor: Running task 12.0 in stage 13.0 (TID 39) -26/04/01 06:05:31 INFO TaskSetManager: Finished task 8.0 in stage 13.0 (TID 35) in 1293 ms on 10.0.0.133 (executor driver) (9/64) -26/04/01 06:05:31 INFO Executor: Running task 13.0 in stage 13.0 (TID 40) -26/04/01 06:05:31 INFO TaskSetManager: Finished task 10.0 in stage 13.0 (TID 37) in 1290 ms on 10.0.0.133 (executor driver) (10/64) -26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:31 INFO Executor: Finished task 9.0 in stage 13.0 (TID 36). 6675 bytes result sent to driver -26/04/01 06:05:31 INFO TaskSetManager: Starting task 14.0 in stage 13.0 (TID 41) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:31 INFO TaskSetManager: Finished task 9.0 in stage 13.0 (TID 36) in 1294 ms on 10.0.0.133 (executor driver) (11/64) -26/04/01 06:05:31 INFO Executor: Running task 14.0 in stage 13.0 (TID 41) -26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:31 INFO Executor: Finished task 11.0 in stage 13.0 (TID 38). 6675 bytes result sent to driver -26/04/01 06:05:31 INFO TaskSetManager: Starting task 15.0 in stage 13.0 (TID 42) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:31 INFO Executor: Running task 15.0 in stage 13.0 (TID 42) -26/04/01 06:05:31 INFO TaskSetManager: Finished task 11.0 in stage 13.0 (TID 38) in 1293 ms on 10.0.0.133 (executor driver) (12/64) -26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:32 INFO Executor: Finished task 14.0 in stage 13.0 (TID 41). 6718 bytes result sent to driver -26/04/01 06:05:32 INFO Executor: Finished task 12.0 in stage 13.0 (TID 39). 6718 bytes result sent to driver -26/04/01 06:05:32 INFO TaskSetManager: Starting task 16.0 in stage 13.0 (TID 43) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:05:32 INFO Executor: Finished task 13.0 in stage 13.0 (TID 40). 6718 bytes result sent to driver -26/04/01 06:05:32 INFO TaskSetManager: Finished task 14.0 in stage 13.0 (TID 41) in 1292 ms on 10.0.0.133 (executor driver) (13/64) -26/04/01 06:05:32 INFO Executor: Running task 16.0 in stage 13.0 (TID 43) -26/04/01 06:05:32 INFO TaskSetManager: Starting task 17.0 in stage 13.0 (TID 44) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:32 INFO Executor: Running task 17.0 in stage 13.0 (TID 44) -26/04/01 06:05:32 INFO TaskSetManager: Starting task 18.0 in stage 13.0 (TID 45) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:05:32 INFO TaskSetManager: Finished task 12.0 in stage 13.0 (TID 39) in 1294 ms on 10.0.0.133 (executor driver) (14/64) -26/04/01 06:05:32 INFO Executor: Running task 18.0 in stage 13.0 (TID 45) -26/04/01 06:05:32 INFO TaskSetManager: Finished task 13.0 in stage 13.0 (TID 40) in 1294 ms on 10.0.0.133 (executor driver) (15/64) -26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:32 INFO Executor: Finished task 15.0 in stage 13.0 (TID 42). 6718 bytes result sent to driver -26/04/01 06:05:32 INFO TaskSetManager: Starting task 19.0 in stage 13.0 (TID 46) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:32 INFO TaskSetManager: Finished task 15.0 in stage 13.0 (TID 42) in 1289 ms on 10.0.0.133 (executor driver) (16/64) -26/04/01 06:05:32 INFO Executor: Running task 19.0 in stage 13.0 (TID 46) -26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:34 INFO Executor: Finished task 18.0 in stage 13.0 (TID 45). 6675 bytes result sent to driver -26/04/01 06:05:34 INFO Executor: Finished task 16.0 in stage 13.0 (TID 43). 6675 bytes result sent to driver -26/04/01 06:05:34 INFO TaskSetManager: Starting task 20.0 in stage 13.0 (TID 47) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:34 INFO Executor: Running task 20.0 in stage 13.0 (TID 47) -26/04/01 06:05:34 INFO TaskSetManager: Starting task 21.0 in stage 13.0 (TID 48) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:05:34 INFO TaskSetManager: Finished task 18.0 in stage 13.0 (TID 45) in 1289 ms on 10.0.0.133 (executor driver) (17/64) -26/04/01 06:05:34 INFO Executor: Running task 21.0 in stage 13.0 (TID 48) -26/04/01 06:05:34 INFO TaskSetManager: Finished task 16.0 in stage 13.0 (TID 43) in 1291 ms on 10.0.0.133 (executor driver) (18/64) -26/04/01 06:05:34 INFO Executor: Finished task 17.0 in stage 13.0 (TID 44). 6675 bytes result sent to driver -26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:34 INFO TaskSetManager: Starting task 22.0 in stage 13.0 (TID 49) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:34 INFO TaskSetManager: Finished task 17.0 in stage 13.0 (TID 44) in 1291 ms on 10.0.0.133 (executor driver) (19/64) -26/04/01 06:05:34 INFO Executor: Running task 22.0 in stage 13.0 (TID 49) -26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:34 INFO Executor: Finished task 19.0 in stage 13.0 (TID 46). 6675 bytes result sent to driver -26/04/01 06:05:34 INFO TaskSetManager: Starting task 23.0 in stage 13.0 (TID 50) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:34 INFO TaskSetManager: Finished task 19.0 in stage 13.0 (TID 46) in 1288 ms on 10.0.0.133 (executor driver) (20/64) -26/04/01 06:05:34 INFO Executor: Running task 23.0 in stage 13.0 (TID 50) -26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:35 INFO Executor: Finished task 21.0 in stage 13.0 (TID 48). 6675 bytes result sent to driver -26/04/01 06:05:35 INFO TaskSetManager: Starting task 24.0 in stage 13.0 (TID 51) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:05:35 INFO TaskSetManager: Finished task 21.0 in stage 13.0 (TID 48) in 1287 ms on 10.0.0.133 (executor driver) (21/64) -26/04/01 06:05:35 INFO Executor: Running task 24.0 in stage 13.0 (TID 51) -26/04/01 06:05:35 INFO Executor: Finished task 20.0 in stage 13.0 (TID 47). 6675 bytes result sent to driver -26/04/01 06:05:35 INFO TaskSetManager: Starting task 25.0 in stage 13.0 (TID 52) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:35 INFO TaskSetManager: Finished task 20.0 in stage 13.0 (TID 47) in 1287 ms on 10.0.0.133 (executor driver) (22/64) -26/04/01 06:05:35 INFO Executor: Running task 25.0 in stage 13.0 (TID 52) -26/04/01 06:05:35 INFO Executor: Finished task 22.0 in stage 13.0 (TID 49). 6675 bytes result sent to driver -26/04/01 06:05:35 INFO TaskSetManager: Starting task 26.0 in stage 13.0 (TID 53) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:35 INFO TaskSetManager: Finished task 22.0 in stage 13.0 (TID 49) in 1286 ms on 10.0.0.133 (executor driver) (23/64) -26/04/01 06:05:35 INFO Executor: Running task 26.0 in stage 13.0 (TID 53) -26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:35 INFO Executor: Finished task 23.0 in stage 13.0 (TID 50). 6675 bytes result sent to driver -26/04/01 06:05:35 INFO TaskSetManager: Starting task 27.0 in stage 13.0 (TID 54) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:35 INFO TaskSetManager: Finished task 23.0 in stage 13.0 (TID 50) in 1289 ms on 10.0.0.133 (executor driver) (24/64) -26/04/01 06:05:35 INFO Executor: Running task 27.0 in stage 13.0 (TID 54) -26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:36 INFO Executor: Finished task 26.0 in stage 13.0 (TID 53). 6675 bytes result sent to driver -26/04/01 06:05:36 INFO TaskSetManager: Starting task 28.0 in stage 13.0 (TID 55) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:36 INFO Executor: Running task 28.0 in stage 13.0 (TID 55) -26/04/01 06:05:36 INFO TaskSetManager: Finished task 26.0 in stage 13.0 (TID 53) in 1287 ms on 10.0.0.133 (executor driver) (25/64) -26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:36 INFO Executor: Finished task 24.0 in stage 13.0 (TID 51). 6675 bytes result sent to driver -26/04/01 06:05:36 INFO TaskSetManager: Starting task 29.0 in stage 13.0 (TID 56) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:36 INFO Executor: Running task 29.0 in stage 13.0 (TID 56) -26/04/01 06:05:36 INFO TaskSetManager: Finished task 24.0 in stage 13.0 (TID 51) in 1290 ms on 10.0.0.133 (executor driver) (26/64) -26/04/01 06:05:36 INFO Executor: Finished task 25.0 in stage 13.0 (TID 52). 6675 bytes result sent to driver -26/04/01 06:05:36 INFO TaskSetManager: Starting task 30.0 in stage 13.0 (TID 57) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:36 INFO Executor: Running task 30.0 in stage 13.0 (TID 57) -26/04/01 06:05:36 INFO TaskSetManager: Finished task 25.0 in stage 13.0 (TID 52) in 1291 ms on 10.0.0.133 (executor driver) (27/64) -26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:36 INFO Executor: Finished task 27.0 in stage 13.0 (TID 54). 6675 bytes result sent to driver -26/04/01 06:05:36 INFO TaskSetManager: Starting task 31.0 in stage 13.0 (TID 58) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:05:36 INFO TaskSetManager: Finished task 27.0 in stage 13.0 (TID 54) in 1293 ms on 10.0.0.133 (executor driver) (28/64) -26/04/01 06:05:36 INFO Executor: Running task 31.0 in stage 13.0 (TID 58) -26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:37 INFO Executor: Finished task 30.0 in stage 13.0 (TID 57). 6675 bytes result sent to driver -26/04/01 06:05:37 INFO Executor: Finished task 28.0 in stage 13.0 (TID 55). 6675 bytes result sent to driver -26/04/01 06:05:37 INFO Executor: Finished task 29.0 in stage 13.0 (TID 56). 6675 bytes result sent to driver -26/04/01 06:05:37 INFO TaskSetManager: Starting task 32.0 in stage 13.0 (TID 59) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:37 INFO Executor: Running task 32.0 in stage 13.0 (TID 59) -26/04/01 06:05:37 INFO TaskSetManager: Starting task 33.0 in stage 13.0 (TID 60) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:37 INFO TaskSetManager: Finished task 30.0 in stage 13.0 (TID 57) in 1289 ms on 10.0.0.133 (executor driver) (29/64) -26/04/01 06:05:37 INFO Executor: Running task 33.0 in stage 13.0 (TID 60) -26/04/01 06:05:37 INFO TaskSetManager: Starting task 34.0 in stage 13.0 (TID 61) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:37 INFO TaskSetManager: Finished task 28.0 in stage 13.0 (TID 55) in 1292 ms on 10.0.0.133 (executor driver) (30/64) -26/04/01 06:05:37 INFO Executor: Running task 34.0 in stage 13.0 (TID 61) -26/04/01 06:05:37 INFO TaskSetManager: Finished task 29.0 in stage 13.0 (TID 56) in 1290 ms on 10.0.0.133 (executor driver) (31/64) -26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:37 INFO Executor: Finished task 31.0 in stage 13.0 (TID 58). 6675 bytes result sent to driver -26/04/01 06:05:37 INFO TaskSetManager: Starting task 35.0 in stage 13.0 (TID 62) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:05:37 INFO Executor: Running task 35.0 in stage 13.0 (TID 62) -26/04/01 06:05:37 INFO TaskSetManager: Finished task 31.0 in stage 13.0 (TID 58) in 1290 ms on 10.0.0.133 (executor driver) (32/64) -26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:38 INFO Executor: Finished task 33.0 in stage 13.0 (TID 60). 6675 bytes result sent to driver -26/04/01 06:05:38 INFO Executor: Finished task 32.0 in stage 13.0 (TID 59). 6675 bytes result sent to driver -26/04/01 06:05:38 INFO TaskSetManager: Starting task 36.0 in stage 13.0 (TID 63) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:38 INFO Executor: Running task 36.0 in stage 13.0 (TID 63) -26/04/01 06:05:38 INFO TaskSetManager: Starting task 37.0 in stage 13.0 (TID 64) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:38 INFO TaskSetManager: Finished task 33.0 in stage 13.0 (TID 60) in 920 ms on 10.0.0.133 (executor driver) (33/64) -26/04/01 06:05:38 INFO TaskSetManager: Finished task 32.0 in stage 13.0 (TID 59) in 920 ms on 10.0.0.133 (executor driver) (34/64) -26/04/01 06:05:38 INFO Executor: Running task 37.0 in stage 13.0 (TID 64) -26/04/01 06:05:38 INFO Executor: Finished task 34.0 in stage 13.0 (TID 61). 6675 bytes result sent to driver -26/04/01 06:05:38 INFO TaskSetManager: Starting task 38.0 in stage 13.0 (TID 65) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:38 INFO TaskSetManager: Finished task 34.0 in stage 13.0 (TID 61) in 919 ms on 10.0.0.133 (executor driver) (35/64) -26/04/01 06:05:38 INFO Executor: Running task 38.0 in stage 13.0 (TID 65) -26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:38 INFO Executor: Finished task 35.0 in stage 13.0 (TID 62). 6675 bytes result sent to driver -26/04/01 06:05:38 INFO TaskSetManager: Starting task 39.0 in stage 13.0 (TID 66) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:38 INFO TaskSetManager: Finished task 35.0 in stage 13.0 (TID 62) in 921 ms on 10.0.0.133 (executor driver) (36/64) -26/04/01 06:05:38 INFO Executor: Running task 39.0 in stage 13.0 (TID 66) -26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:39 INFO Executor: Finished task 37.0 in stage 13.0 (TID 64). 6675 bytes result sent to driver -26/04/01 06:05:39 INFO Executor: Finished task 38.0 in stage 13.0 (TID 65). 6675 bytes result sent to driver -26/04/01 06:05:39 INFO TaskSetManager: Starting task 40.0 in stage 13.0 (TID 67) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:05:39 INFO Executor: Running task 40.0 in stage 13.0 (TID 67) -26/04/01 06:05:39 INFO TaskSetManager: Starting task 41.0 in stage 13.0 (TID 68) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:39 INFO TaskSetManager: Finished task 37.0 in stage 13.0 (TID 64) in 925 ms on 10.0.0.133 (executor driver) (37/64) -26/04/01 06:05:39 INFO Executor: Running task 41.0 in stage 13.0 (TID 68) -26/04/01 06:05:39 INFO TaskSetManager: Finished task 38.0 in stage 13.0 (TID 65) in 924 ms on 10.0.0.133 (executor driver) (38/64) -26/04/01 06:05:39 INFO Executor: Finished task 36.0 in stage 13.0 (TID 63). 6675 bytes result sent to driver -26/04/01 06:05:39 INFO TaskSetManager: Starting task 42.0 in stage 13.0 (TID 69) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:39 INFO TaskSetManager: Finished task 36.0 in stage 13.0 (TID 63) in 926 ms on 10.0.0.133 (executor driver) (39/64) -26/04/01 06:05:39 INFO Executor: Running task 42.0 in stage 13.0 (TID 69) -26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:39 INFO Executor: Finished task 39.0 in stage 13.0 (TID 66). 6675 bytes result sent to driver -26/04/01 06:05:39 INFO TaskSetManager: Starting task 43.0 in stage 13.0 (TID 70) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:05:39 INFO Executor: Running task 43.0 in stage 13.0 (TID 70) -26/04/01 06:05:39 INFO TaskSetManager: Finished task 39.0 in stage 13.0 (TID 66) in 926 ms on 10.0.0.133 (executor driver) (40/64) -26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:40 INFO Executor: Finished task 41.0 in stage 13.0 (TID 68). 6675 bytes result sent to driver -26/04/01 06:05:40 INFO TaskSetManager: Starting task 44.0 in stage 13.0 (TID 71) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:05:40 INFO TaskSetManager: Finished task 41.0 in stage 13.0 (TID 68) in 921 ms on 10.0.0.133 (executor driver) (41/64) -26/04/01 06:05:40 INFO Executor: Running task 44.0 in stage 13.0 (TID 71) -26/04/01 06:05:40 INFO Executor: Finished task 40.0 in stage 13.0 (TID 67). 6675 bytes result sent to driver -26/04/01 06:05:40 INFO TaskSetManager: Starting task 45.0 in stage 13.0 (TID 72) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:40 INFO TaskSetManager: Finished task 40.0 in stage 13.0 (TID 67) in 922 ms on 10.0.0.133 (executor driver) (42/64) -26/04/01 06:05:40 INFO Executor: Running task 45.0 in stage 13.0 (TID 72) -26/04/01 06:05:40 INFO Executor: Finished task 42.0 in stage 13.0 (TID 69). 6675 bytes result sent to driver -26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:40 INFO TaskSetManager: Starting task 46.0 in stage 13.0 (TID 73) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:40 INFO Executor: Running task 46.0 in stage 13.0 (TID 73) -26/04/01 06:05:40 INFO TaskSetManager: Finished task 42.0 in stage 13.0 (TID 69) in 922 ms on 10.0.0.133 (executor driver) (43/64) -26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:40 INFO Executor: Finished task 43.0 in stage 13.0 (TID 70). 6675 bytes result sent to driver -26/04/01 06:05:40 INFO TaskSetManager: Starting task 47.0 in stage 13.0 (TID 74) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:05:40 INFO TaskSetManager: Finished task 43.0 in stage 13.0 (TID 70) in 923 ms on 10.0.0.133 (executor driver) (44/64) -26/04/01 06:05:40 INFO Executor: Running task 47.0 in stage 13.0 (TID 74) -26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:41 INFO Executor: Finished task 46.0 in stage 13.0 (TID 73). 6675 bytes result sent to driver -26/04/01 06:05:41 INFO Executor: Finished task 45.0 in stage 13.0 (TID 72). 6675 bytes result sent to driver -26/04/01 06:05:41 INFO Executor: Finished task 44.0 in stage 13.0 (TID 71). 6675 bytes result sent to driver -26/04/01 06:05:41 INFO TaskSetManager: Starting task 48.0 in stage 13.0 (TID 75) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:41 INFO Executor: Running task 48.0 in stage 13.0 (TID 75) -26/04/01 06:05:41 INFO TaskSetManager: Starting task 49.0 in stage 13.0 (TID 76) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:41 INFO TaskSetManager: Finished task 46.0 in stage 13.0 (TID 73) in 917 ms on 10.0.0.133 (executor driver) (45/64) -26/04/01 06:05:41 INFO Executor: Running task 49.0 in stage 13.0 (TID 76) -26/04/01 06:05:41 INFO TaskSetManager: Finished task 45.0 in stage 13.0 (TID 72) in 918 ms on 10.0.0.133 (executor driver) (46/64) -26/04/01 06:05:41 INFO TaskSetManager: Starting task 50.0 in stage 13.0 (TID 77) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:41 INFO TaskSetManager: Finished task 44.0 in stage 13.0 (TID 71) in 919 ms on 10.0.0.133 (executor driver) (47/64) -26/04/01 06:05:41 INFO Executor: Running task 50.0 in stage 13.0 (TID 77) -26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:41 INFO Executor: Finished task 47.0 in stage 13.0 (TID 74). 6675 bytes result sent to driver -26/04/01 06:05:41 INFO TaskSetManager: Starting task 51.0 in stage 13.0 (TID 78) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:41 INFO Executor: Running task 51.0 in stage 13.0 (TID 78) -26/04/01 06:05:41 INFO TaskSetManager: Finished task 47.0 in stage 13.0 (TID 74) in 920 ms on 10.0.0.133 (executor driver) (48/64) -26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:42 INFO Executor: Finished task 49.0 in stage 13.0 (TID 76). 6675 bytes result sent to driver -26/04/01 06:05:42 INFO Executor: Finished task 50.0 in stage 13.0 (TID 77). 6675 bytes result sent to driver -26/04/01 06:05:42 INFO TaskSetManager: Starting task 52.0 in stage 13.0 (TID 79) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:42 INFO Executor: Finished task 48.0 in stage 13.0 (TID 75). 6675 bytes result sent to driver -26/04/01 06:05:42 INFO Executor: Running task 52.0 in stage 13.0 (TID 79) -26/04/01 06:05:42 INFO TaskSetManager: Starting task 53.0 in stage 13.0 (TID 80) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:42 INFO TaskSetManager: Finished task 49.0 in stage 13.0 (TID 76) in 919 ms on 10.0.0.133 (executor driver) (49/64) -26/04/01 06:05:42 INFO Executor: Running task 53.0 in stage 13.0 (TID 80) -26/04/01 06:05:42 INFO TaskSetManager: Finished task 50.0 in stage 13.0 (TID 77) in 919 ms on 10.0.0.133 (executor driver) (50/64) -26/04/01 06:05:42 INFO TaskSetManager: Starting task 54.0 in stage 13.0 (TID 81) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:05:42 INFO TaskSetManager: Finished task 48.0 in stage 13.0 (TID 75) in 920 ms on 10.0.0.133 (executor driver) (51/64) -26/04/01 06:05:42 INFO Executor: Running task 54.0 in stage 13.0 (TID 81) -26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:42 INFO Executor: Finished task 51.0 in stage 13.0 (TID 78). 6675 bytes result sent to driver -26/04/01 06:05:42 INFO TaskSetManager: Starting task 55.0 in stage 13.0 (TID 82) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:42 INFO TaskSetManager: Finished task 51.0 in stage 13.0 (TID 78) in 917 ms on 10.0.0.133 (executor driver) (52/64) -26/04/01 06:05:42 INFO Executor: Running task 55.0 in stage 13.0 (TID 82) -26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:43 INFO Executor: Finished task 53.0 in stage 13.0 (TID 80). 6675 bytes result sent to driver -26/04/01 06:05:43 INFO TaskSetManager: Starting task 56.0 in stage 13.0 (TID 83) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:05:43 INFO Executor: Finished task 52.0 in stage 13.0 (TID 79). 6675 bytes result sent to driver -26/04/01 06:05:43 INFO TaskSetManager: Finished task 53.0 in stage 13.0 (TID 80) in 918 ms on 10.0.0.133 (executor driver) (53/64) -26/04/01 06:05:43 INFO Executor: Running task 56.0 in stage 13.0 (TID 83) -26/04/01 06:05:43 INFO TaskSetManager: Starting task 57.0 in stage 13.0 (TID 84) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:43 INFO TaskSetManager: Finished task 52.0 in stage 13.0 (TID 79) in 919 ms on 10.0.0.133 (executor driver) (54/64) -26/04/01 06:05:43 INFO Executor: Running task 57.0 in stage 13.0 (TID 84) -26/04/01 06:05:43 INFO Executor: Finished task 54.0 in stage 13.0 (TID 81). 6675 bytes result sent to driver -26/04/01 06:05:43 INFO TaskSetManager: Starting task 58.0 in stage 13.0 (TID 85) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:43 INFO TaskSetManager: Finished task 54.0 in stage 13.0 (TID 81) in 919 ms on 10.0.0.133 (executor driver) (55/64) -26/04/01 06:05:43 INFO Executor: Running task 58.0 in stage 13.0 (TID 85) -26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:43 INFO Executor: Finished task 55.0 in stage 13.0 (TID 82). 6675 bytes result sent to driver -26/04/01 06:05:43 INFO TaskSetManager: Starting task 59.0 in stage 13.0 (TID 86) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:43 INFO TaskSetManager: Finished task 55.0 in stage 13.0 (TID 82) in 918 ms on 10.0.0.133 (executor driver) (56/64) -26/04/01 06:05:43 INFO Executor: Running task 59.0 in stage 13.0 (TID 86) -26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:44 INFO Executor: Finished task 56.0 in stage 13.0 (TID 83). 6675 bytes result sent to driver -26/04/01 06:05:44 INFO Executor: Finished task 58.0 in stage 13.0 (TID 85). 6675 bytes result sent to driver -26/04/01 06:05:44 INFO TaskSetManager: Starting task 60.0 in stage 13.0 (TID 87) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:44 INFO Executor: Running task 60.0 in stage 13.0 (TID 87) -26/04/01 06:05:44 INFO TaskSetManager: Starting task 61.0 in stage 13.0 (TID 88) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:44 INFO TaskSetManager: Finished task 56.0 in stage 13.0 (TID 83) in 920 ms on 10.0.0.133 (executor driver) (57/64) -26/04/01 06:05:44 INFO Executor: Running task 61.0 in stage 13.0 (TID 88) -26/04/01 06:05:44 INFO TaskSetManager: Finished task 58.0 in stage 13.0 (TID 85) in 919 ms on 10.0.0.133 (executor driver) (58/64) -26/04/01 06:05:44 INFO Executor: Finished task 57.0 in stage 13.0 (TID 84). 6675 bytes result sent to driver -26/04/01 06:05:44 INFO TaskSetManager: Starting task 62.0 in stage 13.0 (TID 89) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:05:44 INFO TaskSetManager: Finished task 57.0 in stage 13.0 (TID 84) in 920 ms on 10.0.0.133 (executor driver) (59/64) -26/04/01 06:05:44 INFO Executor: Running task 62.0 in stage 13.0 (TID 89) -26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:44 INFO Executor: Finished task 59.0 in stage 13.0 (TID 86). 6675 bytes result sent to driver -26/04/01 06:05:44 INFO TaskSetManager: Starting task 63.0 in stage 13.0 (TID 90) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:05:44 INFO Executor: Running task 63.0 in stage 13.0 (TID 90) -26/04/01 06:05:44 INFO TaskSetManager: Finished task 59.0 in stage 13.0 (TID 86) in 919 ms on 10.0.0.133 (executor driver) (60/64) -26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:45 INFO Executor: Finished task 60.0 in stage 13.0 (TID 87). 6675 bytes result sent to driver -26/04/01 06:05:45 INFO Executor: Finished task 62.0 in stage 13.0 (TID 89). 6675 bytes result sent to driver -26/04/01 06:05:45 INFO TaskSetManager: Starting task 0.0 in stage 14.0 (TID 91) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:45 INFO Executor: Running task 0.0 in stage 14.0 (TID 91) -26/04/01 06:05:45 INFO TaskSetManager: Starting task 1.0 in stage 14.0 (TID 92) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:05:45 INFO TaskSetManager: Finished task 60.0 in stage 13.0 (TID 87) in 927 ms on 10.0.0.133 (executor driver) (61/64) -26/04/01 06:05:45 INFO Executor: Running task 1.0 in stage 14.0 (TID 92) -26/04/01 06:05:45 INFO TaskSetManager: Finished task 62.0 in stage 13.0 (TID 89) in 926 ms on 10.0.0.133 (executor driver) (62/64) -26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:45 INFO Executor: Finished task 61.0 in stage 13.0 (TID 88). 6675 bytes result sent to driver -26/04/01 06:05:45 INFO TaskSetManager: Starting task 2.0 in stage 14.0 (TID 93) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:05:45 INFO TaskSetManager: Finished task 61.0 in stage 13.0 (TID 88) in 928 ms on 10.0.0.133 (executor driver) (63/64) -26/04/01 06:05:45 INFO Executor: Running task 2.0 in stage 14.0 (TID 93) -26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:45 INFO Executor: Finished task 63.0 in stage 13.0 (TID 90). 6675 bytes result sent to driver -26/04/01 06:05:45 INFO TaskSetManager: Starting task 3.0 in stage 14.0 (TID 94) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:05:45 INFO Executor: Running task 3.0 in stage 14.0 (TID 94) -26/04/01 06:05:45 INFO TaskSetManager: Finished task 63.0 in stage 13.0 (TID 90) in 925 ms on 10.0.0.133 (executor driver) (64/64) -26/04/01 06:05:45 INFO TaskSchedulerImpl: Removed TaskSet 13.0, whose tasks have all completed, from pool -26/04/01 06:05:45 INFO DAGScheduler: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 20.593 s -26/04/01 06:05:45 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:05:45 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 16, ShuffleMapStage 14) -26/04/01 06:05:45 INFO DAGScheduler: waiting: Set() -26/04/01 06:05:45 INFO DAGScheduler: failed: Set() -26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:45 INFO ShufflePartitionsUtil: For shuffle(1, 2), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 06:05:45 INFO DAGScheduler: Registering RDD 44 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 5 -26/04/01 06:05:45 INFO DAGScheduler: Got map stage job 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 6 output partitions -26/04/01 06:05:45 INFO DAGScheduler: Final stage: ShuffleMapStage 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:05:45 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 17, ShuffleMapStage 18) -26/04/01 06:05:45 INFO DAGScheduler: Missing parents: List() -26/04/01 06:05:45 INFO DAGScheduler: Submitting ShuffleMapStage 19 (MapPartitionsRDD[44] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:05:45 INFO MemoryStore: Block broadcast_24 stored as values in memory (estimated size 22.7 KiB, free 8.6 GiB) -26/04/01 06:05:45 INFO MemoryStore: Block broadcast_24_piece0 stored as bytes in memory (estimated size 9.5 KiB, free 8.6 GiB) -26/04/01 06:05:45 INFO BlockManagerInfo: Added broadcast_24_piece0 in memory on 10.0.0.133:58259 (size: 9.5 KiB, free: 8.6 GiB) -26/04/01 06:05:45 INFO SparkContext: Created broadcast 24 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:05:45 INFO DAGScheduler: Submitting 6 missing tasks from ShuffleMapStage 19 (MapPartitionsRDD[44] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5)) -26/04/01 06:05:45 INFO TaskSchedulerImpl: Adding task set 19.0 with 6 tasks resource profile 0 -26/04/01 06:05:49 INFO Executor: Finished task 0.0 in stage 14.0 (TID 91). 6563 bytes result sent to driver -26/04/01 06:05:49 INFO TaskSetManager: Starting task 4.0 in stage 14.0 (TID 95) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:05:49 INFO Executor: Running task 4.0 in stage 14.0 (TID 95) -26/04/01 06:05:49 INFO TaskSetManager: Finished task 0.0 in stage 14.0 (TID 91) in 4156 ms on 10.0.0.133 (executor driver) (1/208) -26/04/01 06:05:49 INFO Executor: Finished task 2.0 in stage 14.0 (TID 93). 6563 bytes result sent to driver -26/04/01 06:05:49 INFO TaskSetManager: Starting task 5.0 in stage 14.0 (TID 96) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:05:49 INFO TaskSetManager: Finished task 2.0 in stage 14.0 (TID 93) in 4154 ms on 10.0.0.133 (executor driver) (2/208) -26/04/01 06:05:49 INFO Executor: Running task 5.0 in stage 14.0 (TID 96) -26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:49 INFO Executor: Finished task 1.0 in stage 14.0 (TID 92). 6563 bytes result sent to driver -26/04/01 06:05:49 INFO TaskSetManager: Starting task 6.0 in stage 14.0 (TID 97) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:49 INFO TaskSetManager: Finished task 1.0 in stage 14.0 (TID 92) in 4160 ms on 10.0.0.133 (executor driver) (3/208) -26/04/01 06:05:49 INFO Executor: Running task 6.0 in stage 14.0 (TID 97) -26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:49 INFO Executor: Finished task 3.0 in stage 14.0 (TID 94). 6563 bytes result sent to driver -26/04/01 06:05:49 INFO TaskSetManager: Starting task 7.0 in stage 14.0 (TID 98) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:05:49 INFO TaskSetManager: Finished task 3.0 in stage 14.0 (TID 94) in 4150 ms on 10.0.0.133 (executor driver) (4/208) -26/04/01 06:05:49 INFO Executor: Running task 7.0 in stage 14.0 (TID 98) -26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:53 INFO Executor: Finished task 4.0 in stage 14.0 (TID 95). 6563 bytes result sent to driver -26/04/01 06:05:53 INFO TaskSetManager: Starting task 8.0 in stage 14.0 (TID 99) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:05:53 INFO TaskSetManager: Finished task 4.0 in stage 14.0 (TID 95) in 4143 ms on 10.0.0.133 (executor driver) (5/208) -26/04/01 06:05:53 INFO Executor: Running task 8.0 in stage 14.0 (TID 99) -26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:53 INFO Executor: Finished task 5.0 in stage 14.0 (TID 96). 6563 bytes result sent to driver -26/04/01 06:05:53 INFO TaskSetManager: Starting task 9.0 in stage 14.0 (TID 100) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:05:53 INFO TaskSetManager: Finished task 5.0 in stage 14.0 (TID 96) in 4144 ms on 10.0.0.133 (executor driver) (6/208) -26/04/01 06:05:53 INFO Executor: Running task 9.0 in stage 14.0 (TID 100) -26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:53 INFO Executor: Finished task 6.0 in stage 14.0 (TID 97). 6563 bytes result sent to driver -26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:53 INFO TaskSetManager: Starting task 10.0 in stage 14.0 (TID 101) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:05:53 INFO TaskSetManager: Finished task 6.0 in stage 14.0 (TID 97) in 4141 ms on 10.0.0.133 (executor driver) (7/208) -26/04/01 06:05:53 INFO Executor: Running task 10.0 in stage 14.0 (TID 101) -26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:53 INFO Executor: Finished task 7.0 in stage 14.0 (TID 98). 6563 bytes result sent to driver -26/04/01 06:05:53 INFO TaskSetManager: Starting task 11.0 in stage 14.0 (TID 102) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:05:53 INFO TaskSetManager: Finished task 7.0 in stage 14.0 (TID 98) in 4124 ms on 10.0.0.133 (executor driver) (8/208) -26/04/01 06:05:53 INFO Executor: Running task 11.0 in stage 14.0 (TID 102) -26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:57 INFO Executor: Finished task 9.0 in stage 14.0 (TID 100). 6563 bytes result sent to driver -26/04/01 06:05:57 INFO TaskSetManager: Starting task 12.0 in stage 14.0 (TID 103) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:05:57 INFO TaskSetManager: Finished task 9.0 in stage 14.0 (TID 100) in 4150 ms on 10.0.0.133 (executor driver) (9/208) -26/04/01 06:05:57 INFO Executor: Running task 12.0 in stage 14.0 (TID 103) -26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:57 INFO Executor: Finished task 8.0 in stage 14.0 (TID 99). 6563 bytes result sent to driver -26/04/01 06:05:57 INFO TaskSetManager: Starting task 13.0 in stage 14.0 (TID 104) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:05:57 INFO TaskSetManager: Finished task 8.0 in stage 14.0 (TID 99) in 4156 ms on 10.0.0.133 (executor driver) (10/208) -26/04/01 06:05:57 INFO Executor: Running task 13.0 in stage 14.0 (TID 104) -26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:57 INFO Executor: Finished task 10.0 in stage 14.0 (TID 101). 6563 bytes result sent to driver -26/04/01 06:05:57 INFO TaskSetManager: Starting task 14.0 in stage 14.0 (TID 105) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:05:57 INFO Executor: Finished task 11.0 in stage 14.0 (TID 102). 6563 bytes result sent to driver -26/04/01 06:05:57 INFO Executor: Running task 14.0 in stage 14.0 (TID 105) -26/04/01 06:05:57 INFO TaskSetManager: Finished task 10.0 in stage 14.0 (TID 101) in 4162 ms on 10.0.0.133 (executor driver) (11/208) -26/04/01 06:05:57 INFO TaskSetManager: Starting task 15.0 in stage 14.0 (TID 106) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:05:57 INFO TaskSetManager: Finished task 11.0 in stage 14.0 (TID 102) in 4154 ms on 10.0.0.133 (executor driver) (12/208) -26/04/01 06:05:57 INFO Executor: Running task 15.0 in stage 14.0 (TID 106) -26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:05:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:01 INFO Executor: Finished task 14.0 in stage 14.0 (TID 105). 6563 bytes result sent to driver -26/04/01 06:06:01 INFO TaskSetManager: Starting task 16.0 in stage 14.0 (TID 107) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:01 INFO TaskSetManager: Finished task 14.0 in stage 14.0 (TID 105) in 4123 ms on 10.0.0.133 (executor driver) (13/208) -26/04/01 06:06:01 INFO Executor: Running task 16.0 in stage 14.0 (TID 107) -26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:01 INFO Executor: Finished task 12.0 in stage 14.0 (TID 103). 6563 bytes result sent to driver -26/04/01 06:06:01 INFO TaskSetManager: Starting task 17.0 in stage 14.0 (TID 108) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:01 INFO Executor: Running task 17.0 in stage 14.0 (TID 108) -26/04/01 06:06:01 INFO TaskSetManager: Finished task 12.0 in stage 14.0 (TID 103) in 4143 ms on 10.0.0.133 (executor driver) (14/208) -26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:01 INFO Executor: Finished task 13.0 in stage 14.0 (TID 104). 6563 bytes result sent to driver -26/04/01 06:06:01 INFO TaskSetManager: Starting task 18.0 in stage 14.0 (TID 109) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:06:01 INFO TaskSetManager: Finished task 13.0 in stage 14.0 (TID 104) in 4141 ms on 10.0.0.133 (executor driver) (15/208) -26/04/01 06:06:01 INFO Executor: Running task 18.0 in stage 14.0 (TID 109) -26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:01 INFO Executor: Finished task 15.0 in stage 14.0 (TID 106). 6563 bytes result sent to driver -26/04/01 06:06:01 INFO TaskSetManager: Starting task 19.0 in stage 14.0 (TID 110) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:06:01 INFO Executor: Running task 19.0 in stage 14.0 (TID 110) -26/04/01 06:06:01 INFO TaskSetManager: Finished task 15.0 in stage 14.0 (TID 106) in 4135 ms on 10.0.0.133 (executor driver) (16/208) -26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:06 INFO Executor: Finished task 18.0 in stage 14.0 (TID 109). 6563 bytes result sent to driver -26/04/01 06:06:06 INFO TaskSetManager: Starting task 20.0 in stage 14.0 (TID 111) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:06 INFO TaskSetManager: Finished task 18.0 in stage 14.0 (TID 109) in 4134 ms on 10.0.0.133 (executor driver) (17/208) -26/04/01 06:06:06 INFO Executor: Running task 20.0 in stage 14.0 (TID 111) -26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:06 INFO Executor: Finished task 16.0 in stage 14.0 (TID 107). 6563 bytes result sent to driver -26/04/01 06:06:06 INFO TaskSetManager: Starting task 21.0 in stage 14.0 (TID 112) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:06 INFO Executor: Running task 21.0 in stage 14.0 (TID 112) -26/04/01 06:06:06 INFO TaskSetManager: Finished task 16.0 in stage 14.0 (TID 107) in 4146 ms on 10.0.0.133 (executor driver) (18/208) -26/04/01 06:06:06 INFO Executor: Finished task 17.0 in stage 14.0 (TID 108). 6563 bytes result sent to driver -26/04/01 06:06:06 INFO TaskSetManager: Starting task 22.0 in stage 14.0 (TID 113) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:06 INFO Executor: Running task 22.0 in stage 14.0 (TID 113) -26/04/01 06:06:06 INFO TaskSetManager: Finished task 17.0 in stage 14.0 (TID 108) in 4142 ms on 10.0.0.133 (executor driver) (19/208) -26/04/01 06:06:06 INFO Executor: Finished task 19.0 in stage 14.0 (TID 110). 6563 bytes result sent to driver -26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:06 INFO TaskSetManager: Starting task 23.0 in stage 14.0 (TID 114) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:06 INFO TaskSetManager: Finished task 19.0 in stage 14.0 (TID 110) in 4137 ms on 10.0.0.133 (executor driver) (20/208) -26/04/01 06:06:06 INFO Executor: Running task 23.0 in stage 14.0 (TID 114) -26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:10 INFO Executor: Finished task 20.0 in stage 14.0 (TID 111). 6563 bytes result sent to driver -26/04/01 06:06:10 INFO Executor: Finished task 22.0 in stage 14.0 (TID 113). 6563 bytes result sent to driver -26/04/01 06:06:10 INFO TaskSetManager: Starting task 24.0 in stage 14.0 (TID 115) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:06:10 INFO Executor: Running task 24.0 in stage 14.0 (TID 115) -26/04/01 06:06:10 INFO TaskSetManager: Starting task 25.0 in stage 14.0 (TID 116) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:10 INFO TaskSetManager: Finished task 20.0 in stage 14.0 (TID 111) in 4146 ms on 10.0.0.133 (executor driver) (21/208) -26/04/01 06:06:10 INFO Executor: Running task 25.0 in stage 14.0 (TID 116) -26/04/01 06:06:10 INFO TaskSetManager: Finished task 22.0 in stage 14.0 (TID 113) in 4140 ms on 10.0.0.133 (executor driver) (22/208) -26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:10 INFO Executor: Finished task 23.0 in stage 14.0 (TID 114). 6563 bytes result sent to driver -26/04/01 06:06:10 INFO TaskSetManager: Starting task 26.0 in stage 14.0 (TID 117) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:10 INFO TaskSetManager: Finished task 23.0 in stage 14.0 (TID 114) in 4141 ms on 10.0.0.133 (executor driver) (23/208) -26/04/01 06:06:10 INFO Executor: Running task 26.0 in stage 14.0 (TID 117) -26/04/01 06:06:10 INFO Executor: Finished task 21.0 in stage 14.0 (TID 112). 6563 bytes result sent to driver -26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:10 INFO TaskSetManager: Starting task 27.0 in stage 14.0 (TID 118) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:10 INFO TaskSetManager: Finished task 21.0 in stage 14.0 (TID 112) in 4143 ms on 10.0.0.133 (executor driver) (24/208) -26/04/01 06:06:10 INFO Executor: Running task 27.0 in stage 14.0 (TID 118) -26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:14 INFO Executor: Finished task 27.0 in stage 14.0 (TID 118). 6563 bytes result sent to driver -26/04/01 06:06:14 INFO TaskSetManager: Starting task 28.0 in stage 14.0 (TID 119) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:14 INFO TaskSetManager: Finished task 27.0 in stage 14.0 (TID 118) in 4091 ms on 10.0.0.133 (executor driver) (25/208) -26/04/01 06:06:14 INFO Executor: Running task 28.0 in stage 14.0 (TID 119) -26/04/01 06:06:14 INFO Executor: Finished task 26.0 in stage 14.0 (TID 117). 6563 bytes result sent to driver -26/04/01 06:06:14 INFO TaskSetManager: Starting task 29.0 in stage 14.0 (TID 120) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:14 INFO TaskSetManager: Finished task 26.0 in stage 14.0 (TID 117) in 4093 ms on 10.0.0.133 (executor driver) (26/208) -26/04/01 06:06:14 INFO Executor: Running task 29.0 in stage 14.0 (TID 120) -26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:14 INFO Executor: Finished task 25.0 in stage 14.0 (TID 116). 6563 bytes result sent to driver -26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:14 INFO TaskSetManager: Starting task 30.0 in stage 14.0 (TID 121) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:06:14 INFO Executor: Finished task 24.0 in stage 14.0 (TID 115). 6563 bytes result sent to driver -26/04/01 06:06:14 INFO Executor: Running task 30.0 in stage 14.0 (TID 121) -26/04/01 06:06:14 INFO TaskSetManager: Starting task 31.0 in stage 14.0 (TID 122) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:14 INFO TaskSetManager: Finished task 25.0 in stage 14.0 (TID 116) in 4095 ms on 10.0.0.133 (executor driver) (27/208) -26/04/01 06:06:14 INFO TaskSetManager: Finished task 24.0 in stage 14.0 (TID 115) in 4096 ms on 10.0.0.133 (executor driver) (28/208) -26/04/01 06:06:14 INFO Executor: Running task 31.0 in stage 14.0 (TID 122) -26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:14 INFO BlockManagerInfo: Removed broadcast_17_piece0 on 10.0.0.133:58259 in memory (size: 8.1 KiB, free: 8.6 GiB) -26/04/01 06:06:18 INFO Executor: Finished task 28.0 in stage 14.0 (TID 119). 6606 bytes result sent to driver -26/04/01 06:06:18 INFO TaskSetManager: Starting task 32.0 in stage 14.0 (TID 123) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:18 INFO Executor: Finished task 31.0 in stage 14.0 (TID 122). 6606 bytes result sent to driver -26/04/01 06:06:18 INFO Executor: Running task 32.0 in stage 14.0 (TID 123) -26/04/01 06:06:18 INFO TaskSetManager: Starting task 33.0 in stage 14.0 (TID 124) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:18 INFO Executor: Finished task 30.0 in stage 14.0 (TID 121). 6606 bytes result sent to driver -26/04/01 06:06:18 INFO TaskSetManager: Finished task 28.0 in stage 14.0 (TID 119) in 4088 ms on 10.0.0.133 (executor driver) (29/208) -26/04/01 06:06:18 INFO Executor: Running task 33.0 in stage 14.0 (TID 124) -26/04/01 06:06:18 INFO TaskSetManager: Finished task 31.0 in stage 14.0 (TID 122) in 4086 ms on 10.0.0.133 (executor driver) (30/208) -26/04/01 06:06:18 INFO TaskSetManager: Starting task 34.0 in stage 14.0 (TID 125) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:18 INFO TaskSetManager: Finished task 30.0 in stage 14.0 (TID 121) in 4086 ms on 10.0.0.133 (executor driver) (31/208) -26/04/01 06:06:18 INFO Executor: Running task 34.0 in stage 14.0 (TID 125) -26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:18 INFO Executor: Finished task 29.0 in stage 14.0 (TID 120). 6606 bytes result sent to driver -26/04/01 06:06:18 INFO TaskSetManager: Starting task 35.0 in stage 14.0 (TID 126) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:18 INFO TaskSetManager: Finished task 29.0 in stage 14.0 (TID 120) in 4093 ms on 10.0.0.133 (executor driver) (32/208) -26/04/01 06:06:18 INFO Executor: Running task 35.0 in stage 14.0 (TID 126) -26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:22 INFO Executor: Finished task 33.0 in stage 14.0 (TID 124). 6563 bytes result sent to driver -26/04/01 06:06:22 INFO Executor: Finished task 32.0 in stage 14.0 (TID 123). 6563 bytes result sent to driver -26/04/01 06:06:22 INFO TaskSetManager: Starting task 36.0 in stage 14.0 (TID 127) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:06:22 INFO TaskSetManager: Finished task 33.0 in stage 14.0 (TID 124) in 4094 ms on 10.0.0.133 (executor driver) (33/208) -26/04/01 06:06:22 INFO Executor: Running task 36.0 in stage 14.0 (TID 127) -26/04/01 06:06:22 INFO TaskSetManager: Starting task 37.0 in stage 14.0 (TID 128) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:22 INFO Executor: Running task 37.0 in stage 14.0 (TID 128) -26/04/01 06:06:22 INFO TaskSetManager: Finished task 32.0 in stage 14.0 (TID 123) in 4094 ms on 10.0.0.133 (executor driver) (34/208) -26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:22 INFO Executor: Finished task 34.0 in stage 14.0 (TID 125). 6563 bytes result sent to driver -26/04/01 06:06:22 INFO Executor: Finished task 35.0 in stage 14.0 (TID 126). 6563 bytes result sent to driver -26/04/01 06:06:22 INFO TaskSetManager: Starting task 38.0 in stage 14.0 (TID 129) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:22 INFO Executor: Running task 38.0 in stage 14.0 (TID 129) -26/04/01 06:06:22 INFO TaskSetManager: Finished task 34.0 in stage 14.0 (TID 125) in 4100 ms on 10.0.0.133 (executor driver) (35/208) -26/04/01 06:06:22 INFO TaskSetManager: Starting task 39.0 in stage 14.0 (TID 130) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:22 INFO Executor: Running task 39.0 in stage 14.0 (TID 130) -26/04/01 06:06:22 INFO TaskSetManager: Finished task 35.0 in stage 14.0 (TID 126) in 4094 ms on 10.0.0.133 (executor driver) (36/208) -26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:26 INFO Executor: Finished task 38.0 in stage 14.0 (TID 129). 6563 bytes result sent to driver -26/04/01 06:06:26 INFO Executor: Finished task 36.0 in stage 14.0 (TID 127). 6563 bytes result sent to driver -26/04/01 06:06:26 INFO TaskSetManager: Starting task 40.0 in stage 14.0 (TID 131) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:26 INFO Executor: Running task 40.0 in stage 14.0 (TID 131) -26/04/01 06:06:26 INFO TaskSetManager: Finished task 38.0 in stage 14.0 (TID 129) in 4154 ms on 10.0.0.133 (executor driver) (37/208) -26/04/01 06:06:26 INFO Executor: Finished task 37.0 in stage 14.0 (TID 128). 6563 bytes result sent to driver -26/04/01 06:06:26 INFO TaskSetManager: Starting task 41.0 in stage 14.0 (TID 132) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:26 INFO TaskSetManager: Finished task 36.0 in stage 14.0 (TID 127) in 4161 ms on 10.0.0.133 (executor driver) (38/208) -26/04/01 06:06:26 INFO Executor: Running task 41.0 in stage 14.0 (TID 132) -26/04/01 06:06:26 INFO TaskSetManager: Starting task 42.0 in stage 14.0 (TID 133) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:06:26 INFO TaskSetManager: Finished task 37.0 in stage 14.0 (TID 128) in 4161 ms on 10.0.0.133 (executor driver) (39/208) -26/04/01 06:06:26 INFO Executor: Running task 42.0 in stage 14.0 (TID 133) -26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:26 INFO Executor: Finished task 39.0 in stage 14.0 (TID 130). 6563 bytes result sent to driver -26/04/01 06:06:26 INFO TaskSetManager: Starting task 43.0 in stage 14.0 (TID 134) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:06:26 INFO TaskSetManager: Finished task 39.0 in stage 14.0 (TID 130) in 4165 ms on 10.0.0.133 (executor driver) (40/208) -26/04/01 06:06:26 INFO Executor: Running task 43.0 in stage 14.0 (TID 134) -26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:30 INFO Executor: Finished task 42.0 in stage 14.0 (TID 133). 6563 bytes result sent to driver -26/04/01 06:06:30 INFO TaskSetManager: Starting task 44.0 in stage 14.0 (TID 135) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:30 INFO TaskSetManager: Finished task 42.0 in stage 14.0 (TID 133) in 4088 ms on 10.0.0.133 (executor driver) (41/208) -26/04/01 06:06:30 INFO Executor: Running task 44.0 in stage 14.0 (TID 135) -26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:30 INFO Executor: Finished task 43.0 in stage 14.0 (TID 134). 6563 bytes result sent to driver -26/04/01 06:06:30 INFO TaskSetManager: Starting task 45.0 in stage 14.0 (TID 136) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:30 INFO Executor: Running task 45.0 in stage 14.0 (TID 136) -26/04/01 06:06:30 INFO TaskSetManager: Finished task 43.0 in stage 14.0 (TID 134) in 4083 ms on 10.0.0.133 (executor driver) (42/208) -26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:30 INFO Executor: Finished task 41.0 in stage 14.0 (TID 132). 6563 bytes result sent to driver -26/04/01 06:06:30 INFO TaskSetManager: Starting task 46.0 in stage 14.0 (TID 137) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:30 INFO TaskSetManager: Finished task 41.0 in stage 14.0 (TID 132) in 4137 ms on 10.0.0.133 (executor driver) (43/208) -26/04/01 06:06:30 INFO Executor: Running task 46.0 in stage 14.0 (TID 137) -26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:30 INFO Executor: Finished task 40.0 in stage 14.0 (TID 131). 6563 bytes result sent to driver -26/04/01 06:06:30 INFO TaskSetManager: Starting task 47.0 in stage 14.0 (TID 138) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:30 INFO TaskSetManager: Finished task 40.0 in stage 14.0 (TID 131) in 4153 ms on 10.0.0.133 (executor driver) (44/208) -26/04/01 06:06:30 INFO Executor: Running task 47.0 in stage 14.0 (TID 138) -26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:34 INFO Executor: Finished task 45.0 in stage 14.0 (TID 136). 6563 bytes result sent to driver -26/04/01 06:06:34 INFO TaskSetManager: Starting task 48.0 in stage 14.0 (TID 139) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:06:34 INFO Executor: Running task 48.0 in stage 14.0 (TID 139) -26/04/01 06:06:34 INFO TaskSetManager: Finished task 45.0 in stage 14.0 (TID 136) in 3932 ms on 10.0.0.133 (executor driver) (45/208) -26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:34 INFO Executor: Finished task 46.0 in stage 14.0 (TID 137). 6563 bytes result sent to driver -26/04/01 06:06:34 INFO TaskSetManager: Starting task 49.0 in stage 14.0 (TID 140) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:34 INFO TaskSetManager: Finished task 46.0 in stage 14.0 (TID 137) in 4089 ms on 10.0.0.133 (executor driver) (46/208) -26/04/01 06:06:34 INFO Executor: Running task 49.0 in stage 14.0 (TID 140) -26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:34 INFO Executor: Finished task 47.0 in stage 14.0 (TID 138). 6563 bytes result sent to driver -26/04/01 06:06:34 INFO TaskSetManager: Starting task 50.0 in stage 14.0 (TID 141) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:34 INFO Executor: Running task 50.0 in stage 14.0 (TID 141) -26/04/01 06:06:34 INFO TaskSetManager: Finished task 47.0 in stage 14.0 (TID 138) in 4078 ms on 10.0.0.133 (executor driver) (47/208) -26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:34 INFO Executor: Finished task 44.0 in stage 14.0 (TID 135). 6563 bytes result sent to driver -26/04/01 06:06:34 INFO TaskSetManager: Starting task 51.0 in stage 14.0 (TID 142) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:34 INFO TaskSetManager: Finished task 44.0 in stage 14.0 (TID 135) in 4240 ms on 10.0.0.133 (executor driver) (48/208) -26/04/01 06:06:34 INFO Executor: Running task 51.0 in stage 14.0 (TID 142) -26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:38 INFO Executor: Finished task 48.0 in stage 14.0 (TID 139). 6563 bytes result sent to driver -26/04/01 06:06:38 INFO TaskSetManager: Starting task 52.0 in stage 14.0 (TID 143) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:38 INFO TaskSetManager: Finished task 48.0 in stage 14.0 (TID 139) in 4162 ms on 10.0.0.133 (executor driver) (49/208) -26/04/01 06:06:38 INFO Executor: Running task 52.0 in stage 14.0 (TID 143) -26/04/01 06:06:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:38 INFO Executor: Finished task 49.0 in stage 14.0 (TID 140). 6563 bytes result sent to driver -26/04/01 06:06:38 INFO TaskSetManager: Starting task 53.0 in stage 14.0 (TID 144) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:38 INFO TaskSetManager: Finished task 49.0 in stage 14.0 (TID 140) in 4146 ms on 10.0.0.133 (executor driver) (50/208) -26/04/01 06:06:38 INFO Executor: Running task 53.0 in stage 14.0 (TID 144) -26/04/01 06:06:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:38 INFO Executor: Finished task 50.0 in stage 14.0 (TID 141). 6563 bytes result sent to driver -26/04/01 06:06:38 INFO TaskSetManager: Starting task 54.0 in stage 14.0 (TID 145) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:06:38 INFO TaskSetManager: Finished task 50.0 in stage 14.0 (TID 141) in 4156 ms on 10.0.0.133 (executor driver) (51/208) -26/04/01 06:06:38 INFO Executor: Running task 54.0 in stage 14.0 (TID 145) -26/04/01 06:06:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:39 INFO Executor: Finished task 51.0 in stage 14.0 (TID 142). 6563 bytes result sent to driver -26/04/01 06:06:39 INFO TaskSetManager: Starting task 55.0 in stage 14.0 (TID 146) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:39 INFO Executor: Running task 55.0 in stage 14.0 (TID 146) -26/04/01 06:06:39 INFO TaskSetManager: Finished task 51.0 in stage 14.0 (TID 142) in 4157 ms on 10.0.0.133 (executor driver) (52/208) -26/04/01 06:06:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:42 INFO Executor: Finished task 52.0 in stage 14.0 (TID 143). 6563 bytes result sent to driver -26/04/01 06:06:42 INFO TaskSetManager: Starting task 56.0 in stage 14.0 (TID 147) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:42 INFO Executor: Running task 56.0 in stage 14.0 (TID 147) -26/04/01 06:06:42 INFO TaskSetManager: Finished task 52.0 in stage 14.0 (TID 143) in 4152 ms on 10.0.0.133 (executor driver) (53/208) -26/04/01 06:06:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:43 INFO Executor: Finished task 53.0 in stage 14.0 (TID 144). 6563 bytes result sent to driver -26/04/01 06:06:43 INFO TaskSetManager: Starting task 57.0 in stage 14.0 (TID 148) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:43 INFO TaskSetManager: Finished task 53.0 in stage 14.0 (TID 144) in 4134 ms on 10.0.0.133 (executor driver) (54/208) -26/04/01 06:06:43 INFO Executor: Running task 57.0 in stage 14.0 (TID 148) -26/04/01 06:06:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:43 INFO Executor: Finished task 54.0 in stage 14.0 (TID 145). 6563 bytes result sent to driver -26/04/01 06:06:43 INFO TaskSetManager: Starting task 58.0 in stage 14.0 (TID 149) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:43 INFO TaskSetManager: Finished task 54.0 in stage 14.0 (TID 145) in 4140 ms on 10.0.0.133 (executor driver) (55/208) -26/04/01 06:06:43 INFO Executor: Running task 58.0 in stage 14.0 (TID 149) -26/04/01 06:06:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:43 INFO Executor: Finished task 55.0 in stage 14.0 (TID 146). 6563 bytes result sent to driver -26/04/01 06:06:43 INFO TaskSetManager: Starting task 59.0 in stage 14.0 (TID 150) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:43 INFO TaskSetManager: Finished task 55.0 in stage 14.0 (TID 146) in 4131 ms on 10.0.0.133 (executor driver) (56/208) -26/04/01 06:06:43 INFO Executor: Running task 59.0 in stage 14.0 (TID 150) -26/04/01 06:06:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:47 INFO Executor: Finished task 56.0 in stage 14.0 (TID 147). 6563 bytes result sent to driver -26/04/01 06:06:47 INFO TaskSetManager: Starting task 60.0 in stage 14.0 (TID 151) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:06:47 INFO TaskSetManager: Finished task 56.0 in stage 14.0 (TID 147) in 4143 ms on 10.0.0.133 (executor driver) (57/208) -26/04/01 06:06:47 INFO Executor: Running task 60.0 in stage 14.0 (TID 151) -26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:47 INFO Executor: Finished task 57.0 in stage 14.0 (TID 148). 6563 bytes result sent to driver -26/04/01 06:06:47 INFO TaskSetManager: Starting task 61.0 in stage 14.0 (TID 152) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:47 INFO TaskSetManager: Finished task 57.0 in stage 14.0 (TID 148) in 4150 ms on 10.0.0.133 (executor driver) (58/208) -26/04/01 06:06:47 INFO Executor: Running task 61.0 in stage 14.0 (TID 152) -26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:47 INFO Executor: Finished task 58.0 in stage 14.0 (TID 149). 6563 bytes result sent to driver -26/04/01 06:06:47 INFO TaskSetManager: Starting task 62.0 in stage 14.0 (TID 153) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:47 INFO TaskSetManager: Finished task 58.0 in stage 14.0 (TID 149) in 4140 ms on 10.0.0.133 (executor driver) (59/208) -26/04/01 06:06:47 INFO Executor: Running task 62.0 in stage 14.0 (TID 153) -26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:47 INFO Executor: Finished task 59.0 in stage 14.0 (TID 150). 6563 bytes result sent to driver -26/04/01 06:06:47 INFO TaskSetManager: Starting task 63.0 in stage 14.0 (TID 154) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:47 INFO TaskSetManager: Finished task 59.0 in stage 14.0 (TID 150) in 4141 ms on 10.0.0.133 (executor driver) (60/208) -26/04/01 06:06:47 INFO Executor: Running task 63.0 in stage 14.0 (TID 154) -26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:51 INFO Executor: Finished task 60.0 in stage 14.0 (TID 151). 6563 bytes result sent to driver -26/04/01 06:06:51 INFO TaskSetManager: Starting task 64.0 in stage 14.0 (TID 155) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:51 INFO TaskSetManager: Finished task 60.0 in stage 14.0 (TID 151) in 4102 ms on 10.0.0.133 (executor driver) (61/208) -26/04/01 06:06:51 INFO Executor: Running task 64.0 in stage 14.0 (TID 155) -26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:51 INFO Executor: Finished task 61.0 in stage 14.0 (TID 152). 6563 bytes result sent to driver -26/04/01 06:06:51 INFO Executor: Finished task 62.0 in stage 14.0 (TID 153). 6563 bytes result sent to driver -26/04/01 06:06:51 INFO TaskSetManager: Starting task 65.0 in stage 14.0 (TID 156) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:06:51 INFO Executor: Running task 65.0 in stage 14.0 (TID 156) -26/04/01 06:06:51 INFO TaskSetManager: Starting task 66.0 in stage 14.0 (TID 157) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:06:51 INFO TaskSetManager: Finished task 61.0 in stage 14.0 (TID 152) in 4102 ms on 10.0.0.133 (executor driver) (62/208) -26/04/01 06:06:51 INFO Executor: Running task 66.0 in stage 14.0 (TID 157) -26/04/01 06:06:51 INFO TaskSetManager: Finished task 62.0 in stage 14.0 (TID 153) in 4092 ms on 10.0.0.133 (executor driver) (63/208) -26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:51 INFO Executor: Finished task 63.0 in stage 14.0 (TID 154). 6563 bytes result sent to driver -26/04/01 06:06:51 INFO TaskSetManager: Starting task 67.0 in stage 14.0 (TID 158) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:06:51 INFO TaskSetManager: Finished task 63.0 in stage 14.0 (TID 154) in 4098 ms on 10.0.0.133 (executor driver) (64/208) -26/04/01 06:06:51 INFO Executor: Running task 67.0 in stage 14.0 (TID 158) -26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:55 INFO Executor: Finished task 64.0 in stage 14.0 (TID 155). 6563 bytes result sent to driver -26/04/01 06:06:55 INFO TaskSetManager: Starting task 68.0 in stage 14.0 (TID 159) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:55 INFO Executor: Running task 68.0 in stage 14.0 (TID 159) -26/04/01 06:06:55 INFO TaskSetManager: Finished task 64.0 in stage 14.0 (TID 155) in 4101 ms on 10.0.0.133 (executor driver) (65/208) -26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:55 INFO Executor: Finished task 66.0 in stage 14.0 (TID 157). 6563 bytes result sent to driver -26/04/01 06:06:55 INFO TaskSetManager: Starting task 69.0 in stage 14.0 (TID 160) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:55 INFO Executor: Running task 69.0 in stage 14.0 (TID 160) -26/04/01 06:06:55 INFO TaskSetManager: Finished task 66.0 in stage 14.0 (TID 157) in 4090 ms on 10.0.0.133 (executor driver) (66/208) -26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:55 INFO Executor: Finished task 65.0 in stage 14.0 (TID 156). 6563 bytes result sent to driver -26/04/01 06:06:55 INFO TaskSetManager: Starting task 70.0 in stage 14.0 (TID 161) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:55 INFO TaskSetManager: Finished task 65.0 in stage 14.0 (TID 156) in 4100 ms on 10.0.0.133 (executor driver) (67/208) -26/04/01 06:06:55 INFO Executor: Running task 70.0 in stage 14.0 (TID 161) -26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:55 INFO Executor: Finished task 67.0 in stage 14.0 (TID 158). 6563 bytes result sent to driver -26/04/01 06:06:55 INFO TaskSetManager: Starting task 71.0 in stage 14.0 (TID 162) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:55 INFO Executor: Running task 71.0 in stage 14.0 (TID 162) -26/04/01 06:06:55 INFO TaskSetManager: Finished task 67.0 in stage 14.0 (TID 158) in 4100 ms on 10.0.0.133 (executor driver) (68/208) -26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:59 INFO Executor: Finished task 68.0 in stage 14.0 (TID 159). 6606 bytes result sent to driver -26/04/01 06:06:59 INFO TaskSetManager: Starting task 72.0 in stage 14.0 (TID 163) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:06:59 INFO TaskSetManager: Finished task 68.0 in stage 14.0 (TID 159) in 4097 ms on 10.0.0.133 (executor driver) (69/208) -26/04/01 06:06:59 INFO Executor: Running task 72.0 in stage 14.0 (TID 163) -26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:59 INFO Executor: Finished task 69.0 in stage 14.0 (TID 160). 6606 bytes result sent to driver -26/04/01 06:06:59 INFO TaskSetManager: Starting task 73.0 in stage 14.0 (TID 164) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:06:59 INFO TaskSetManager: Finished task 69.0 in stage 14.0 (TID 160) in 4081 ms on 10.0.0.133 (executor driver) (70/208) -26/04/01 06:06:59 INFO Executor: Running task 73.0 in stage 14.0 (TID 164) -26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:59 INFO Executor: Finished task 70.0 in stage 14.0 (TID 161). 6606 bytes result sent to driver -26/04/01 06:06:59 INFO TaskSetManager: Starting task 74.0 in stage 14.0 (TID 165) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:59 INFO TaskSetManager: Finished task 70.0 in stage 14.0 (TID 161) in 4097 ms on 10.0.0.133 (executor driver) (71/208) -26/04/01 06:06:59 INFO Executor: Running task 74.0 in stage 14.0 (TID 165) -26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:59 INFO Executor: Finished task 71.0 in stage 14.0 (TID 162). 6606 bytes result sent to driver -26/04/01 06:06:59 INFO TaskSetManager: Starting task 75.0 in stage 14.0 (TID 166) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:06:59 INFO TaskSetManager: Finished task 71.0 in stage 14.0 (TID 162) in 4092 ms on 10.0.0.133 (executor driver) (72/208) -26/04/01 06:06:59 INFO Executor: Running task 75.0 in stage 14.0 (TID 166) -26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:06:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:03 INFO Executor: Finished task 72.0 in stage 14.0 (TID 163). 6563 bytes result sent to driver -26/04/01 06:07:03 INFO TaskSetManager: Starting task 76.0 in stage 14.0 (TID 167) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:03 INFO TaskSetManager: Finished task 72.0 in stage 14.0 (TID 163) in 4143 ms on 10.0.0.133 (executor driver) (73/208) -26/04/01 06:07:03 INFO Executor: Running task 76.0 in stage 14.0 (TID 167) -26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:03 INFO Executor: Finished task 73.0 in stage 14.0 (TID 164). 6563 bytes result sent to driver -26/04/01 06:07:03 INFO TaskSetManager: Starting task 77.0 in stage 14.0 (TID 168) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:03 INFO TaskSetManager: Finished task 73.0 in stage 14.0 (TID 164) in 4138 ms on 10.0.0.133 (executor driver) (74/208) -26/04/01 06:07:03 INFO Executor: Running task 77.0 in stage 14.0 (TID 168) -26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:03 INFO Executor: Finished task 74.0 in stage 14.0 (TID 165). 6563 bytes result sent to driver -26/04/01 06:07:03 INFO TaskSetManager: Starting task 78.0 in stage 14.0 (TID 169) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:07:03 INFO TaskSetManager: Finished task 74.0 in stage 14.0 (TID 165) in 4125 ms on 10.0.0.133 (executor driver) (75/208) -26/04/01 06:07:03 INFO Executor: Running task 78.0 in stage 14.0 (TID 169) -26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:03 INFO Executor: Finished task 75.0 in stage 14.0 (TID 166). 6563 bytes result sent to driver -26/04/01 06:07:03 INFO TaskSetManager: Starting task 79.0 in stage 14.0 (TID 170) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:03 INFO TaskSetManager: Finished task 75.0 in stage 14.0 (TID 166) in 4127 ms on 10.0.0.133 (executor driver) (76/208) -26/04/01 06:07:03 INFO Executor: Running task 79.0 in stage 14.0 (TID 170) -26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:07 INFO Executor: Finished task 76.0 in stage 14.0 (TID 167). 6563 bytes result sent to driver -26/04/01 06:07:07 INFO TaskSetManager: Starting task 80.0 in stage 14.0 (TID 171) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:07 INFO TaskSetManager: Finished task 76.0 in stage 14.0 (TID 167) in 4140 ms on 10.0.0.133 (executor driver) (77/208) -26/04/01 06:07:07 INFO Executor: Running task 80.0 in stage 14.0 (TID 171) -26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:07 INFO Executor: Finished task 77.0 in stage 14.0 (TID 168). 6563 bytes result sent to driver -26/04/01 06:07:07 INFO TaskSetManager: Starting task 81.0 in stage 14.0 (TID 172) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:07 INFO TaskSetManager: Finished task 77.0 in stage 14.0 (TID 168) in 4130 ms on 10.0.0.133 (executor driver) (78/208) -26/04/01 06:07:07 INFO Executor: Running task 81.0 in stage 14.0 (TID 172) -26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:07 INFO Executor: Finished task 78.0 in stage 14.0 (TID 169). 6563 bytes result sent to driver -26/04/01 06:07:07 INFO TaskSetManager: Starting task 82.0 in stage 14.0 (TID 173) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:07 INFO TaskSetManager: Finished task 78.0 in stage 14.0 (TID 169) in 4127 ms on 10.0.0.133 (executor driver) (79/208) -26/04/01 06:07:07 INFO Executor: Running task 82.0 in stage 14.0 (TID 173) -26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:07 INFO Executor: Finished task 79.0 in stage 14.0 (TID 170). 6563 bytes result sent to driver -26/04/01 06:07:07 INFO TaskSetManager: Starting task 83.0 in stage 14.0 (TID 174) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:07 INFO TaskSetManager: Finished task 79.0 in stage 14.0 (TID 170) in 4126 ms on 10.0.0.133 (executor driver) (80/208) -26/04/01 06:07:07 INFO Executor: Running task 83.0 in stage 14.0 (TID 174) -26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:11 INFO Executor: Finished task 80.0 in stage 14.0 (TID 171). 6563 bytes result sent to driver -26/04/01 06:07:11 INFO TaskSetManager: Starting task 84.0 in stage 14.0 (TID 175) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:07:11 INFO TaskSetManager: Finished task 80.0 in stage 14.0 (TID 171) in 4139 ms on 10.0.0.133 (executor driver) (81/208) -26/04/01 06:07:11 INFO Executor: Running task 84.0 in stage 14.0 (TID 175) -26/04/01 06:07:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:11 INFO Executor: Finished task 81.0 in stage 14.0 (TID 172). 6563 bytes result sent to driver -26/04/01 06:07:11 INFO TaskSetManager: Starting task 85.0 in stage 14.0 (TID 176) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:11 INFO Executor: Running task 85.0 in stage 14.0 (TID 176) -26/04/01 06:07:11 INFO TaskSetManager: Finished task 81.0 in stage 14.0 (TID 172) in 4144 ms on 10.0.0.133 (executor driver) (82/208) -26/04/01 06:07:11 INFO Executor: Finished task 82.0 in stage 14.0 (TID 173). 6563 bytes result sent to driver -26/04/01 06:07:11 INFO TaskSetManager: Starting task 86.0 in stage 14.0 (TID 177) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:11 INFO Executor: Running task 86.0 in stage 14.0 (TID 177) -26/04/01 06:07:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:11 INFO TaskSetManager: Finished task 82.0 in stage 14.0 (TID 173) in 4135 ms on 10.0.0.133 (executor driver) (83/208) -26/04/01 06:07:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:12 INFO Executor: Finished task 83.0 in stage 14.0 (TID 174). 6563 bytes result sent to driver -26/04/01 06:07:12 INFO TaskSetManager: Starting task 87.0 in stage 14.0 (TID 178) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:12 INFO TaskSetManager: Finished task 83.0 in stage 14.0 (TID 174) in 4141 ms on 10.0.0.133 (executor driver) (84/208) -26/04/01 06:07:12 INFO Executor: Running task 87.0 in stage 14.0 (TID 178) -26/04/01 06:07:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:15 INFO Executor: Finished task 84.0 in stage 14.0 (TID 175). 6563 bytes result sent to driver -26/04/01 06:07:15 INFO TaskSetManager: Starting task 88.0 in stage 14.0 (TID 179) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:15 INFO TaskSetManager: Finished task 84.0 in stage 14.0 (TID 175) in 4147 ms on 10.0.0.133 (executor driver) (85/208) -26/04/01 06:07:15 INFO Executor: Running task 88.0 in stage 14.0 (TID 179) -26/04/01 06:07:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:16 INFO Executor: Finished task 86.0 in stage 14.0 (TID 177). 6563 bytes result sent to driver -26/04/01 06:07:16 INFO TaskSetManager: Starting task 89.0 in stage 14.0 (TID 180) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:16 INFO TaskSetManager: Finished task 86.0 in stage 14.0 (TID 177) in 4139 ms on 10.0.0.133 (executor driver) (86/208) -26/04/01 06:07:16 INFO Executor: Running task 89.0 in stage 14.0 (TID 180) -26/04/01 06:07:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:16 INFO Executor: Finished task 85.0 in stage 14.0 (TID 176). 6563 bytes result sent to driver -26/04/01 06:07:16 INFO TaskSetManager: Starting task 90.0 in stage 14.0 (TID 181) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:07:16 INFO Executor: Running task 90.0 in stage 14.0 (TID 181) -26/04/01 06:07:16 INFO TaskSetManager: Finished task 85.0 in stage 14.0 (TID 176) in 4154 ms on 10.0.0.133 (executor driver) (87/208) -26/04/01 06:07:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:16 INFO Executor: Finished task 87.0 in stage 14.0 (TID 178). 6563 bytes result sent to driver -26/04/01 06:07:16 INFO TaskSetManager: Starting task 91.0 in stage 14.0 (TID 182) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:16 INFO Executor: Running task 91.0 in stage 14.0 (TID 182) -26/04/01 06:07:16 INFO TaskSetManager: Finished task 87.0 in stage 14.0 (TID 178) in 4146 ms on 10.0.0.133 (executor driver) (88/208) -26/04/01 06:07:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:20 INFO Executor: Finished task 88.0 in stage 14.0 (TID 179). 6563 bytes result sent to driver -26/04/01 06:07:20 INFO TaskSetManager: Starting task 92.0 in stage 14.0 (TID 183) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:20 INFO TaskSetManager: Finished task 88.0 in stage 14.0 (TID 179) in 4137 ms on 10.0.0.133 (executor driver) (89/208) -26/04/01 06:07:20 INFO Executor: Running task 92.0 in stage 14.0 (TID 183) -26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:20 INFO Executor: Finished task 91.0 in stage 14.0 (TID 182). 6563 bytes result sent to driver -26/04/01 06:07:20 INFO TaskSetManager: Starting task 93.0 in stage 14.0 (TID 184) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:20 INFO Executor: Running task 93.0 in stage 14.0 (TID 184) -26/04/01 06:07:20 INFO TaskSetManager: Finished task 91.0 in stage 14.0 (TID 182) in 3937 ms on 10.0.0.133 (executor driver) (90/208) -26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:20 INFO Executor: Finished task 89.0 in stage 14.0 (TID 180). 6563 bytes result sent to driver -26/04/01 06:07:20 INFO TaskSetManager: Starting task 94.0 in stage 14.0 (TID 185) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:20 INFO TaskSetManager: Finished task 89.0 in stage 14.0 (TID 180) in 4147 ms on 10.0.0.133 (executor driver) (91/208) -26/04/01 06:07:20 INFO Executor: Running task 94.0 in stage 14.0 (TID 185) -26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:20 INFO Executor: Finished task 90.0 in stage 14.0 (TID 181). 6563 bytes result sent to driver -26/04/01 06:07:20 INFO TaskSetManager: Starting task 95.0 in stage 14.0 (TID 186) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:20 INFO Executor: Running task 95.0 in stage 14.0 (TID 186) -26/04/01 06:07:20 INFO TaskSetManager: Finished task 90.0 in stage 14.0 (TID 181) in 4231 ms on 10.0.0.133 (executor driver) (92/208) -26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:24 INFO Executor: Finished task 92.0 in stage 14.0 (TID 183). 6563 bytes result sent to driver -26/04/01 06:07:24 INFO TaskSetManager: Starting task 96.0 in stage 14.0 (TID 187) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:07:24 INFO TaskSetManager: Finished task 92.0 in stage 14.0 (TID 183) in 4083 ms on 10.0.0.133 (executor driver) (93/208) -26/04/01 06:07:24 INFO Executor: Running task 96.0 in stage 14.0 (TID 187) -26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:24 INFO Executor: Finished task 93.0 in stage 14.0 (TID 184). 6563 bytes result sent to driver -26/04/01 06:07:24 INFO TaskSetManager: Starting task 97.0 in stage 14.0 (TID 188) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:07:24 INFO TaskSetManager: Finished task 93.0 in stage 14.0 (TID 184) in 4072 ms on 10.0.0.133 (executor driver) (94/208) -26/04/01 06:07:24 INFO Executor: Running task 97.0 in stage 14.0 (TID 188) -26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:24 INFO Executor: Finished task 94.0 in stage 14.0 (TID 185). 6563 bytes result sent to driver -26/04/01 06:07:24 INFO TaskSetManager: Starting task 98.0 in stage 14.0 (TID 189) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:24 INFO TaskSetManager: Finished task 94.0 in stage 14.0 (TID 185) in 4083 ms on 10.0.0.133 (executor driver) (95/208) -26/04/01 06:07:24 INFO Executor: Running task 98.0 in stage 14.0 (TID 189) -26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:24 INFO Executor: Finished task 95.0 in stage 14.0 (TID 186). 6563 bytes result sent to driver -26/04/01 06:07:24 INFO TaskSetManager: Starting task 99.0 in stage 14.0 (TID 190) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:24 INFO TaskSetManager: Finished task 95.0 in stage 14.0 (TID 186) in 4092 ms on 10.0.0.133 (executor driver) (96/208) -26/04/01 06:07:24 INFO Executor: Running task 99.0 in stage 14.0 (TID 190) -26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:28 INFO Executor: Finished task 96.0 in stage 14.0 (TID 187). 6563 bytes result sent to driver -26/04/01 06:07:28 INFO TaskSetManager: Starting task 100.0 in stage 14.0 (TID 191) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:28 INFO TaskSetManager: Finished task 96.0 in stage 14.0 (TID 187) in 4149 ms on 10.0.0.133 (executor driver) (97/208) -26/04/01 06:07:28 INFO Executor: Running task 100.0 in stage 14.0 (TID 191) -26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:28 INFO Executor: Finished task 97.0 in stage 14.0 (TID 188). 6563 bytes result sent to driver -26/04/01 06:07:28 INFO TaskSetManager: Starting task 101.0 in stage 14.0 (TID 192) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:28 INFO Executor: Running task 101.0 in stage 14.0 (TID 192) -26/04/01 06:07:28 INFO TaskSetManager: Finished task 97.0 in stage 14.0 (TID 188) in 4139 ms on 10.0.0.133 (executor driver) (98/208) -26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:28 INFO Executor: Finished task 98.0 in stage 14.0 (TID 189). 6563 bytes result sent to driver -26/04/01 06:07:28 INFO TaskSetManager: Starting task 102.0 in stage 14.0 (TID 193) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:07:28 INFO Executor: Running task 102.0 in stage 14.0 (TID 193) -26/04/01 06:07:28 INFO TaskSetManager: Finished task 98.0 in stage 14.0 (TID 189) in 4151 ms on 10.0.0.133 (executor driver) (99/208) -26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:28 INFO Executor: Finished task 99.0 in stage 14.0 (TID 190). 6563 bytes result sent to driver -26/04/01 06:07:28 INFO TaskSetManager: Starting task 103.0 in stage 14.0 (TID 194) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:28 INFO TaskSetManager: Finished task 99.0 in stage 14.0 (TID 190) in 4133 ms on 10.0.0.133 (executor driver) (100/208) -26/04/01 06:07:28 INFO Executor: Running task 103.0 in stage 14.0 (TID 194) -26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:32 INFO Executor: Finished task 100.0 in stage 14.0 (TID 191). 6563 bytes result sent to driver -26/04/01 06:07:32 INFO TaskSetManager: Starting task 104.0 in stage 14.0 (TID 195) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:32 INFO TaskSetManager: Finished task 100.0 in stage 14.0 (TID 191) in 4150 ms on 10.0.0.133 (executor driver) (101/208) -26/04/01 06:07:32 INFO Executor: Running task 104.0 in stage 14.0 (TID 195) -26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:32 INFO Executor: Finished task 101.0 in stage 14.0 (TID 192). 6563 bytes result sent to driver -26/04/01 06:07:32 INFO TaskSetManager: Starting task 105.0 in stage 14.0 (TID 196) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:32 INFO Executor: Running task 105.0 in stage 14.0 (TID 196) -26/04/01 06:07:32 INFO TaskSetManager: Finished task 101.0 in stage 14.0 (TID 192) in 4149 ms on 10.0.0.133 (executor driver) (102/208) -26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:32 INFO Executor: Finished task 102.0 in stage 14.0 (TID 193). 6563 bytes result sent to driver -26/04/01 06:07:32 INFO TaskSetManager: Starting task 106.0 in stage 14.0 (TID 197) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:32 INFO TaskSetManager: Finished task 102.0 in stage 14.0 (TID 193) in 4134 ms on 10.0.0.133 (executor driver) (103/208) -26/04/01 06:07:32 INFO Executor: Running task 106.0 in stage 14.0 (TID 197) -26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:32 INFO Executor: Finished task 103.0 in stage 14.0 (TID 194). 6563 bytes result sent to driver -26/04/01 06:07:32 INFO TaskSetManager: Starting task 107.0 in stage 14.0 (TID 198) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:32 INFO TaskSetManager: Finished task 103.0 in stage 14.0 (TID 194) in 4153 ms on 10.0.0.133 (executor driver) (104/208) -26/04/01 06:07:32 INFO Executor: Running task 107.0 in stage 14.0 (TID 198) -26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:36 INFO Executor: Finished task 104.0 in stage 14.0 (TID 195). 6563 bytes result sent to driver -26/04/01 06:07:36 INFO TaskSetManager: Starting task 108.0 in stage 14.0 (TID 199) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:07:36 INFO TaskSetManager: Finished task 104.0 in stage 14.0 (TID 195) in 4140 ms on 10.0.0.133 (executor driver) (105/208) -26/04/01 06:07:36 INFO Executor: Running task 108.0 in stage 14.0 (TID 199) -26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:36 INFO Executor: Finished task 105.0 in stage 14.0 (TID 196). 6563 bytes result sent to driver -26/04/01 06:07:36 INFO TaskSetManager: Starting task 109.0 in stage 14.0 (TID 200) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:07:36 INFO TaskSetManager: Finished task 105.0 in stage 14.0 (TID 196) in 4136 ms on 10.0.0.133 (executor driver) (106/208) -26/04/01 06:07:36 INFO Executor: Running task 109.0 in stage 14.0 (TID 200) -26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:36 INFO Executor: Finished task 106.0 in stage 14.0 (TID 197). 6563 bytes result sent to driver -26/04/01 06:07:36 INFO TaskSetManager: Starting task 110.0 in stage 14.0 (TID 201) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:36 INFO TaskSetManager: Finished task 106.0 in stage 14.0 (TID 197) in 4155 ms on 10.0.0.133 (executor driver) (107/208) -26/04/01 06:07:36 INFO Executor: Running task 110.0 in stage 14.0 (TID 201) -26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:36 INFO Executor: Finished task 107.0 in stage 14.0 (TID 198). 6563 bytes result sent to driver -26/04/01 06:07:36 INFO TaskSetManager: Starting task 111.0 in stage 14.0 (TID 202) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:36 INFO TaskSetManager: Finished task 107.0 in stage 14.0 (TID 198) in 4144 ms on 10.0.0.133 (executor driver) (108/208) -26/04/01 06:07:36 INFO Executor: Running task 111.0 in stage 14.0 (TID 202) -26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:40 INFO Executor: Finished task 108.0 in stage 14.0 (TID 199). 6563 bytes result sent to driver -26/04/01 06:07:40 INFO TaskSetManager: Starting task 112.0 in stage 14.0 (TID 203) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:40 INFO TaskSetManager: Finished task 108.0 in stage 14.0 (TID 199) in 4150 ms on 10.0.0.133 (executor driver) (109/208) -26/04/01 06:07:40 INFO Executor: Running task 112.0 in stage 14.0 (TID 203) -26/04/01 06:07:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:40 INFO Executor: Finished task 109.0 in stage 14.0 (TID 200). 6563 bytes result sent to driver -26/04/01 06:07:40 INFO TaskSetManager: Starting task 113.0 in stage 14.0 (TID 204) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:40 INFO TaskSetManager: Finished task 109.0 in stage 14.0 (TID 200) in 4149 ms on 10.0.0.133 (executor driver) (110/208) -26/04/01 06:07:40 INFO Executor: Running task 113.0 in stage 14.0 (TID 204) -26/04/01 06:07:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:40 INFO Executor: Finished task 110.0 in stage 14.0 (TID 201). 6563 bytes result sent to driver -26/04/01 06:07:40 INFO TaskSetManager: Starting task 114.0 in stage 14.0 (TID 205) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:07:40 INFO Executor: Running task 114.0 in stage 14.0 (TID 205) -26/04/01 06:07:40 INFO TaskSetManager: Finished task 110.0 in stage 14.0 (TID 201) in 4157 ms on 10.0.0.133 (executor driver) (111/208) -26/04/01 06:07:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:41 INFO Executor: Finished task 111.0 in stage 14.0 (TID 202). 6606 bytes result sent to driver -26/04/01 06:07:41 INFO TaskSetManager: Starting task 115.0 in stage 14.0 (TID 206) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:41 INFO TaskSetManager: Finished task 111.0 in stage 14.0 (TID 202) in 4154 ms on 10.0.0.133 (executor driver) (112/208) -26/04/01 06:07:41 INFO Executor: Running task 115.0 in stage 14.0 (TID 206) -26/04/01 06:07:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:44 INFO Executor: Finished task 112.0 in stage 14.0 (TID 203). 6606 bytes result sent to driver -26/04/01 06:07:44 INFO TaskSetManager: Starting task 116.0 in stage 14.0 (TID 207) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:44 INFO TaskSetManager: Finished task 112.0 in stage 14.0 (TID 203) in 4147 ms on 10.0.0.133 (executor driver) (113/208) -26/04/01 06:07:44 INFO Executor: Running task 116.0 in stage 14.0 (TID 207) -26/04/01 06:07:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:44 INFO Executor: Finished task 113.0 in stage 14.0 (TID 204). 6606 bytes result sent to driver -26/04/01 06:07:44 INFO TaskSetManager: Starting task 117.0 in stage 14.0 (TID 208) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:44 INFO TaskSetManager: Finished task 113.0 in stage 14.0 (TID 204) in 4150 ms on 10.0.0.133 (executor driver) (114/208) -26/04/01 06:07:44 INFO Executor: Running task 117.0 in stage 14.0 (TID 208) -26/04/01 06:07:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:45 INFO Executor: Finished task 114.0 in stage 14.0 (TID 205). 6563 bytes result sent to driver -26/04/01 06:07:45 INFO TaskSetManager: Starting task 118.0 in stage 14.0 (TID 209) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:45 INFO TaskSetManager: Finished task 114.0 in stage 14.0 (TID 205) in 4153 ms on 10.0.0.133 (executor driver) (115/208) -26/04/01 06:07:45 INFO Executor: Running task 118.0 in stage 14.0 (TID 209) -26/04/01 06:07:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:45 INFO Executor: Finished task 115.0 in stage 14.0 (TID 206). 6563 bytes result sent to driver -26/04/01 06:07:45 INFO TaskSetManager: Starting task 119.0 in stage 14.0 (TID 210) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:45 INFO TaskSetManager: Finished task 115.0 in stage 14.0 (TID 206) in 4147 ms on 10.0.0.133 (executor driver) (116/208) -26/04/01 06:07:45 INFO Executor: Running task 119.0 in stage 14.0 (TID 210) -26/04/01 06:07:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:49 INFO Executor: Finished task 116.0 in stage 14.0 (TID 207). 6563 bytes result sent to driver -26/04/01 06:07:49 INFO TaskSetManager: Starting task 120.0 in stage 14.0 (TID 211) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:07:49 INFO TaskSetManager: Finished task 116.0 in stage 14.0 (TID 207) in 4158 ms on 10.0.0.133 (executor driver) (117/208) -26/04/01 06:07:49 INFO Executor: Running task 120.0 in stage 14.0 (TID 211) -26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:49 INFO Executor: Finished task 117.0 in stage 14.0 (TID 208). 6563 bytes result sent to driver -26/04/01 06:07:49 INFO TaskSetManager: Starting task 121.0 in stage 14.0 (TID 212) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:49 INFO TaskSetManager: Finished task 117.0 in stage 14.0 (TID 208) in 4152 ms on 10.0.0.133 (executor driver) (118/208) -26/04/01 06:07:49 INFO Executor: Running task 121.0 in stage 14.0 (TID 212) -26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:49 INFO Executor: Finished task 118.0 in stage 14.0 (TID 209). 6563 bytes result sent to driver -26/04/01 06:07:49 INFO TaskSetManager: Starting task 122.0 in stage 14.0 (TID 213) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:49 INFO TaskSetManager: Finished task 118.0 in stage 14.0 (TID 209) in 4153 ms on 10.0.0.133 (executor driver) (119/208) -26/04/01 06:07:49 INFO Executor: Running task 122.0 in stage 14.0 (TID 213) -26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:49 INFO Executor: Finished task 119.0 in stage 14.0 (TID 210). 6563 bytes result sent to driver -26/04/01 06:07:49 INFO TaskSetManager: Starting task 123.0 in stage 14.0 (TID 214) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:49 INFO TaskSetManager: Finished task 119.0 in stage 14.0 (TID 210) in 4157 ms on 10.0.0.133 (executor driver) (120/208) -26/04/01 06:07:49 INFO Executor: Running task 123.0 in stage 14.0 (TID 214) -26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:53 INFO Executor: Finished task 120.0 in stage 14.0 (TID 211). 6563 bytes result sent to driver -26/04/01 06:07:53 INFO Executor: Finished task 121.0 in stage 14.0 (TID 212). 6563 bytes result sent to driver -26/04/01 06:07:53 INFO TaskSetManager: Starting task 124.0 in stage 14.0 (TID 215) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:53 INFO TaskSetManager: Finished task 120.0 in stage 14.0 (TID 211) in 4155 ms on 10.0.0.133 (executor driver) (121/208) -26/04/01 06:07:53 INFO Executor: Running task 124.0 in stage 14.0 (TID 215) -26/04/01 06:07:53 INFO TaskSetManager: Starting task 125.0 in stage 14.0 (TID 216) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:53 INFO TaskSetManager: Finished task 121.0 in stage 14.0 (TID 212) in 4152 ms on 10.0.0.133 (executor driver) (122/208) -26/04/01 06:07:53 INFO Executor: Running task 125.0 in stage 14.0 (TID 216) -26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:53 INFO Executor: Finished task 122.0 in stage 14.0 (TID 213). 6563 bytes result sent to driver -26/04/01 06:07:53 INFO TaskSetManager: Starting task 126.0 in stage 14.0 (TID 217) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:07:53 INFO TaskSetManager: Finished task 122.0 in stage 14.0 (TID 213) in 4151 ms on 10.0.0.133 (executor driver) (123/208) -26/04/01 06:07:53 INFO Executor: Running task 126.0 in stage 14.0 (TID 217) -26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:53 INFO Executor: Finished task 123.0 in stage 14.0 (TID 214). 6563 bytes result sent to driver -26/04/01 06:07:53 INFO TaskSetManager: Starting task 127.0 in stage 14.0 (TID 218) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:07:53 INFO TaskSetManager: Finished task 123.0 in stage 14.0 (TID 214) in 4155 ms on 10.0.0.133 (executor driver) (124/208) -26/04/01 06:07:53 INFO Executor: Running task 127.0 in stage 14.0 (TID 218) -26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:57 INFO Executor: Finished task 124.0 in stage 14.0 (TID 215). 6563 bytes result sent to driver -26/04/01 06:07:57 INFO TaskSetManager: Starting task 128.0 in stage 14.0 (TID 219) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:57 INFO TaskSetManager: Finished task 124.0 in stage 14.0 (TID 215) in 4144 ms on 10.0.0.133 (executor driver) (125/208) -26/04/01 06:07:57 INFO Executor: Running task 128.0 in stage 14.0 (TID 219) -26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:57 INFO Executor: Finished task 125.0 in stage 14.0 (TID 216). 6563 bytes result sent to driver -26/04/01 06:07:57 INFO TaskSetManager: Starting task 129.0 in stage 14.0 (TID 220) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:57 INFO Executor: Running task 129.0 in stage 14.0 (TID 220) -26/04/01 06:07:57 INFO TaskSetManager: Finished task 125.0 in stage 14.0 (TID 216) in 4149 ms on 10.0.0.133 (executor driver) (126/208) -26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:57 INFO Executor: Finished task 126.0 in stage 14.0 (TID 217). 6563 bytes result sent to driver -26/04/01 06:07:57 INFO TaskSetManager: Starting task 130.0 in stage 14.0 (TID 221) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:57 INFO TaskSetManager: Finished task 126.0 in stage 14.0 (TID 217) in 4149 ms on 10.0.0.133 (executor driver) (127/208) -26/04/01 06:07:57 INFO Executor: Running task 130.0 in stage 14.0 (TID 221) -26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:57 INFO Executor: Finished task 127.0 in stage 14.0 (TID 218). 6563 bytes result sent to driver -26/04/01 06:07:57 INFO TaskSetManager: Starting task 131.0 in stage 14.0 (TID 222) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:07:57 INFO TaskSetManager: Finished task 127.0 in stage 14.0 (TID 218) in 4153 ms on 10.0.0.133 (executor driver) (128/208) -26/04/01 06:07:57 INFO Executor: Running task 131.0 in stage 14.0 (TID 222) -26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:07:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:01 INFO Executor: Finished task 129.0 in stage 14.0 (TID 220). 6563 bytes result sent to driver -26/04/01 06:08:01 INFO TaskSetManager: Starting task 132.0 in stage 14.0 (TID 223) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:08:01 INFO TaskSetManager: Finished task 129.0 in stage 14.0 (TID 220) in 4129 ms on 10.0.0.133 (executor driver) (129/208) -26/04/01 06:08:01 INFO Executor: Running task 132.0 in stage 14.0 (TID 223) -26/04/01 06:08:01 INFO Executor: Finished task 128.0 in stage 14.0 (TID 219). 6563 bytes result sent to driver -26/04/01 06:08:01 INFO TaskSetManager: Starting task 133.0 in stage 14.0 (TID 224) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:01 INFO TaskSetManager: Finished task 128.0 in stage 14.0 (TID 219) in 4135 ms on 10.0.0.133 (executor driver) (130/208) -26/04/01 06:08:01 INFO Executor: Running task 133.0 in stage 14.0 (TID 224) -26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:01 INFO Executor: Finished task 130.0 in stage 14.0 (TID 221). 6563 bytes result sent to driver -26/04/01 06:08:01 INFO TaskSetManager: Starting task 134.0 in stage 14.0 (TID 225) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:01 INFO TaskSetManager: Finished task 130.0 in stage 14.0 (TID 221) in 4132 ms on 10.0.0.133 (executor driver) (131/208) -26/04/01 06:08:01 INFO Executor: Running task 134.0 in stage 14.0 (TID 225) -26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:01 INFO Executor: Finished task 131.0 in stage 14.0 (TID 222). 6563 bytes result sent to driver -26/04/01 06:08:01 INFO TaskSetManager: Starting task 135.0 in stage 14.0 (TID 226) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:01 INFO TaskSetManager: Finished task 131.0 in stage 14.0 (TID 222) in 4130 ms on 10.0.0.133 (executor driver) (132/208) -26/04/01 06:08:01 INFO Executor: Running task 135.0 in stage 14.0 (TID 226) -26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:05 INFO Executor: Finished task 133.0 in stage 14.0 (TID 224). 6563 bytes result sent to driver -26/04/01 06:08:05 INFO TaskSetManager: Starting task 136.0 in stage 14.0 (TID 227) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:05 INFO TaskSetManager: Finished task 133.0 in stage 14.0 (TID 224) in 4082 ms on 10.0.0.133 (executor driver) (133/208) -26/04/01 06:08:05 INFO Executor: Running task 136.0 in stage 14.0 (TID 227) -26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:05 INFO Executor: Finished task 132.0 in stage 14.0 (TID 223). 6563 bytes result sent to driver -26/04/01 06:08:05 INFO TaskSetManager: Starting task 137.0 in stage 14.0 (TID 228) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:05 INFO TaskSetManager: Finished task 132.0 in stage 14.0 (TID 223) in 4086 ms on 10.0.0.133 (executor driver) (134/208) -26/04/01 06:08:05 INFO Executor: Running task 137.0 in stage 14.0 (TID 228) -26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:05 INFO Executor: Finished task 134.0 in stage 14.0 (TID 225). 6563 bytes result sent to driver -26/04/01 06:08:05 INFO TaskSetManager: Starting task 138.0 in stage 14.0 (TID 229) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:08:05 INFO TaskSetManager: Finished task 134.0 in stage 14.0 (TID 225) in 4086 ms on 10.0.0.133 (executor driver) (135/208) -26/04/01 06:08:05 INFO Executor: Running task 138.0 in stage 14.0 (TID 229) -26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:05 INFO Executor: Finished task 135.0 in stage 14.0 (TID 226). 6563 bytes result sent to driver -26/04/01 06:08:05 INFO TaskSetManager: Starting task 139.0 in stage 14.0 (TID 230) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:05 INFO TaskSetManager: Finished task 135.0 in stage 14.0 (TID 226) in 4086 ms on 10.0.0.133 (executor driver) (136/208) -26/04/01 06:08:05 INFO Executor: Running task 139.0 in stage 14.0 (TID 230) -26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:09 INFO Executor: Finished task 136.0 in stage 14.0 (TID 227). 6563 bytes result sent to driver -26/04/01 06:08:09 INFO TaskSetManager: Starting task 140.0 in stage 14.0 (TID 231) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:09 INFO TaskSetManager: Finished task 136.0 in stage 14.0 (TID 227) in 4086 ms on 10.0.0.133 (executor driver) (137/208) -26/04/01 06:08:09 INFO Executor: Running task 140.0 in stage 14.0 (TID 231) -26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:09 INFO Executor: Finished task 137.0 in stage 14.0 (TID 228). 6563 bytes result sent to driver -26/04/01 06:08:09 INFO TaskSetManager: Starting task 141.0 in stage 14.0 (TID 232) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:09 INFO TaskSetManager: Finished task 137.0 in stage 14.0 (TID 228) in 4188 ms on 10.0.0.133 (executor driver) (138/208) -26/04/01 06:08:09 INFO Executor: Running task 141.0 in stage 14.0 (TID 232) -26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:09 INFO Executor: Finished task 138.0 in stage 14.0 (TID 229). 6563 bytes result sent to driver -26/04/01 06:08:09 INFO TaskSetManager: Starting task 142.0 in stage 14.0 (TID 233) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:09 INFO TaskSetManager: Finished task 138.0 in stage 14.0 (TID 229) in 4084 ms on 10.0.0.133 (executor driver) (139/208) -26/04/01 06:08:09 INFO Executor: Running task 142.0 in stage 14.0 (TID 233) -26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:09 INFO Executor: Finished task 139.0 in stage 14.0 (TID 230). 6563 bytes result sent to driver -26/04/01 06:08:09 INFO TaskSetManager: Starting task 143.0 in stage 14.0 (TID 234) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:09 INFO Executor: Running task 143.0 in stage 14.0 (TID 234) -26/04/01 06:08:09 INFO TaskSetManager: Finished task 139.0 in stage 14.0 (TID 230) in 4080 ms on 10.0.0.133 (executor driver) (140/208) -26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:13 INFO Executor: Finished task 140.0 in stage 14.0 (TID 231). 6563 bytes result sent to driver -26/04/01 06:08:13 INFO TaskSetManager: Starting task 144.0 in stage 14.0 (TID 235) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:08:13 INFO TaskSetManager: Finished task 140.0 in stage 14.0 (TID 231) in 4126 ms on 10.0.0.133 (executor driver) (141/208) -26/04/01 06:08:13 INFO Executor: Running task 144.0 in stage 14.0 (TID 235) -26/04/01 06:08:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:13 INFO Executor: Finished task 141.0 in stage 14.0 (TID 232). 6563 bytes result sent to driver -26/04/01 06:08:13 INFO TaskSetManager: Starting task 145.0 in stage 14.0 (TID 236) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:08:13 INFO TaskSetManager: Finished task 141.0 in stage 14.0 (TID 232) in 4100 ms on 10.0.0.133 (executor driver) (142/208) -26/04/01 06:08:13 INFO Executor: Running task 145.0 in stage 14.0 (TID 236) -26/04/01 06:08:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:13 INFO Executor: Finished task 142.0 in stage 14.0 (TID 233). 6563 bytes result sent to driver -26/04/01 06:08:13 INFO TaskSetManager: Starting task 146.0 in stage 14.0 (TID 237) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:13 INFO Executor: Running task 146.0 in stage 14.0 (TID 237) -26/04/01 06:08:13 INFO TaskSetManager: Finished task 142.0 in stage 14.0 (TID 233) in 4087 ms on 10.0.0.133 (executor driver) (143/208) -26/04/01 06:08:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:14 INFO Executor: Finished task 143.0 in stage 14.0 (TID 234). 6563 bytes result sent to driver -26/04/01 06:08:14 INFO TaskSetManager: Starting task 147.0 in stage 14.0 (TID 238) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:14 INFO TaskSetManager: Finished task 143.0 in stage 14.0 (TID 234) in 4095 ms on 10.0.0.133 (executor driver) (144/208) -26/04/01 06:08:14 INFO Executor: Running task 147.0 in stage 14.0 (TID 238) -26/04/01 06:08:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:17 INFO Executor: Finished task 144.0 in stage 14.0 (TID 235). 6563 bytes result sent to driver -26/04/01 06:08:17 INFO TaskSetManager: Starting task 148.0 in stage 14.0 (TID 239) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:17 INFO TaskSetManager: Finished task 144.0 in stage 14.0 (TID 235) in 4140 ms on 10.0.0.133 (executor driver) (145/208) -26/04/01 06:08:17 INFO Executor: Running task 148.0 in stage 14.0 (TID 239) -26/04/01 06:08:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:18 INFO Executor: Finished task 145.0 in stage 14.0 (TID 236). 6563 bytes result sent to driver -26/04/01 06:08:18 INFO TaskSetManager: Starting task 149.0 in stage 14.0 (TID 240) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:18 INFO TaskSetManager: Finished task 145.0 in stage 14.0 (TID 236) in 4132 ms on 10.0.0.133 (executor driver) (146/208) -26/04/01 06:08:18 INFO Executor: Running task 149.0 in stage 14.0 (TID 240) -26/04/01 06:08:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:18 INFO Executor: Finished task 146.0 in stage 14.0 (TID 237). 6563 bytes result sent to driver -26/04/01 06:08:18 INFO TaskSetManager: Starting task 150.0 in stage 14.0 (TID 241) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:08:18 INFO Executor: Running task 150.0 in stage 14.0 (TID 241) -26/04/01 06:08:18 INFO TaskSetManager: Finished task 146.0 in stage 14.0 (TID 237) in 4125 ms on 10.0.0.133 (executor driver) (147/208) -26/04/01 06:08:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:18 INFO Executor: Finished task 147.0 in stage 14.0 (TID 238). 6563 bytes result sent to driver -26/04/01 06:08:18 INFO TaskSetManager: Starting task 151.0 in stage 14.0 (TID 242) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:18 INFO TaskSetManager: Finished task 147.0 in stage 14.0 (TID 238) in 4125 ms on 10.0.0.133 (executor driver) (148/208) -26/04/01 06:08:18 INFO Executor: Running task 151.0 in stage 14.0 (TID 242) -26/04/01 06:08:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:22 INFO Executor: Finished task 148.0 in stage 14.0 (TID 239). 6563 bytes result sent to driver -26/04/01 06:08:22 INFO TaskSetManager: Starting task 152.0 in stage 14.0 (TID 243) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:22 INFO TaskSetManager: Finished task 148.0 in stage 14.0 (TID 239) in 4157 ms on 10.0.0.133 (executor driver) (149/208) -26/04/01 06:08:22 INFO Executor: Running task 152.0 in stage 14.0 (TID 243) -26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:22 INFO Executor: Finished task 149.0 in stage 14.0 (TID 240). 6563 bytes result sent to driver -26/04/01 06:08:22 INFO TaskSetManager: Starting task 153.0 in stage 14.0 (TID 244) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:22 INFO Executor: Running task 153.0 in stage 14.0 (TID 244) -26/04/01 06:08:22 INFO TaskSetManager: Finished task 149.0 in stage 14.0 (TID 240) in 4140 ms on 10.0.0.133 (executor driver) (150/208) -26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:22 INFO Executor: Finished task 150.0 in stage 14.0 (TID 241). 6563 bytes result sent to driver -26/04/01 06:08:22 INFO TaskSetManager: Starting task 154.0 in stage 14.0 (TID 245) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:22 INFO TaskSetManager: Finished task 150.0 in stage 14.0 (TID 241) in 4160 ms on 10.0.0.133 (executor driver) (151/208) -26/04/01 06:08:22 INFO Executor: Running task 154.0 in stage 14.0 (TID 245) -26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:22 INFO Executor: Finished task 151.0 in stage 14.0 (TID 242). 6563 bytes result sent to driver -26/04/01 06:08:22 INFO TaskSetManager: Starting task 155.0 in stage 14.0 (TID 246) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:22 INFO Executor: Running task 155.0 in stage 14.0 (TID 246) -26/04/01 06:08:22 INFO TaskSetManager: Finished task 151.0 in stage 14.0 (TID 242) in 4147 ms on 10.0.0.133 (executor driver) (152/208) -26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:26 INFO Executor: Finished task 152.0 in stage 14.0 (TID 243). 6606 bytes result sent to driver -26/04/01 06:08:26 INFO TaskSetManager: Starting task 156.0 in stage 14.0 (TID 247) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:08:26 INFO TaskSetManager: Finished task 152.0 in stage 14.0 (TID 243) in 4136 ms on 10.0.0.133 (executor driver) (153/208) -26/04/01 06:08:26 INFO Executor: Running task 156.0 in stage 14.0 (TID 247) -26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:26 INFO Executor: Finished task 153.0 in stage 14.0 (TID 244). 6606 bytes result sent to driver -26/04/01 06:08:26 INFO TaskSetManager: Starting task 157.0 in stage 14.0 (TID 248) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:26 INFO TaskSetManager: Finished task 153.0 in stage 14.0 (TID 244) in 4143 ms on 10.0.0.133 (executor driver) (154/208) -26/04/01 06:08:26 INFO Executor: Running task 157.0 in stage 14.0 (TID 248) -26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:26 INFO Executor: Finished task 154.0 in stage 14.0 (TID 245). 6606 bytes result sent to driver -26/04/01 06:08:26 INFO TaskSetManager: Starting task 158.0 in stage 14.0 (TID 249) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:26 INFO Executor: Running task 158.0 in stage 14.0 (TID 249) -26/04/01 06:08:26 INFO TaskSetManager: Finished task 154.0 in stage 14.0 (TID 245) in 4143 ms on 10.0.0.133 (executor driver) (155/208) -26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:26 INFO Executor: Finished task 155.0 in stage 14.0 (TID 246). 6606 bytes result sent to driver -26/04/01 06:08:26 INFO TaskSetManager: Starting task 159.0 in stage 14.0 (TID 250) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:26 INFO TaskSetManager: Finished task 155.0 in stage 14.0 (TID 246) in 4140 ms on 10.0.0.133 (executor driver) (156/208) -26/04/01 06:08:26 INFO Executor: Running task 159.0 in stage 14.0 (TID 250) -26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:30 INFO Executor: Finished task 156.0 in stage 14.0 (TID 247). 6563 bytes result sent to driver -26/04/01 06:08:30 INFO TaskSetManager: Starting task 160.0 in stage 14.0 (TID 251) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:30 INFO TaskSetManager: Finished task 156.0 in stage 14.0 (TID 247) in 4109 ms on 10.0.0.133 (executor driver) (157/208) -26/04/01 06:08:30 INFO Executor: Running task 160.0 in stage 14.0 (TID 251) -26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:30 INFO Executor: Finished task 157.0 in stage 14.0 (TID 248). 6563 bytes result sent to driver -26/04/01 06:08:30 INFO TaskSetManager: Starting task 161.0 in stage 14.0 (TID 252) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:30 INFO TaskSetManager: Finished task 157.0 in stage 14.0 (TID 248) in 4116 ms on 10.0.0.133 (executor driver) (158/208) -26/04/01 06:08:30 INFO Executor: Running task 161.0 in stage 14.0 (TID 252) -26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:30 INFO Executor: Finished task 158.0 in stage 14.0 (TID 249). 6563 bytes result sent to driver -26/04/01 06:08:30 INFO TaskSetManager: Starting task 162.0 in stage 14.0 (TID 253) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:08:30 INFO TaskSetManager: Finished task 158.0 in stage 14.0 (TID 249) in 4106 ms on 10.0.0.133 (executor driver) (159/208) -26/04/01 06:08:30 INFO Executor: Running task 162.0 in stage 14.0 (TID 253) -26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:30 INFO Executor: Finished task 159.0 in stage 14.0 (TID 250). 6563 bytes result sent to driver -26/04/01 06:08:30 INFO TaskSetManager: Starting task 163.0 in stage 14.0 (TID 254) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:08:30 INFO TaskSetManager: Finished task 159.0 in stage 14.0 (TID 250) in 4118 ms on 10.0.0.133 (executor driver) (160/208) -26/04/01 06:08:30 INFO Executor: Running task 163.0 in stage 14.0 (TID 254) -26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:34 INFO Executor: Finished task 160.0 in stage 14.0 (TID 251). 6563 bytes result sent to driver -26/04/01 06:08:34 INFO TaskSetManager: Starting task 164.0 in stage 14.0 (TID 255) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:34 INFO TaskSetManager: Finished task 160.0 in stage 14.0 (TID 251) in 4102 ms on 10.0.0.133 (executor driver) (161/208) -26/04/01 06:08:34 INFO Executor: Running task 164.0 in stage 14.0 (TID 255) -26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:34 INFO Executor: Finished task 161.0 in stage 14.0 (TID 252). 6563 bytes result sent to driver -26/04/01 06:08:34 INFO TaskSetManager: Starting task 165.0 in stage 14.0 (TID 256) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:34 INFO TaskSetManager: Finished task 161.0 in stage 14.0 (TID 252) in 4091 ms on 10.0.0.133 (executor driver) (162/208) -26/04/01 06:08:34 INFO Executor: Running task 165.0 in stage 14.0 (TID 256) -26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:34 INFO Executor: Finished task 162.0 in stage 14.0 (TID 253). 6563 bytes result sent to driver -26/04/01 06:08:34 INFO TaskSetManager: Starting task 166.0 in stage 14.0 (TID 257) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:34 INFO TaskSetManager: Finished task 162.0 in stage 14.0 (TID 253) in 4097 ms on 10.0.0.133 (executor driver) (163/208) -26/04/01 06:08:34 INFO Executor: Running task 166.0 in stage 14.0 (TID 257) -26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:34 INFO Executor: Finished task 163.0 in stage 14.0 (TID 254). 6563 bytes result sent to driver -26/04/01 06:08:34 INFO TaskSetManager: Starting task 167.0 in stage 14.0 (TID 258) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:34 INFO TaskSetManager: Finished task 163.0 in stage 14.0 (TID 254) in 4103 ms on 10.0.0.133 (executor driver) (164/208) -26/04/01 06:08:34 INFO Executor: Running task 167.0 in stage 14.0 (TID 258) -26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:38 INFO Executor: Finished task 164.0 in stage 14.0 (TID 255). 6563 bytes result sent to driver -26/04/01 06:08:38 INFO TaskSetManager: Starting task 168.0 in stage 14.0 (TID 259) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:08:38 INFO TaskSetManager: Finished task 164.0 in stage 14.0 (TID 255) in 4099 ms on 10.0.0.133 (executor driver) (165/208) -26/04/01 06:08:38 INFO Executor: Running task 168.0 in stage 14.0 (TID 259) -26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:38 INFO Executor: Finished task 165.0 in stage 14.0 (TID 256). 6563 bytes result sent to driver -26/04/01 06:08:38 INFO TaskSetManager: Starting task 169.0 in stage 14.0 (TID 260) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:38 INFO TaskSetManager: Finished task 165.0 in stage 14.0 (TID 256) in 4101 ms on 10.0.0.133 (executor driver) (166/208) -26/04/01 06:08:38 INFO Executor: Running task 169.0 in stage 14.0 (TID 260) -26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:38 INFO Executor: Finished task 166.0 in stage 14.0 (TID 257). 6563 bytes result sent to driver -26/04/01 06:08:38 INFO TaskSetManager: Starting task 170.0 in stage 14.0 (TID 261) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:38 INFO TaskSetManager: Finished task 166.0 in stage 14.0 (TID 257) in 4112 ms on 10.0.0.133 (executor driver) (167/208) -26/04/01 06:08:38 INFO Executor: Running task 170.0 in stage 14.0 (TID 261) -26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:38 INFO Executor: Finished task 167.0 in stage 14.0 (TID 258). 6563 bytes result sent to driver -26/04/01 06:08:38 INFO TaskSetManager: Starting task 171.0 in stage 14.0 (TID 262) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:38 INFO TaskSetManager: Finished task 167.0 in stage 14.0 (TID 258) in 4103 ms on 10.0.0.133 (executor driver) (168/208) -26/04/01 06:08:38 INFO Executor: Running task 171.0 in stage 14.0 (TID 262) -26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:42 INFO Executor: Finished task 168.0 in stage 14.0 (TID 259). 6563 bytes result sent to driver -26/04/01 06:08:42 INFO TaskSetManager: Starting task 172.0 in stage 14.0 (TID 263) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:42 INFO TaskSetManager: Finished task 168.0 in stage 14.0 (TID 259) in 4116 ms on 10.0.0.133 (executor driver) (169/208) -26/04/01 06:08:42 INFO Executor: Running task 172.0 in stage 14.0 (TID 263) -26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:42 INFO Executor: Finished task 169.0 in stage 14.0 (TID 260). 6563 bytes result sent to driver -26/04/01 06:08:42 INFO TaskSetManager: Starting task 173.0 in stage 14.0 (TID 264) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:42 INFO TaskSetManager: Finished task 169.0 in stage 14.0 (TID 260) in 4101 ms on 10.0.0.133 (executor driver) (170/208) -26/04/01 06:08:42 INFO Executor: Running task 173.0 in stage 14.0 (TID 264) -26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:42 INFO Executor: Finished task 170.0 in stage 14.0 (TID 261). 6563 bytes result sent to driver -26/04/01 06:08:42 INFO TaskSetManager: Starting task 174.0 in stage 14.0 (TID 265) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:08:42 INFO Executor: Running task 174.0 in stage 14.0 (TID 265) -26/04/01 06:08:42 INFO TaskSetManager: Finished task 170.0 in stage 14.0 (TID 261) in 4110 ms on 10.0.0.133 (executor driver) (171/208) -26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:42 INFO Executor: Finished task 171.0 in stage 14.0 (TID 262). 6563 bytes result sent to driver -26/04/01 06:08:42 INFO TaskSetManager: Starting task 175.0 in stage 14.0 (TID 266) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:08:42 INFO TaskSetManager: Finished task 171.0 in stage 14.0 (TID 262) in 4104 ms on 10.0.0.133 (executor driver) (172/208) -26/04/01 06:08:42 INFO Executor: Running task 175.0 in stage 14.0 (TID 266) -26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:46 INFO Executor: Finished task 172.0 in stage 14.0 (TID 263). 6563 bytes result sent to driver -26/04/01 06:08:46 INFO TaskSetManager: Starting task 176.0 in stage 14.0 (TID 267) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:46 INFO Executor: Running task 176.0 in stage 14.0 (TID 267) -26/04/01 06:08:46 INFO TaskSetManager: Finished task 172.0 in stage 14.0 (TID 263) in 4102 ms on 10.0.0.133 (executor driver) (173/208) -26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:46 INFO Executor: Finished task 173.0 in stage 14.0 (TID 264). 6563 bytes result sent to driver -26/04/01 06:08:46 INFO TaskSetManager: Starting task 177.0 in stage 14.0 (TID 268) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:46 INFO Executor: Running task 177.0 in stage 14.0 (TID 268) -26/04/01 06:08:46 INFO TaskSetManager: Finished task 173.0 in stage 14.0 (TID 264) in 4091 ms on 10.0.0.133 (executor driver) (174/208) -26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:46 INFO Executor: Finished task 174.0 in stage 14.0 (TID 265). 6563 bytes result sent to driver -26/04/01 06:08:46 INFO TaskSetManager: Starting task 178.0 in stage 14.0 (TID 269) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:46 INFO Executor: Running task 178.0 in stage 14.0 (TID 269) -26/04/01 06:08:46 INFO TaskSetManager: Finished task 174.0 in stage 14.0 (TID 265) in 4146 ms on 10.0.0.133 (executor driver) (175/208) -26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:46 INFO Executor: Finished task 175.0 in stage 14.0 (TID 266). 6563 bytes result sent to driver -26/04/01 06:08:46 INFO TaskSetManager: Starting task 179.0 in stage 14.0 (TID 270) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:46 INFO Executor: Running task 179.0 in stage 14.0 (TID 270) -26/04/01 06:08:46 INFO TaskSetManager: Finished task 175.0 in stage 14.0 (TID 266) in 4143 ms on 10.0.0.133 (executor driver) (176/208) -26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:50 INFO Executor: Finished task 176.0 in stage 14.0 (TID 267). 6563 bytes result sent to driver -26/04/01 06:08:50 INFO TaskSetManager: Starting task 180.0 in stage 14.0 (TID 271) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:08:50 INFO Executor: Running task 180.0 in stage 14.0 (TID 271) -26/04/01 06:08:50 INFO TaskSetManager: Finished task 176.0 in stage 14.0 (TID 267) in 4158 ms on 10.0.0.133 (executor driver) (177/208) -26/04/01 06:08:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:50 INFO Executor: Finished task 177.0 in stage 14.0 (TID 268). 6563 bytes result sent to driver -26/04/01 06:08:50 INFO TaskSetManager: Starting task 181.0 in stage 14.0 (TID 272) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:50 INFO TaskSetManager: Finished task 177.0 in stage 14.0 (TID 268) in 4148 ms on 10.0.0.133 (executor driver) (178/208) -26/04/01 06:08:50 INFO Executor: Running task 181.0 in stage 14.0 (TID 272) -26/04/01 06:08:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:51 INFO Executor: Finished task 178.0 in stage 14.0 (TID 269). 6563 bytes result sent to driver -26/04/01 06:08:51 INFO TaskSetManager: Starting task 182.0 in stage 14.0 (TID 273) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:51 INFO TaskSetManager: Finished task 178.0 in stage 14.0 (TID 269) in 4151 ms on 10.0.0.133 (executor driver) (179/208) -26/04/01 06:08:51 INFO Executor: Running task 182.0 in stage 14.0 (TID 273) -26/04/01 06:08:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:51 INFO Executor: Finished task 179.0 in stage 14.0 (TID 270). 6563 bytes result sent to driver -26/04/01 06:08:51 INFO TaskSetManager: Starting task 183.0 in stage 14.0 (TID 274) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:51 INFO TaskSetManager: Finished task 179.0 in stage 14.0 (TID 270) in 4147 ms on 10.0.0.133 (executor driver) (180/208) -26/04/01 06:08:51 INFO Executor: Running task 183.0 in stage 14.0 (TID 274) -26/04/01 06:08:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:55 INFO Executor: Finished task 180.0 in stage 14.0 (TID 271). 6563 bytes result sent to driver -26/04/01 06:08:55 INFO TaskSetManager: Starting task 184.0 in stage 14.0 (TID 275) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:55 INFO TaskSetManager: Finished task 180.0 in stage 14.0 (TID 271) in 4147 ms on 10.0.0.133 (executor driver) (181/208) -26/04/01 06:08:55 INFO Executor: Running task 184.0 in stage 14.0 (TID 275) -26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:55 INFO Executor: Finished task 181.0 in stage 14.0 (TID 272). 6563 bytes result sent to driver -26/04/01 06:08:55 INFO TaskSetManager: Starting task 185.0 in stage 14.0 (TID 276) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:55 INFO TaskSetManager: Finished task 181.0 in stage 14.0 (TID 272) in 4138 ms on 10.0.0.133 (executor driver) (182/208) -26/04/01 06:08:55 INFO Executor: Running task 185.0 in stage 14.0 (TID 276) -26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:55 INFO Executor: Finished task 182.0 in stage 14.0 (TID 273). 6563 bytes result sent to driver -26/04/01 06:08:55 INFO TaskSetManager: Starting task 186.0 in stage 14.0 (TID 277) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:08:55 INFO Executor: Running task 186.0 in stage 14.0 (TID 277) -26/04/01 06:08:55 INFO TaskSetManager: Finished task 182.0 in stage 14.0 (TID 273) in 4137 ms on 10.0.0.133 (executor driver) (183/208) -26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:55 INFO Executor: Finished task 183.0 in stage 14.0 (TID 274). 6563 bytes result sent to driver -26/04/01 06:08:55 INFO TaskSetManager: Starting task 187.0 in stage 14.0 (TID 278) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:08:55 INFO TaskSetManager: Finished task 183.0 in stage 14.0 (TID 274) in 4143 ms on 10.0.0.133 (executor driver) (184/208) -26/04/01 06:08:55 INFO Executor: Running task 187.0 in stage 14.0 (TID 278) -26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:59 INFO Executor: Finished task 184.0 in stage 14.0 (TID 275). 6563 bytes result sent to driver -26/04/01 06:08:59 INFO TaskSetManager: Starting task 188.0 in stage 14.0 (TID 279) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:59 INFO TaskSetManager: Finished task 184.0 in stage 14.0 (TID 275) in 4155 ms on 10.0.0.133 (executor driver) (185/208) -26/04/01 06:08:59 INFO Executor: Running task 188.0 in stage 14.0 (TID 279) -26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:59 INFO Executor: Finished task 185.0 in stage 14.0 (TID 276). 6563 bytes result sent to driver -26/04/01 06:08:59 INFO TaskSetManager: Starting task 189.0 in stage 14.0 (TID 280) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:59 INFO TaskSetManager: Finished task 185.0 in stage 14.0 (TID 276) in 4142 ms on 10.0.0.133 (executor driver) (186/208) -26/04/01 06:08:59 INFO Executor: Running task 189.0 in stage 14.0 (TID 280) -26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:59 INFO Executor: Finished task 186.0 in stage 14.0 (TID 277). 6563 bytes result sent to driver -26/04/01 06:08:59 INFO TaskSetManager: Starting task 190.0 in stage 14.0 (TID 281) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:59 INFO TaskSetManager: Finished task 186.0 in stage 14.0 (TID 277) in 4117 ms on 10.0.0.133 (executor driver) (187/208) -26/04/01 06:08:59 INFO Executor: Running task 190.0 in stage 14.0 (TID 281) -26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:59 INFO Executor: Finished task 187.0 in stage 14.0 (TID 278). 6563 bytes result sent to driver -26/04/01 06:08:59 INFO TaskSetManager: Starting task 191.0 in stage 14.0 (TID 282) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:08:59 INFO TaskSetManager: Finished task 187.0 in stage 14.0 (TID 278) in 4117 ms on 10.0.0.133 (executor driver) (188/208) -26/04/01 06:08:59 INFO Executor: Running task 191.0 in stage 14.0 (TID 282) -26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:08:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:03 INFO Executor: Finished task 188.0 in stage 14.0 (TID 279). 6563 bytes result sent to driver -26/04/01 06:09:03 INFO TaskSetManager: Starting task 192.0 in stage 14.0 (TID 283) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:09:03 INFO TaskSetManager: Finished task 188.0 in stage 14.0 (TID 279) in 4108 ms on 10.0.0.133 (executor driver) (189/208) -26/04/01 06:09:03 INFO Executor: Running task 192.0 in stage 14.0 (TID 283) -26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:03 INFO Executor: Finished task 189.0 in stage 14.0 (TID 280). 6563 bytes result sent to driver -26/04/01 06:09:03 INFO TaskSetManager: Starting task 193.0 in stage 14.0 (TID 284) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:09:03 INFO TaskSetManager: Finished task 189.0 in stage 14.0 (TID 280) in 4112 ms on 10.0.0.133 (executor driver) (190/208) -26/04/01 06:09:03 INFO Executor: Running task 193.0 in stage 14.0 (TID 284) -26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:03 INFO Executor: Finished task 190.0 in stage 14.0 (TID 281). 6563 bytes result sent to driver -26/04/01 06:09:03 INFO TaskSetManager: Starting task 194.0 in stage 14.0 (TID 285) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:09:03 INFO TaskSetManager: Finished task 190.0 in stage 14.0 (TID 281) in 4105 ms on 10.0.0.133 (executor driver) (191/208) -26/04/01 06:09:03 INFO Executor: Running task 194.0 in stage 14.0 (TID 285) -26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:03 INFO Executor: Finished task 191.0 in stage 14.0 (TID 282). 6563 bytes result sent to driver -26/04/01 06:09:03 INFO TaskSetManager: Starting task 195.0 in stage 14.0 (TID 286) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:09:03 INFO Executor: Running task 195.0 in stage 14.0 (TID 286) -26/04/01 06:09:03 INFO TaskSetManager: Finished task 191.0 in stage 14.0 (TID 282) in 4105 ms on 10.0.0.133 (executor driver) (192/208) -26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:05 INFO Executor: Finished task 192.0 in stage 14.0 (TID 283). 6563 bytes result sent to driver -26/04/01 06:09:05 INFO TaskSetManager: Starting task 196.0 in stage 14.0 (TID 287) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:09:05 INFO TaskSetManager: Finished task 192.0 in stage 14.0 (TID 283) in 2584 ms on 10.0.0.133 (executor driver) (193/208) -26/04/01 06:09:05 INFO Executor: Running task 196.0 in stage 14.0 (TID 287) -26/04/01 06:09:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:05 INFO Executor: Finished task 193.0 in stage 14.0 (TID 284). 6563 bytes result sent to driver -26/04/01 06:09:05 INFO TaskSetManager: Starting task 197.0 in stage 14.0 (TID 288) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:09:05 INFO TaskSetManager: Finished task 193.0 in stage 14.0 (TID 284) in 2580 ms on 10.0.0.133 (executor driver) (194/208) -26/04/01 06:09:05 INFO Executor: Running task 197.0 in stage 14.0 (TID 288) -26/04/01 06:09:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:05 INFO Executor: Finished task 194.0 in stage 14.0 (TID 285). 6563 bytes result sent to driver -26/04/01 06:09:05 INFO TaskSetManager: Starting task 198.0 in stage 14.0 (TID 289) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:09:05 INFO TaskSetManager: Finished task 194.0 in stage 14.0 (TID 285) in 2585 ms on 10.0.0.133 (executor driver) (195/208) -26/04/01 06:09:05 INFO Executor: Running task 198.0 in stage 14.0 (TID 289) -26/04/01 06:09:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:06 INFO Executor: Finished task 195.0 in stage 14.0 (TID 286). 6563 bytes result sent to driver -26/04/01 06:09:06 INFO TaskSetManager: Starting task 199.0 in stage 14.0 (TID 290) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:09:06 INFO TaskSetManager: Finished task 195.0 in stage 14.0 (TID 286) in 2585 ms on 10.0.0.133 (executor driver) (196/208) -26/04/01 06:09:06 INFO Executor: Running task 199.0 in stage 14.0 (TID 290) -26/04/01 06:09:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:08 INFO Executor: Finished task 197.0 in stage 14.0 (TID 288). 6606 bytes result sent to driver -26/04/01 06:09:08 INFO TaskSetManager: Starting task 200.0 in stage 14.0 (TID 291) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9346 bytes) -26/04/01 06:09:08 INFO TaskSetManager: Finished task 197.0 in stage 14.0 (TID 288) in 2569 ms on 10.0.0.133 (executor driver) (197/208) -26/04/01 06:09:08 INFO Executor: Running task 200.0 in stage 14.0 (TID 291) -26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:08 INFO Executor: Finished task 196.0 in stage 14.0 (TID 287). 6606 bytes result sent to driver -26/04/01 06:09:08 INFO TaskSetManager: Starting task 201.0 in stage 14.0 (TID 292) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:09:08 INFO TaskSetManager: Finished task 196.0 in stage 14.0 (TID 287) in 2577 ms on 10.0.0.133 (executor driver) (198/208) -26/04/01 06:09:08 INFO Executor: Running task 201.0 in stage 14.0 (TID 292) -26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:08 INFO Executor: Finished task 198.0 in stage 14.0 (TID 289). 6606 bytes result sent to driver -26/04/01 06:09:08 INFO TaskSetManager: Starting task 202.0 in stage 14.0 (TID 293) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:09:08 INFO TaskSetManager: Finished task 198.0 in stage 14.0 (TID 289) in 2576 ms on 10.0.0.133 (executor driver) (199/208) -26/04/01 06:09:08 INFO Executor: Running task 202.0 in stage 14.0 (TID 293) -26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:08 INFO Executor: Finished task 199.0 in stage 14.0 (TID 290). 6606 bytes result sent to driver -26/04/01 06:09:08 INFO TaskSetManager: Starting task 203.0 in stage 14.0 (TID 294) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:09:08 INFO TaskSetManager: Finished task 199.0 in stage 14.0 (TID 290) in 2578 ms on 10.0.0.133 (executor driver) (200/208) -26/04/01 06:09:08 INFO Executor: Running task 203.0 in stage 14.0 (TID 294) -26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:11 INFO Executor: Finished task 200.0 in stage 14.0 (TID 291). 6563 bytes result sent to driver -26/04/01 06:09:11 INFO Executor: Finished task 201.0 in stage 14.0 (TID 292). 6563 bytes result sent to driver -26/04/01 06:09:11 INFO TaskSetManager: Starting task 204.0 in stage 14.0 (TID 295) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:09:11 INFO TaskSetManager: Starting task 205.0 in stage 14.0 (TID 296) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:09:11 INFO Executor: Running task 204.0 in stage 14.0 (TID 295) -26/04/01 06:09:11 INFO TaskSetManager: Finished task 200.0 in stage 14.0 (TID 291) in 2582 ms on 10.0.0.133 (executor driver) (201/208) -26/04/01 06:09:11 INFO Executor: Running task 205.0 in stage 14.0 (TID 296) -26/04/01 06:09:11 INFO TaskSetManager: Finished task 201.0 in stage 14.0 (TID 292) in 2576 ms on 10.0.0.133 (executor driver) (202/208) -26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:11 INFO Executor: Finished task 202.0 in stage 14.0 (TID 293). 6563 bytes result sent to driver -26/04/01 06:09:11 INFO TaskSetManager: Starting task 206.0 in stage 14.0 (TID 297) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:09:11 INFO TaskSetManager: Finished task 202.0 in stage 14.0 (TID 293) in 2561 ms on 10.0.0.133 (executor driver) (203/208) -26/04/01 06:09:11 INFO Executor: Running task 206.0 in stage 14.0 (TID 297) -26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:11 INFO Executor: Finished task 203.0 in stage 14.0 (TID 294). 6563 bytes result sent to driver -26/04/01 06:09:11 INFO TaskSetManager: Starting task 207.0 in stage 14.0 (TID 298) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:09:11 INFO TaskSetManager: Finished task 203.0 in stage 14.0 (TID 294) in 2563 ms on 10.0.0.133 (executor driver) (204/208) -26/04/01 06:09:11 INFO Executor: Running task 207.0 in stage 14.0 (TID 298) -26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO Executor: Finished task 205.0 in stage 14.0 (TID 296). 6563 bytes result sent to driver -26/04/01 06:09:13 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 299) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:09:13 INFO TaskSetManager: Finished task 205.0 in stage 14.0 (TID 296) in 2554 ms on 10.0.0.133 (executor driver) (205/208) -26/04/01 06:09:13 INFO Executor: Running task 0.0 in stage 15.0 (TID 299) -26/04/01 06:09:13 INFO Executor: Finished task 204.0 in stage 14.0 (TID 295). 6563 bytes result sent to driver -26/04/01 06:09:13 INFO TaskSetManager: Starting task 0.0 in stage 16.0 (TID 300) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9722 bytes) -26/04/01 06:09:13 INFO Executor: Running task 0.0 in stage 16.0 (TID 300) -26/04/01 06:09:13 INFO TaskSetManager: Finished task 204.0 in stage 14.0 (TID 295) in 2555 ms on 10.0.0.133 (executor driver) (206/208) -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO Executor: Finished task 0.0 in stage 15.0 (TID 299). 11159 bytes result sent to driver -26/04/01 06:09:13 INFO TaskSetManager: Starting task 1.0 in stage 16.0 (TID 301) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9724 bytes) -26/04/01 06:09:13 INFO Executor: Running task 1.0 in stage 16.0 (TID 301) -26/04/01 06:09:13 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 299) in 14 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:09:13 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool -26/04/01 06:09:13 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 228.857 s -26/04/01 06:09:13 INFO DAGScheduler: Job 14 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:09:13 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished -26/04/01 06:09:13 INFO DAGScheduler: Job 14 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 228.856920 s -26/04/01 06:09:13 INFO Utils: Coalesced 1 broadcast batches into 1 (5 rows) -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO MemoryStore: Block broadcast_25 stored as values in memory (estimated size 608.0 B, free 8.4 GiB) -26/04/01 06:09:13 INFO MemoryStore: Block broadcast_25_piece0 stored as bytes in memory (estimated size 563.0 B, free 8.4 GiB) -26/04/01 06:09:13 INFO BlockManagerInfo: Added broadcast_25_piece0 in memory on 10.0.0.133:58259 (size: 563.0 B, free: 8.6 GiB) -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO SparkContext: Created broadcast 25 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:09:13 INFO Executor: Finished task 206.0 in stage 14.0 (TID 297). 6563 bytes result sent to driver -26/04/01 06:09:13 INFO TaskSetManager: Starting task 2.0 in stage 16.0 (TID 302) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9724 bytes) -26/04/01 06:09:13 INFO TaskSetManager: Finished task 206.0 in stage 14.0 (TID 297) in 2559 ms on 10.0.0.133 (executor driver) (207/208) -26/04/01 06:09:13 INFO Executor: Running task 2.0 in stage 16.0 (TID 302) -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO Executor: Finished task 207.0 in stage 14.0 (TID 298). 6563 bytes result sent to driver -26/04/01 06:09:13 INFO TaskSetManager: Starting task 3.0 in stage 16.0 (TID 303) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9721 bytes) -26/04/01 06:09:13 INFO TaskSetManager: Finished task 207.0 in stage 14.0 (TID 298) in 2552 ms on 10.0.0.133 (executor driver) (208/208) -26/04/01 06:09:13 INFO TaskSchedulerImpl: Removed TaskSet 14.0, whose tasks have all completed, from pool -26/04/01 06:09:13 INFO Executor: Running task 3.0 in stage 16.0 (TID 303) -26/04/01 06:09:13 INFO DAGScheduler: ShuffleMapStage 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 229.022 s -26/04/01 06:09:13 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:09:13 INFO DAGScheduler: running: Set(ShuffleMapStage 19, ShuffleMapStage 16) -26/04/01 06:09:13 INFO DAGScheduler: waiting: Set() -26/04/01 06:09:13 INFO DAGScheduler: failed: Set() -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO Executor: Finished task 0.0 in stage 16.0 (TID 300). 6563 bytes result sent to driver -26/04/01 06:09:13 INFO TaskSetManager: Starting task 0.0 in stage 19.0 (TID 304) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:13 INFO Executor: Running task 0.0 in stage 19.0 (TID 304) -26/04/01 06:09:13 INFO TaskSetManager: Finished task 0.0 in stage 16.0 (TID 300) in 211 ms on 10.0.0.133 (executor driver) (1/4) -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO Executor: Finished task 1.0 in stage 16.0 (TID 301). 6563 bytes result sent to driver -26/04/01 06:09:13 INFO TaskSetManager: Starting task 1.0 in stage 19.0 (TID 305) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:13 INFO TaskSetManager: Finished task 1.0 in stage 16.0 (TID 301) in 213 ms on 10.0.0.133 (executor driver) (2/4) -26/04/01 06:09:13 INFO Executor: Running task 1.0 in stage 19.0 (TID 305) -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO Executor: Finished task 2.0 in stage 16.0 (TID 302). 6563 bytes result sent to driver -26/04/01 06:09:13 INFO TaskSetManager: Starting task 2.0 in stage 19.0 (TID 306) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:13 INFO TaskSetManager: Finished task 2.0 in stage 16.0 (TID 302) in 212 ms on 10.0.0.133 (executor driver) (3/4) -26/04/01 06:09:13 INFO Executor: Running task 2.0 in stage 19.0 (TID 306) -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO Executor: Finished task 3.0 in stage 16.0 (TID 303). 6563 bytes result sent to driver -26/04/01 06:09:13 INFO TaskSetManager: Starting task 3.0 in stage 19.0 (TID 307) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:13 INFO Executor: Running task 3.0 in stage 19.0 (TID 307) -26/04/01 06:09:13 INFO TaskSetManager: Finished task 3.0 in stage 16.0 (TID 303) in 205 ms on 10.0.0.133 (executor driver) (4/4) -26/04/01 06:09:13 INFO TaskSchedulerImpl: Removed TaskSet 16.0, whose tasks have all completed, from pool -26/04/01 06:09:13 INFO DAGScheduler: ShuffleMapStage 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 229.211 s -26/04/01 06:09:13 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:09:13 INFO DAGScheduler: running: Set(ShuffleMapStage 19) -26/04/01 06:09:13 INFO DAGScheduler: waiting: Set() -26/04/01 06:09:13 INFO DAGScheduler: failed: Set() -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:14 INFO BlockManagerInfo: Removed broadcast_20_piece0 on 10.0.0.133:58259 in memory (size: 8.0 KiB, free: 8.6 GiB) -26/04/01 06:09:14 INFO BlockManagerInfo: Removed broadcast_21_piece0 on 10.0.0.133:58259 in memory (size: 30.7 KiB, free: 8.6 GiB) -26/04/01 06:09:14 INFO BlockManagerInfo: Removed broadcast_22_piece0 on 10.0.0.133:58259 in memory (size: 7.8 KiB, free: 8.6 GiB) -26/04/01 06:09:25 INFO Executor: Finished task 0.0 in stage 19.0 (TID 304). 9594 bytes result sent to driver -26/04/01 06:09:25 INFO TaskSetManager: Starting task 4.0 in stage 19.0 (TID 308) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:25 INFO Executor: Running task 4.0 in stage 19.0 (TID 308) -26/04/01 06:09:25 INFO TaskSetManager: Finished task 0.0 in stage 19.0 (TID 304) in 11253 ms on 10.0.0.133 (executor driver) (1/6) -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:25 INFO Executor: Finished task 1.0 in stage 19.0 (TID 305). 9594 bytes result sent to driver -26/04/01 06:09:25 INFO TaskSetManager: Starting task 5.0 in stage 19.0 (TID 309) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:25 INFO TaskSetManager: Finished task 1.0 in stage 19.0 (TID 305) in 11584 ms on 10.0.0.133 (executor driver) (2/6) -26/04/01 06:09:25 INFO Executor: Running task 5.0 in stage 19.0 (TID 309) -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 16 (11.1 MiB) non-empty blocks including 16 (11.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 64 (23.7 MiB) non-empty blocks including 64 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 16 (11.1 MiB) non-empty blocks including 16 (11.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Getting 64 (23.7 MiB) non-empty blocks including 64 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:25 INFO Executor: Finished task 3.0 in stage 19.0 (TID 307). 9594 bytes result sent to driver -26/04/01 06:09:25 INFO TaskSetManager: Finished task 3.0 in stage 19.0 (TID 307) in 11954 ms on 10.0.0.133 (executor driver) (3/6) -26/04/01 06:09:25 INFO Executor: Finished task 2.0 in stage 19.0 (TID 306). 9594 bytes result sent to driver -26/04/01 06:09:25 INFO TaskSetManager: Finished task 2.0 in stage 19.0 (TID 306) in 12054 ms on 10.0.0.133 (executor driver) (4/6) -26/04/01 06:09:30 INFO Executor: Finished task 5.0 in stage 19.0 (TID 309). 9594 bytes result sent to driver -26/04/01 06:09:30 INFO TaskSetManager: Finished task 5.0 in stage 19.0 (TID 309) in 5491 ms on 10.0.0.133 (executor driver) (5/6) -26/04/01 06:09:35 INFO Executor: Finished task 4.0 in stage 19.0 (TID 308). 9594 bytes result sent to driver -26/04/01 06:09:35 INFO TaskSetManager: Finished task 4.0 in stage 19.0 (TID 308) in 10141 ms on 10.0.0.133 (executor driver) (6/6) -26/04/01 06:09:35 INFO TaskSchedulerImpl: Removed TaskSet 19.0, whose tasks have all completed, from pool -26/04/01 06:09:35 INFO DAGScheduler: ShuffleMapStage 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 229.847 s -26/04/01 06:09:35 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:09:35 INFO DAGScheduler: running: Set() -26/04/01 06:09:35 INFO DAGScheduler: waiting: Set() -26/04/01 06:09:35 INFO DAGScheduler: failed: Set() -26/04/01 06:09:35 INFO ShufflePartitionsUtil: For shuffle(5, 3), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 06:09:35 INFO DAGScheduler: Registering RDD 48 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 6 -26/04/01 06:09:35 INFO DAGScheduler: Got map stage job 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions -26/04/01 06:09:35 INFO DAGScheduler: Final stage: ShuffleMapStage 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:09:35 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 22, ShuffleMapStage 23) -26/04/01 06:09:35 INFO DAGScheduler: Missing parents: List() -26/04/01 06:09:35 INFO DAGScheduler: Submitting ShuffleMapStage 24 (MapPartitionsRDD[48] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:09:35 INFO MemoryStore: Block broadcast_26 stored as values in memory (estimated size 29.4 KiB, free 8.6 GiB) -26/04/01 06:09:35 INFO MemoryStore: Block broadcast_26_piece0 stored as bytes in memory (estimated size 11.7 KiB, free 8.6 GiB) -26/04/01 06:09:35 INFO BlockManagerInfo: Added broadcast_26_piece0 in memory on 10.0.0.133:58259 (size: 11.7 KiB, free: 8.6 GiB) -26/04/01 06:09:35 INFO SparkContext: Created broadcast 26 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:09:35 INFO DAGScheduler: Submitting 200 missing tasks from ShuffleMapStage 24 (MapPartitionsRDD[48] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 06:09:35 INFO TaskSchedulerImpl: Adding task set 24.0 with 200 tasks resource profile 0 -26/04/01 06:09:35 INFO TaskSetManager: Starting task 0.0 in stage 24.0 (TID 310) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:35 INFO TaskSetManager: Starting task 1.0 in stage 24.0 (TID 311) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:35 INFO TaskSetManager: Starting task 2.0 in stage 24.0 (TID 312) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:35 INFO TaskSetManager: Starting task 3.0 in stage 24.0 (TID 313) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:35 INFO Executor: Running task 0.0 in stage 24.0 (TID 310) -26/04/01 06:09:35 INFO Executor: Running task 2.0 in stage 24.0 (TID 312) -26/04/01 06:09:35 INFO Executor: Running task 1.0 in stage 24.0 (TID 311) -26/04/01 06:09:35 INFO Executor: Running task 3.0 in stage 24.0 (TID 313) -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:38 INFO Executor: Finished task 1.0 in stage 24.0 (TID 311). 9201 bytes result sent to driver -26/04/01 06:09:38 INFO Executor: Finished task 0.0 in stage 24.0 (TID 310). 9201 bytes result sent to driver -26/04/01 06:09:38 INFO TaskSetManager: Starting task 4.0 in stage 24.0 (TID 314) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:38 INFO Executor: Running task 4.0 in stage 24.0 (TID 314) -26/04/01 06:09:38 INFO TaskSetManager: Starting task 5.0 in stage 24.0 (TID 315) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:38 INFO TaskSetManager: Finished task 1.0 in stage 24.0 (TID 311) in 3572 ms on 10.0.0.133 (executor driver) (1/200) -26/04/01 06:09:38 INFO Executor: Running task 5.0 in stage 24.0 (TID 315) -26/04/01 06:09:38 INFO TaskSetManager: Finished task 0.0 in stage 24.0 (TID 310) in 3572 ms on 10.0.0.133 (executor driver) (2/200) -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO Executor: Finished task 2.0 in stage 24.0 (TID 312). 9201 bytes result sent to driver -26/04/01 06:09:38 INFO TaskSetManager: Starting task 6.0 in stage 24.0 (TID 316) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:38 INFO TaskSetManager: Finished task 2.0 in stage 24.0 (TID 312) in 3584 ms on 10.0.0.133 (executor driver) (3/200) -26/04/01 06:09:38 INFO Executor: Running task 6.0 in stage 24.0 (TID 316) -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO Executor: Finished task 3.0 in stage 24.0 (TID 313). 9201 bytes result sent to driver -26/04/01 06:09:38 INFO TaskSetManager: Starting task 7.0 in stage 24.0 (TID 317) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:38 INFO Executor: Running task 7.0 in stage 24.0 (TID 317) -26/04/01 06:09:38 INFO TaskSetManager: Finished task 3.0 in stage 24.0 (TID 313) in 3589 ms on 10.0.0.133 (executor driver) (4/200) -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:42 INFO Executor: Finished task 6.0 in stage 24.0 (TID 316). 8470 bytes result sent to driver -26/04/01 06:09:42 INFO TaskSetManager: Starting task 8.0 in stage 24.0 (TID 318) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:42 INFO Executor: Running task 8.0 in stage 24.0 (TID 318) -26/04/01 06:09:42 INFO TaskSetManager: Finished task 6.0 in stage 24.0 (TID 316) in 3535 ms on 10.0.0.133 (executor driver) (5/200) -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:42 INFO Executor: Finished task 4.0 in stage 24.0 (TID 314). 8470 bytes result sent to driver -26/04/01 06:09:42 INFO TaskSetManager: Starting task 9.0 in stage 24.0 (TID 319) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:42 INFO TaskSetManager: Finished task 4.0 in stage 24.0 (TID 314) in 3564 ms on 10.0.0.133 (executor driver) (6/200) -26/04/01 06:09:42 INFO Executor: Running task 9.0 in stage 24.0 (TID 319) -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:42 INFO Executor: Finished task 7.0 in stage 24.0 (TID 317). 8470 bytes result sent to driver -26/04/01 06:09:42 INFO TaskSetManager: Starting task 10.0 in stage 24.0 (TID 320) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:42 INFO Executor: Running task 10.0 in stage 24.0 (TID 320) -26/04/01 06:09:42 INFO TaskSetManager: Finished task 7.0 in stage 24.0 (TID 317) in 3562 ms on 10.0.0.133 (executor driver) (7/200) -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:42 INFO Executor: Finished task 5.0 in stage 24.0 (TID 315). 8470 bytes result sent to driver -26/04/01 06:09:42 INFO TaskSetManager: Starting task 11.0 in stage 24.0 (TID 321) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:42 INFO Executor: Running task 11.0 in stage 24.0 (TID 321) -26/04/01 06:09:42 INFO TaskSetManager: Finished task 5.0 in stage 24.0 (TID 315) in 3591 ms on 10.0.0.133 (executor driver) (8/200) -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:45 INFO Executor: Finished task 8.0 in stage 24.0 (TID 318). 8470 bytes result sent to driver -26/04/01 06:09:45 INFO TaskSetManager: Starting task 12.0 in stage 24.0 (TID 322) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:45 INFO Executor: Running task 12.0 in stage 24.0 (TID 322) -26/04/01 06:09:45 INFO TaskSetManager: Finished task 8.0 in stage 24.0 (TID 318) in 3495 ms on 10.0.0.133 (executor driver) (9/200) -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO Executor: Finished task 9.0 in stage 24.0 (TID 319). 8470 bytes result sent to driver -26/04/01 06:09:45 INFO TaskSetManager: Starting task 13.0 in stage 24.0 (TID 323) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:45 INFO Executor: Running task 13.0 in stage 24.0 (TID 323) -26/04/01 06:09:45 INFO TaskSetManager: Finished task 9.0 in stage 24.0 (TID 319) in 3489 ms on 10.0.0.133 (executor driver) (10/200) -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:45 INFO Executor: Finished task 10.0 in stage 24.0 (TID 320). 8470 bytes result sent to driver -26/04/01 06:09:45 INFO TaskSetManager: Starting task 14.0 in stage 24.0 (TID 324) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:45 INFO TaskSetManager: Finished task 10.0 in stage 24.0 (TID 320) in 3488 ms on 10.0.0.133 (executor driver) (11/200) -26/04/01 06:09:45 INFO Executor: Running task 14.0 in stage 24.0 (TID 324) -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1027.0 KiB) non-empty blocks including 6 (1027.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1027.0 KiB) non-empty blocks including 6 (1027.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:45 INFO Executor: Finished task 11.0 in stage 24.0 (TID 321). 8470 bytes result sent to driver -26/04/01 06:09:45 INFO TaskSetManager: Starting task 15.0 in stage 24.0 (TID 325) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:45 INFO Executor: Running task 15.0 in stage 24.0 (TID 325) -26/04/01 06:09:45 INFO TaskSetManager: Finished task 11.0 in stage 24.0 (TID 321) in 3497 ms on 10.0.0.133 (executor driver) (12/200) -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:49 INFO Executor: Finished task 12.0 in stage 24.0 (TID 322). 9201 bytes result sent to driver -26/04/01 06:09:49 INFO TaskSetManager: Starting task 16.0 in stage 24.0 (TID 326) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:49 INFO TaskSetManager: Finished task 12.0 in stage 24.0 (TID 322) in 3497 ms on 10.0.0.133 (executor driver) (13/200) -26/04/01 06:09:49 INFO Executor: Running task 16.0 in stage 24.0 (TID 326) -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO Executor: Finished task 13.0 in stage 24.0 (TID 323). 9201 bytes result sent to driver -26/04/01 06:09:49 INFO TaskSetManager: Starting task 17.0 in stage 24.0 (TID 327) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:49 INFO TaskSetManager: Finished task 13.0 in stage 24.0 (TID 323) in 3497 ms on 10.0.0.133 (executor driver) (14/200) -26/04/01 06:09:49 INFO Executor: Running task 17.0 in stage 24.0 (TID 327) -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:49 INFO Executor: Finished task 14.0 in stage 24.0 (TID 324). 9201 bytes result sent to driver -26/04/01 06:09:49 INFO TaskSetManager: Starting task 18.0 in stage 24.0 (TID 328) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:49 INFO TaskSetManager: Finished task 14.0 in stage 24.0 (TID 324) in 3530 ms on 10.0.0.133 (executor driver) (15/200) -26/04/01 06:09:49 INFO Executor: Running task 18.0 in stage 24.0 (TID 328) -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO Executor: Finished task 15.0 in stage 24.0 (TID 325). 9201 bytes result sent to driver -26/04/01 06:09:49 INFO TaskSetManager: Starting task 19.0 in stage 24.0 (TID 329) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:49 INFO Executor: Running task 19.0 in stage 24.0 (TID 329) -26/04/01 06:09:49 INFO TaskSetManager: Finished task 15.0 in stage 24.0 (TID 325) in 3519 ms on 10.0.0.133 (executor driver) (16/200) -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:52 INFO Executor: Finished task 16.0 in stage 24.0 (TID 326). 8470 bytes result sent to driver -26/04/01 06:09:52 INFO TaskSetManager: Starting task 20.0 in stage 24.0 (TID 330) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:52 INFO Executor: Running task 20.0 in stage 24.0 (TID 330) -26/04/01 06:09:52 INFO TaskSetManager: Finished task 16.0 in stage 24.0 (TID 326) in 3468 ms on 10.0.0.133 (executor driver) (17/200) -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:52 INFO Executor: Finished task 17.0 in stage 24.0 (TID 327). 8470 bytes result sent to driver -26/04/01 06:09:52 INFO TaskSetManager: Starting task 21.0 in stage 24.0 (TID 331) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:52 INFO Executor: Running task 21.0 in stage 24.0 (TID 331) -26/04/01 06:09:52 INFO TaskSetManager: Finished task 17.0 in stage 24.0 (TID 327) in 3477 ms on 10.0.0.133 (executor driver) (18/200) -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:52 INFO Executor: Finished task 19.0 in stage 24.0 (TID 329). 8470 bytes result sent to driver -26/04/01 06:09:52 INFO TaskSetManager: Starting task 22.0 in stage 24.0 (TID 332) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:52 INFO TaskSetManager: Finished task 19.0 in stage 24.0 (TID 329) in 3475 ms on 10.0.0.133 (executor driver) (19/200) -26/04/01 06:09:52 INFO Executor: Running task 22.0 in stage 24.0 (TID 332) -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO Executor: Finished task 18.0 in stage 24.0 (TID 328). 8470 bytes result sent to driver -26/04/01 06:09:52 INFO TaskSetManager: Starting task 23.0 in stage 24.0 (TID 333) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:52 INFO Executor: Running task 23.0 in stage 24.0 (TID 333) -26/04/01 06:09:52 INFO TaskSetManager: Finished task 18.0 in stage 24.0 (TID 328) in 3486 ms on 10.0.0.133 (executor driver) (20/200) -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:56 INFO Executor: Finished task 20.0 in stage 24.0 (TID 330). 9201 bytes result sent to driver -26/04/01 06:09:56 INFO TaskSetManager: Starting task 24.0 in stage 24.0 (TID 334) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:56 INFO TaskSetManager: Finished task 20.0 in stage 24.0 (TID 330) in 3473 ms on 10.0.0.133 (executor driver) (21/200) -26/04/01 06:09:56 INFO Executor: Running task 24.0 in stage 24.0 (TID 334) -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO Executor: Finished task 21.0 in stage 24.0 (TID 331). 9201 bytes result sent to driver -26/04/01 06:09:56 INFO TaskSetManager: Starting task 25.0 in stage 24.0 (TID 335) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:56 INFO Executor: Running task 25.0 in stage 24.0 (TID 335) -26/04/01 06:09:56 INFO TaskSetManager: Finished task 21.0 in stage 24.0 (TID 331) in 3460 ms on 10.0.0.133 (executor driver) (22/200) -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:56 INFO Executor: Finished task 22.0 in stage 24.0 (TID 332). 9201 bytes result sent to driver -26/04/01 06:09:56 INFO TaskSetManager: Starting task 26.0 in stage 24.0 (TID 336) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:56 INFO Executor: Running task 26.0 in stage 24.0 (TID 336) -26/04/01 06:09:56 INFO TaskSetManager: Finished task 22.0 in stage 24.0 (TID 332) in 3467 ms on 10.0.0.133 (executor driver) (23/200) -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO Executor: Finished task 23.0 in stage 24.0 (TID 333). 9201 bytes result sent to driver -26/04/01 06:09:56 INFO TaskSetManager: Starting task 27.0 in stage 24.0 (TID 337) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:56 INFO TaskSetManager: Finished task 23.0 in stage 24.0 (TID 333) in 3469 ms on 10.0.0.133 (executor driver) (24/200) -26/04/01 06:09:56 INFO Executor: Running task 27.0 in stage 24.0 (TID 337) -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:59 INFO Executor: Finished task 25.0 in stage 24.0 (TID 335). 8470 bytes result sent to driver -26/04/01 06:09:59 INFO TaskSetManager: Starting task 28.0 in stage 24.0 (TID 338) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:59 INFO Executor: Running task 28.0 in stage 24.0 (TID 338) -26/04/01 06:09:59 INFO TaskSetManager: Finished task 25.0 in stage 24.0 (TID 335) in 3485 ms on 10.0.0.133 (executor driver) (25/200) -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:59 INFO Executor: Finished task 24.0 in stage 24.0 (TID 334). 8470 bytes result sent to driver -26/04/01 06:09:59 INFO TaskSetManager: Starting task 29.0 in stage 24.0 (TID 339) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:59 INFO TaskSetManager: Finished task 24.0 in stage 24.0 (TID 334) in 3504 ms on 10.0.0.133 (executor driver) (26/200) -26/04/01 06:09:59 INFO Executor: Running task 29.0 in stage 24.0 (TID 339) -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:59 INFO Executor: Finished task 26.0 in stage 24.0 (TID 336). 8470 bytes result sent to driver -26/04/01 06:09:59 INFO TaskSetManager: Starting task 30.0 in stage 24.0 (TID 340) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:59 INFO TaskSetManager: Finished task 26.0 in stage 24.0 (TID 336) in 3487 ms on 10.0.0.133 (executor driver) (27/200) -26/04/01 06:09:59 INFO Executor: Running task 30.0 in stage 24.0 (TID 340) -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO Executor: Finished task 27.0 in stage 24.0 (TID 337). 8470 bytes result sent to driver -26/04/01 06:09:59 INFO TaskSetManager: Starting task 31.0 in stage 24.0 (TID 341) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9428 bytes) -26/04/01 06:09:59 INFO Executor: Running task 31.0 in stage 24.0 (TID 341) -26/04/01 06:09:59 INFO TaskSetManager: Finished task 27.0 in stage 24.0 (TID 337) in 3487 ms on 10.0.0.133 (executor driver) (28/200) -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:09:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:09:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:03 INFO Executor: Finished task 28.0 in stage 24.0 (TID 338). 8470 bytes result sent to driver -26/04/01 06:10:03 INFO TaskSetManager: Starting task 32.0 in stage 24.0 (TID 342) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:03 INFO Executor: Running task 32.0 in stage 24.0 (TID 342) -26/04/01 06:10:03 INFO TaskSetManager: Finished task 28.0 in stage 24.0 (TID 338) in 3464 ms on 10.0.0.133 (executor driver) (29/200) -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:03 INFO Executor: Finished task 29.0 in stage 24.0 (TID 339). 8470 bytes result sent to driver -26/04/01 06:10:03 INFO TaskSetManager: Starting task 33.0 in stage 24.0 (TID 343) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:03 INFO TaskSetManager: Finished task 29.0 in stage 24.0 (TID 339) in 3468 ms on 10.0.0.133 (executor driver) (30/200) -26/04/01 06:10:03 INFO Executor: Running task 33.0 in stage 24.0 (TID 343) -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:03 INFO Executor: Finished task 31.0 in stage 24.0 (TID 341). 8470 bytes result sent to driver -26/04/01 06:10:03 INFO TaskSetManager: Starting task 34.0 in stage 24.0 (TID 344) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:03 INFO TaskSetManager: Finished task 31.0 in stage 24.0 (TID 341) in 3457 ms on 10.0.0.133 (executor driver) (31/200) -26/04/01 06:10:03 INFO Executor: Running task 34.0 in stage 24.0 (TID 344) -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO Executor: Finished task 30.0 in stage 24.0 (TID 340). 8470 bytes result sent to driver -26/04/01 06:10:03 INFO TaskSetManager: Starting task 35.0 in stage 24.0 (TID 345) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:03 INFO Executor: Running task 35.0 in stage 24.0 (TID 345) -26/04/01 06:10:03 INFO TaskSetManager: Finished task 30.0 in stage 24.0 (TID 340) in 3472 ms on 10.0.0.133 (executor driver) (32/200) -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:06 INFO Executor: Finished task 32.0 in stage 24.0 (TID 342). 9201 bytes result sent to driver -26/04/01 06:10:06 INFO TaskSetManager: Starting task 36.0 in stage 24.0 (TID 346) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:06 INFO Executor: Running task 36.0 in stage 24.0 (TID 346) -26/04/01 06:10:06 INFO TaskSetManager: Finished task 32.0 in stage 24.0 (TID 342) in 3428 ms on 10.0.0.133 (executor driver) (33/200) -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:06 INFO Executor: Finished task 33.0 in stage 24.0 (TID 343). 9201 bytes result sent to driver -26/04/01 06:10:06 INFO TaskSetManager: Starting task 37.0 in stage 24.0 (TID 347) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:06 INFO TaskSetManager: Finished task 33.0 in stage 24.0 (TID 343) in 3523 ms on 10.0.0.133 (executor driver) (34/200) -26/04/01 06:10:06 INFO Executor: Running task 37.0 in stage 24.0 (TID 347) -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO Executor: Finished task 35.0 in stage 24.0 (TID 345). 9201 bytes result sent to driver -26/04/01 06:10:06 INFO TaskSetManager: Starting task 38.0 in stage 24.0 (TID 348) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:06 INFO Executor: Running task 38.0 in stage 24.0 (TID 348) -26/04/01 06:10:06 INFO TaskSetManager: Finished task 35.0 in stage 24.0 (TID 345) in 3471 ms on 10.0.0.133 (executor driver) (35/200) -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO Executor: Finished task 34.0 in stage 24.0 (TID 344). 9201 bytes result sent to driver -26/04/01 06:10:06 INFO TaskSetManager: Starting task 39.0 in stage 24.0 (TID 349) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:06 INFO Executor: Running task 39.0 in stage 24.0 (TID 349) -26/04/01 06:10:06 INFO TaskSetManager: Finished task 34.0 in stage 24.0 (TID 344) in 3482 ms on 10.0.0.133 (executor driver) (36/200) -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:10 INFO Executor: Finished task 36.0 in stage 24.0 (TID 346). 8470 bytes result sent to driver -26/04/01 06:10:10 INFO TaskSetManager: Starting task 40.0 in stage 24.0 (TID 350) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:10 INFO Executor: Running task 40.0 in stage 24.0 (TID 350) -26/04/01 06:10:10 INFO TaskSetManager: Finished task 36.0 in stage 24.0 (TID 346) in 3370 ms on 10.0.0.133 (executor driver) (37/200) -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:10 INFO Executor: Finished task 38.0 in stage 24.0 (TID 348). 8470 bytes result sent to driver -26/04/01 06:10:10 INFO TaskSetManager: Starting task 41.0 in stage 24.0 (TID 351) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:10 INFO Executor: Finished task 39.0 in stage 24.0 (TID 349). 8470 bytes result sent to driver -26/04/01 06:10:10 INFO TaskSetManager: Finished task 38.0 in stage 24.0 (TID 348) in 3498 ms on 10.0.0.133 (executor driver) (38/200) -26/04/01 06:10:10 INFO Executor: Running task 41.0 in stage 24.0 (TID 351) -26/04/01 06:10:10 INFO TaskSetManager: Starting task 42.0 in stage 24.0 (TID 352) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:10 INFO Executor: Running task 42.0 in stage 24.0 (TID 352) -26/04/01 06:10:10 INFO TaskSetManager: Finished task 39.0 in stage 24.0 (TID 349) in 3497 ms on 10.0.0.133 (executor driver) (39/200) -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO Executor: Finished task 37.0 in stage 24.0 (TID 347). 8470 bytes result sent to driver -26/04/01 06:10:10 INFO TaskSetManager: Starting task 43.0 in stage 24.0 (TID 353) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:10 INFO TaskSetManager: Finished task 37.0 in stage 24.0 (TID 347) in 3504 ms on 10.0.0.133 (executor driver) (40/200) -26/04/01 06:10:10 INFO Executor: Running task 43.0 in stage 24.0 (TID 353) -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:13 INFO Executor: Finished task 40.0 in stage 24.0 (TID 350). 8470 bytes result sent to driver -26/04/01 06:10:13 INFO TaskSetManager: Starting task 44.0 in stage 24.0 (TID 354) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:13 INFO Executor: Running task 44.0 in stage 24.0 (TID 354) -26/04/01 06:10:13 INFO TaskSetManager: Finished task 40.0 in stage 24.0 (TID 350) in 3359 ms on 10.0.0.133 (executor driver) (41/200) -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:13 INFO Executor: Finished task 43.0 in stage 24.0 (TID 353). 8470 bytes result sent to driver -26/04/01 06:10:13 INFO TaskSetManager: Starting task 45.0 in stage 24.0 (TID 355) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:13 INFO TaskSetManager: Finished task 43.0 in stage 24.0 (TID 353) in 3491 ms on 10.0.0.133 (executor driver) (42/200) -26/04/01 06:10:13 INFO Executor: Running task 45.0 in stage 24.0 (TID 355) -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO Executor: Finished task 42.0 in stage 24.0 (TID 352). 8470 bytes result sent to driver -26/04/01 06:10:13 INFO TaskSetManager: Starting task 46.0 in stage 24.0 (TID 356) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:13 INFO Executor: Running task 46.0 in stage 24.0 (TID 356) -26/04/01 06:10:13 INFO TaskSetManager: Finished task 42.0 in stage 24.0 (TID 352) in 3505 ms on 10.0.0.133 (executor driver) (43/200) -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO Executor: Finished task 41.0 in stage 24.0 (TID 351). 8470 bytes result sent to driver -26/04/01 06:10:13 INFO TaskSetManager: Starting task 47.0 in stage 24.0 (TID 357) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:13 INFO TaskSetManager: Finished task 41.0 in stage 24.0 (TID 351) in 3506 ms on 10.0.0.133 (executor driver) (44/200) -26/04/01 06:10:13 INFO Executor: Running task 47.0 in stage 24.0 (TID 357) -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:16 INFO Executor: Finished task 44.0 in stage 24.0 (TID 354). 9201 bytes result sent to driver -26/04/01 06:10:16 INFO TaskSetManager: Starting task 48.0 in stage 24.0 (TID 358) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:16 INFO TaskSetManager: Finished task 44.0 in stage 24.0 (TID 354) in 3362 ms on 10.0.0.133 (executor driver) (45/200) -26/04/01 06:10:16 INFO Executor: Running task 48.0 in stage 24.0 (TID 358) -26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:17 INFO Executor: Finished task 45.0 in stage 24.0 (TID 355). 9201 bytes result sent to driver -26/04/01 06:10:17 INFO TaskSetManager: Starting task 49.0 in stage 24.0 (TID 359) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:17 INFO TaskSetManager: Finished task 45.0 in stage 24.0 (TID 355) in 3408 ms on 10.0.0.133 (executor driver) (46/200) -26/04/01 06:10:17 INFO Executor: Running task 49.0 in stage 24.0 (TID 359) -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:17 INFO Executor: Finished task 46.0 in stage 24.0 (TID 356). 9201 bytes result sent to driver -26/04/01 06:10:17 INFO TaskSetManager: Starting task 50.0 in stage 24.0 (TID 360) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:17 INFO TaskSetManager: Finished task 46.0 in stage 24.0 (TID 356) in 3479 ms on 10.0.0.133 (executor driver) (47/200) -26/04/01 06:10:17 INFO Executor: Running task 50.0 in stage 24.0 (TID 360) -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:17 INFO Executor: Finished task 47.0 in stage 24.0 (TID 357). 9201 bytes result sent to driver -26/04/01 06:10:17 INFO TaskSetManager: Starting task 51.0 in stage 24.0 (TID 361) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:17 INFO Executor: Running task 51.0 in stage 24.0 (TID 361) -26/04/01 06:10:17 INFO TaskSetManager: Finished task 47.0 in stage 24.0 (TID 357) in 3481 ms on 10.0.0.133 (executor driver) (48/200) -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:20 INFO Executor: Finished task 48.0 in stage 24.0 (TID 358). 8470 bytes result sent to driver -26/04/01 06:10:20 INFO TaskSetManager: Starting task 52.0 in stage 24.0 (TID 362) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:20 INFO Executor: Running task 52.0 in stage 24.0 (TID 362) -26/04/01 06:10:20 INFO TaskSetManager: Finished task 48.0 in stage 24.0 (TID 358) in 3396 ms on 10.0.0.133 (executor driver) (49/200) -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:20 INFO Executor: Finished task 49.0 in stage 24.0 (TID 359). 8470 bytes result sent to driver -26/04/01 06:10:20 INFO TaskSetManager: Starting task 53.0 in stage 24.0 (TID 363) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:20 INFO Executor: Running task 53.0 in stage 24.0 (TID 363) -26/04/01 06:10:20 INFO TaskSetManager: Finished task 49.0 in stage 24.0 (TID 359) in 3414 ms on 10.0.0.133 (executor driver) (50/200) -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:20 INFO Executor: Finished task 50.0 in stage 24.0 (TID 360). 8470 bytes result sent to driver -26/04/01 06:10:20 INFO TaskSetManager: Starting task 54.0 in stage 24.0 (TID 364) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:20 INFO Executor: Running task 54.0 in stage 24.0 (TID 364) -26/04/01 06:10:20 INFO TaskSetManager: Finished task 50.0 in stage 24.0 (TID 360) in 3469 ms on 10.0.0.133 (executor driver) (51/200) -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO Executor: Finished task 51.0 in stage 24.0 (TID 361). 8470 bytes result sent to driver -26/04/01 06:10:20 INFO TaskSetManager: Starting task 55.0 in stage 24.0 (TID 365) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:20 INFO Executor: Running task 55.0 in stage 24.0 (TID 365) -26/04/01 06:10:20 INFO TaskSetManager: Finished task 51.0 in stage 24.0 (TID 361) in 3476 ms on 10.0.0.133 (executor driver) (52/200) -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:23 INFO Executor: Finished task 52.0 in stage 24.0 (TID 362). 8470 bytes result sent to driver -26/04/01 06:10:23 INFO TaskSetManager: Starting task 56.0 in stage 24.0 (TID 366) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:23 INFO TaskSetManager: Finished task 52.0 in stage 24.0 (TID 362) in 3346 ms on 10.0.0.133 (executor driver) (53/200) -26/04/01 06:10:23 INFO Executor: Running task 56.0 in stage 24.0 (TID 366) -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:23 INFO Executor: Finished task 53.0 in stage 24.0 (TID 363). 8470 bytes result sent to driver -26/04/01 06:10:23 INFO TaskSetManager: Starting task 57.0 in stage 24.0 (TID 367) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:23 INFO TaskSetManager: Finished task 53.0 in stage 24.0 (TID 363) in 3364 ms on 10.0.0.133 (executor driver) (54/200) -26/04/01 06:10:23 INFO Executor: Running task 57.0 in stage 24.0 (TID 367) -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:24 INFO Executor: Finished task 54.0 in stage 24.0 (TID 364). 8470 bytes result sent to driver -26/04/01 06:10:24 INFO TaskSetManager: Starting task 58.0 in stage 24.0 (TID 368) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:24 INFO TaskSetManager: Finished task 54.0 in stage 24.0 (TID 364) in 3434 ms on 10.0.0.133 (executor driver) (55/200) -26/04/01 06:10:24 INFO Executor: Running task 58.0 in stage 24.0 (TID 368) -26/04/01 06:10:24 INFO Executor: Finished task 55.0 in stage 24.0 (TID 365). 8470 bytes result sent to driver -26/04/01 06:10:24 INFO TaskSetManager: Starting task 59.0 in stage 24.0 (TID 369) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:24 INFO TaskSetManager: Finished task 55.0 in stage 24.0 (TID 365) in 3424 ms on 10.0.0.133 (executor driver) (56/200) -26/04/01 06:10:24 INFO Executor: Running task 59.0 in stage 24.0 (TID 369) -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:26 INFO Executor: Finished task 56.0 in stage 24.0 (TID 366). 9201 bytes result sent to driver -26/04/01 06:10:26 INFO TaskSetManager: Starting task 60.0 in stage 24.0 (TID 370) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:26 INFO TaskSetManager: Finished task 56.0 in stage 24.0 (TID 366) in 3358 ms on 10.0.0.133 (executor driver) (57/200) -26/04/01 06:10:26 INFO Executor: Running task 60.0 in stage 24.0 (TID 370) -26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:27 INFO Executor: Finished task 57.0 in stage 24.0 (TID 367). 9201 bytes result sent to driver -26/04/01 06:10:27 INFO TaskSetManager: Starting task 61.0 in stage 24.0 (TID 371) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:27 INFO Executor: Running task 61.0 in stage 24.0 (TID 371) -26/04/01 06:10:27 INFO TaskSetManager: Finished task 57.0 in stage 24.0 (TID 367) in 3366 ms on 10.0.0.133 (executor driver) (58/200) -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:27 INFO Executor: Finished task 58.0 in stage 24.0 (TID 368). 9201 bytes result sent to driver -26/04/01 06:10:27 INFO TaskSetManager: Starting task 62.0 in stage 24.0 (TID 372) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:27 INFO TaskSetManager: Finished task 58.0 in stage 24.0 (TID 368) in 3433 ms on 10.0.0.133 (executor driver) (59/200) -26/04/01 06:10:27 INFO Executor: Running task 62.0 in stage 24.0 (TID 372) -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:27 INFO Executor: Finished task 59.0 in stage 24.0 (TID 369). 9201 bytes result sent to driver -26/04/01 06:10:27 INFO TaskSetManager: Starting task 63.0 in stage 24.0 (TID 373) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:27 INFO TaskSetManager: Finished task 59.0 in stage 24.0 (TID 369) in 3435 ms on 10.0.0.133 (executor driver) (60/200) -26/04/01 06:10:27 INFO Executor: Running task 63.0 in stage 24.0 (TID 373) -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 6 (999.4 KiB) non-empty blocks including 6 (999.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 6 (999.4 KiB) non-empty blocks including 6 (999.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:30 INFO Executor: Finished task 60.0 in stage 24.0 (TID 370). 8470 bytes result sent to driver -26/04/01 06:10:30 INFO TaskSetManager: Starting task 64.0 in stage 24.0 (TID 374) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:30 INFO Executor: Running task 64.0 in stage 24.0 (TID 374) -26/04/01 06:10:30 INFO TaskSetManager: Finished task 60.0 in stage 24.0 (TID 370) in 3374 ms on 10.0.0.133 (executor driver) (61/200) -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.3 MiB) non-empty blocks including 208 (45.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.3 MiB) non-empty blocks including 208 (45.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:30 INFO Executor: Finished task 61.0 in stage 24.0 (TID 371). 8470 bytes result sent to driver -26/04/01 06:10:30 INFO TaskSetManager: Starting task 65.0 in stage 24.0 (TID 375) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:30 INFO Executor: Running task 65.0 in stage 24.0 (TID 375) -26/04/01 06:10:30 INFO TaskSetManager: Finished task 61.0 in stage 24.0 (TID 371) in 3385 ms on 10.0.0.133 (executor driver) (62/200) -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:31 INFO Executor: Finished task 63.0 in stage 24.0 (TID 373). 8470 bytes result sent to driver -26/04/01 06:10:31 INFO TaskSetManager: Starting task 66.0 in stage 24.0 (TID 376) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:31 INFO TaskSetManager: Finished task 63.0 in stage 24.0 (TID 373) in 3430 ms on 10.0.0.133 (executor driver) (63/200) -26/04/01 06:10:31 INFO Executor: Running task 66.0 in stage 24.0 (TID 376) -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:31 INFO Executor: Finished task 62.0 in stage 24.0 (TID 372). 8470 bytes result sent to driver -26/04/01 06:10:31 INFO TaskSetManager: Starting task 67.0 in stage 24.0 (TID 377) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:31 INFO TaskSetManager: Finished task 62.0 in stage 24.0 (TID 372) in 3443 ms on 10.0.0.133 (executor driver) (64/200) -26/04/01 06:10:31 INFO Executor: Running task 67.0 in stage 24.0 (TID 377) -26/04/01 06:10:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:33 INFO Executor: Finished task 64.0 in stage 24.0 (TID 374). 8470 bytes result sent to driver -26/04/01 06:10:33 INFO TaskSetManager: Starting task 68.0 in stage 24.0 (TID 378) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:33 INFO TaskSetManager: Finished task 64.0 in stage 24.0 (TID 374) in 3352 ms on 10.0.0.133 (executor driver) (65/200) -26/04/01 06:10:33 INFO Executor: Running task 68.0 in stage 24.0 (TID 378) -26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:34 INFO Executor: Finished task 65.0 in stage 24.0 (TID 375). 8470 bytes result sent to driver -26/04/01 06:10:34 INFO TaskSetManager: Starting task 69.0 in stage 24.0 (TID 379) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:34 INFO TaskSetManager: Finished task 65.0 in stage 24.0 (TID 375) in 3340 ms on 10.0.0.133 (executor driver) (66/200) -26/04/01 06:10:34 INFO Executor: Running task 69.0 in stage 24.0 (TID 379) -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:34 INFO Executor: Finished task 67.0 in stage 24.0 (TID 377). 8470 bytes result sent to driver -26/04/01 06:10:34 INFO TaskSetManager: Starting task 70.0 in stage 24.0 (TID 380) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:34 INFO TaskSetManager: Finished task 67.0 in stage 24.0 (TID 377) in 3430 ms on 10.0.0.133 (executor driver) (67/200) -26/04/01 06:10:34 INFO Executor: Running task 70.0 in stage 24.0 (TID 380) -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:34 INFO Executor: Finished task 66.0 in stage 24.0 (TID 376). 8470 bytes result sent to driver -26/04/01 06:10:34 INFO TaskSetManager: Starting task 71.0 in stage 24.0 (TID 381) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:34 INFO Executor: Running task 71.0 in stage 24.0 (TID 381) -26/04/01 06:10:34 INFO TaskSetManager: Finished task 66.0 in stage 24.0 (TID 376) in 3450 ms on 10.0.0.133 (executor driver) (68/200) -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:36 INFO Executor: Finished task 68.0 in stage 24.0 (TID 378). 9201 bytes result sent to driver -26/04/01 06:10:36 INFO TaskSetManager: Starting task 72.0 in stage 24.0 (TID 382) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:36 INFO Executor: Running task 72.0 in stage 24.0 (TID 382) -26/04/01 06:10:36 INFO TaskSetManager: Finished task 68.0 in stage 24.0 (TID 378) in 3350 ms on 10.0.0.133 (executor driver) (69/200) -26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:37 INFO Executor: Finished task 69.0 in stage 24.0 (TID 379). 9201 bytes result sent to driver -26/04/01 06:10:37 INFO TaskSetManager: Starting task 73.0 in stage 24.0 (TID 383) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:37 INFO TaskSetManager: Finished task 69.0 in stage 24.0 (TID 379) in 3337 ms on 10.0.0.133 (executor driver) (70/200) -26/04/01 06:10:37 INFO Executor: Running task 73.0 in stage 24.0 (TID 383) -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:37 INFO Executor: Finished task 71.0 in stage 24.0 (TID 381). 9201 bytes result sent to driver -26/04/01 06:10:37 INFO TaskSetManager: Starting task 74.0 in stage 24.0 (TID 384) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:37 INFO TaskSetManager: Finished task 71.0 in stage 24.0 (TID 381) in 3421 ms on 10.0.0.133 (executor driver) (71/200) -26/04/01 06:10:37 INFO Executor: Running task 74.0 in stage 24.0 (TID 384) -26/04/01 06:10:37 INFO Executor: Finished task 70.0 in stage 24.0 (TID 380). 9201 bytes result sent to driver -26/04/01 06:10:37 INFO TaskSetManager: Starting task 75.0 in stage 24.0 (TID 385) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:37 INFO Executor: Running task 75.0 in stage 24.0 (TID 385) -26/04/01 06:10:37 INFO TaskSetManager: Finished task 70.0 in stage 24.0 (TID 380) in 3430 ms on 10.0.0.133 (executor driver) (72/200) -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:40 INFO Executor: Finished task 72.0 in stage 24.0 (TID 382). 8470 bytes result sent to driver -26/04/01 06:10:40 INFO TaskSetManager: Starting task 76.0 in stage 24.0 (TID 386) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:40 INFO Executor: Running task 76.0 in stage 24.0 (TID 386) -26/04/01 06:10:40 INFO TaskSetManager: Finished task 72.0 in stage 24.0 (TID 382) in 3355 ms on 10.0.0.133 (executor driver) (73/200) -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:40 INFO Executor: Finished task 73.0 in stage 24.0 (TID 383). 8470 bytes result sent to driver -26/04/01 06:10:40 INFO TaskSetManager: Starting task 77.0 in stage 24.0 (TID 387) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:40 INFO Executor: Running task 77.0 in stage 24.0 (TID 387) -26/04/01 06:10:40 INFO TaskSetManager: Finished task 73.0 in stage 24.0 (TID 383) in 3350 ms on 10.0.0.133 (executor driver) (74/200) -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:41 INFO Executor: Finished task 74.0 in stage 24.0 (TID 384). 8470 bytes result sent to driver -26/04/01 06:10:41 INFO TaskSetManager: Starting task 78.0 in stage 24.0 (TID 388) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:41 INFO Executor: Running task 78.0 in stage 24.0 (TID 388) -26/04/01 06:10:41 INFO TaskSetManager: Finished task 74.0 in stage 24.0 (TID 384) in 3429 ms on 10.0.0.133 (executor driver) (75/200) -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:41 INFO Executor: Finished task 75.0 in stage 24.0 (TID 385). 8470 bytes result sent to driver -26/04/01 06:10:41 INFO TaskSetManager: Starting task 79.0 in stage 24.0 (TID 389) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:41 INFO Executor: Running task 79.0 in stage 24.0 (TID 389) -26/04/01 06:10:41 INFO TaskSetManager: Finished task 75.0 in stage 24.0 (TID 385) in 3446 ms on 10.0.0.133 (executor driver) (76/200) -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:43 INFO Executor: Finished task 76.0 in stage 24.0 (TID 386). 8470 bytes result sent to driver -26/04/01 06:10:43 INFO TaskSetManager: Starting task 80.0 in stage 24.0 (TID 390) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:43 INFO Executor: Running task 80.0 in stage 24.0 (TID 390) -26/04/01 06:10:43 INFO TaskSetManager: Finished task 76.0 in stage 24.0 (TID 386) in 3356 ms on 10.0.0.133 (executor driver) (77/200) -26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:44 INFO Executor: Finished task 77.0 in stage 24.0 (TID 387). 8470 bytes result sent to driver -26/04/01 06:10:44 INFO TaskSetManager: Starting task 81.0 in stage 24.0 (TID 391) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:44 INFO Executor: Running task 81.0 in stage 24.0 (TID 391) -26/04/01 06:10:44 INFO TaskSetManager: Finished task 77.0 in stage 24.0 (TID 387) in 3345 ms on 10.0.0.133 (executor driver) (78/200) -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:44 INFO Executor: Finished task 78.0 in stage 24.0 (TID 388). 8470 bytes result sent to driver -26/04/01 06:10:44 INFO TaskSetManager: Starting task 82.0 in stage 24.0 (TID 392) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:44 INFO Executor: Running task 82.0 in stage 24.0 (TID 392) -26/04/01 06:10:44 INFO TaskSetManager: Finished task 78.0 in stage 24.0 (TID 388) in 3430 ms on 10.0.0.133 (executor driver) (79/200) -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:44 INFO Executor: Finished task 79.0 in stage 24.0 (TID 389). 8470 bytes result sent to driver -26/04/01 06:10:44 INFO TaskSetManager: Starting task 83.0 in stage 24.0 (TID 393) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:44 INFO Executor: Running task 83.0 in stage 24.0 (TID 393) -26/04/01 06:10:44 INFO TaskSetManager: Finished task 79.0 in stage 24.0 (TID 389) in 3414 ms on 10.0.0.133 (executor driver) (80/200) -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:46 INFO Executor: Finished task 80.0 in stage 24.0 (TID 390). 9201 bytes result sent to driver -26/04/01 06:10:46 INFO TaskSetManager: Starting task 84.0 in stage 24.0 (TID 394) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:46 INFO Executor: Running task 84.0 in stage 24.0 (TID 394) -26/04/01 06:10:46 INFO TaskSetManager: Finished task 80.0 in stage 24.0 (TID 390) in 3354 ms on 10.0.0.133 (executor driver) (81/200) -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:47 INFO Executor: Finished task 81.0 in stage 24.0 (TID 391). 9201 bytes result sent to driver -26/04/01 06:10:47 INFO TaskSetManager: Starting task 85.0 in stage 24.0 (TID 395) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:47 INFO Executor: Running task 85.0 in stage 24.0 (TID 395) -26/04/01 06:10:47 INFO TaskSetManager: Finished task 81.0 in stage 24.0 (TID 391) in 3360 ms on 10.0.0.133 (executor driver) (82/200) -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:48 INFO Executor: Finished task 83.0 in stage 24.0 (TID 393). 9201 bytes result sent to driver -26/04/01 06:10:48 INFO TaskSetManager: Starting task 86.0 in stage 24.0 (TID 396) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:48 INFO Executor: Running task 86.0 in stage 24.0 (TID 396) -26/04/01 06:10:48 INFO TaskSetManager: Finished task 83.0 in stage 24.0 (TID 393) in 3423 ms on 10.0.0.133 (executor driver) (83/200) -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:48 INFO Executor: Finished task 82.0 in stage 24.0 (TID 392). 9201 bytes result sent to driver -26/04/01 06:10:48 INFO TaskSetManager: Starting task 87.0 in stage 24.0 (TID 397) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:48 INFO TaskSetManager: Finished task 82.0 in stage 24.0 (TID 392) in 3428 ms on 10.0.0.133 (executor driver) (84/200) -26/04/01 06:10:48 INFO Executor: Running task 87.0 in stage 24.0 (TID 397) -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:50 INFO Executor: Finished task 84.0 in stage 24.0 (TID 394). 8470 bytes result sent to driver -26/04/01 06:10:50 INFO TaskSetManager: Starting task 88.0 in stage 24.0 (TID 398) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:50 INFO Executor: Running task 88.0 in stage 24.0 (TID 398) -26/04/01 06:10:50 INFO TaskSetManager: Finished task 84.0 in stage 24.0 (TID 394) in 3363 ms on 10.0.0.133 (executor driver) (85/200) -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:50 INFO Executor: Finished task 85.0 in stage 24.0 (TID 395). 8470 bytes result sent to driver -26/04/01 06:10:50 INFO TaskSetManager: Starting task 89.0 in stage 24.0 (TID 399) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:50 INFO Executor: Running task 89.0 in stage 24.0 (TID 399) -26/04/01 06:10:50 INFO TaskSetManager: Finished task 85.0 in stage 24.0 (TID 395) in 3370 ms on 10.0.0.133 (executor driver) (86/200) -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:51 INFO Executor: Finished task 87.0 in stage 24.0 (TID 397). 8470 bytes result sent to driver -26/04/01 06:10:51 INFO TaskSetManager: Starting task 90.0 in stage 24.0 (TID 400) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:51 INFO Executor: Running task 90.0 in stage 24.0 (TID 400) -26/04/01 06:10:51 INFO TaskSetManager: Finished task 87.0 in stage 24.0 (TID 397) in 3433 ms on 10.0.0.133 (executor driver) (87/200) -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:51 INFO Executor: Finished task 86.0 in stage 24.0 (TID 396). 8470 bytes result sent to driver -26/04/01 06:10:51 INFO TaskSetManager: Starting task 91.0 in stage 24.0 (TID 401) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:51 INFO Executor: Running task 91.0 in stage 24.0 (TID 401) -26/04/01 06:10:51 INFO TaskSetManager: Finished task 86.0 in stage 24.0 (TID 396) in 3441 ms on 10.0.0.133 (executor driver) (88/200) -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:53 INFO Executor: Finished task 88.0 in stage 24.0 (TID 398). 8470 bytes result sent to driver -26/04/01 06:10:53 INFO TaskSetManager: Starting task 92.0 in stage 24.0 (TID 402) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:53 INFO TaskSetManager: Finished task 88.0 in stage 24.0 (TID 398) in 3349 ms on 10.0.0.133 (executor driver) (89/200) -26/04/01 06:10:53 INFO Executor: Running task 92.0 in stage 24.0 (TID 402) -26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:54 INFO Executor: Finished task 89.0 in stage 24.0 (TID 399). 8470 bytes result sent to driver -26/04/01 06:10:54 INFO TaskSetManager: Starting task 93.0 in stage 24.0 (TID 403) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:54 INFO Executor: Running task 93.0 in stage 24.0 (TID 403) -26/04/01 06:10:54 INFO TaskSetManager: Finished task 89.0 in stage 24.0 (TID 399) in 3380 ms on 10.0.0.133 (executor driver) (90/200) -26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:55 INFO Executor: Finished task 91.0 in stage 24.0 (TID 401). 8470 bytes result sent to driver -26/04/01 06:10:55 INFO TaskSetManager: Starting task 94.0 in stage 24.0 (TID 404) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:55 INFO Executor: Running task 94.0 in stage 24.0 (TID 404) -26/04/01 06:10:55 INFO TaskSetManager: Finished task 91.0 in stage 24.0 (TID 401) in 3439 ms on 10.0.0.133 (executor driver) (91/200) -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:55 INFO Executor: Finished task 90.0 in stage 24.0 (TID 400). 8470 bytes result sent to driver -26/04/01 06:10:55 INFO TaskSetManager: Starting task 95.0 in stage 24.0 (TID 405) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:55 INFO TaskSetManager: Finished task 90.0 in stage 24.0 (TID 400) in 3445 ms on 10.0.0.133 (executor driver) (92/200) -26/04/01 06:10:55 INFO Executor: Running task 95.0 in stage 24.0 (TID 405) -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:57 INFO Executor: Finished task 92.0 in stage 24.0 (TID 402). 9201 bytes result sent to driver -26/04/01 06:10:57 INFO TaskSetManager: Starting task 96.0 in stage 24.0 (TID 406) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:57 INFO Executor: Running task 96.0 in stage 24.0 (TID 406) -26/04/01 06:10:57 INFO TaskSetManager: Finished task 92.0 in stage 24.0 (TID 402) in 3346 ms on 10.0.0.133 (executor driver) (93/200) -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:57 INFO Executor: Finished task 93.0 in stage 24.0 (TID 403). 9201 bytes result sent to driver -26/04/01 06:10:57 INFO TaskSetManager: Starting task 97.0 in stage 24.0 (TID 407) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:57 INFO Executor: Running task 97.0 in stage 24.0 (TID 407) -26/04/01 06:10:57 INFO TaskSetManager: Finished task 93.0 in stage 24.0 (TID 403) in 3342 ms on 10.0.0.133 (executor driver) (94/200) -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:58 INFO Executor: Finished task 95.0 in stage 24.0 (TID 405). 9201 bytes result sent to driver -26/04/01 06:10:58 INFO TaskSetManager: Starting task 98.0 in stage 24.0 (TID 408) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:58 INFO TaskSetManager: Finished task 95.0 in stage 24.0 (TID 405) in 3427 ms on 10.0.0.133 (executor driver) (95/200) -26/04/01 06:10:58 INFO Executor: Running task 98.0 in stage 24.0 (TID 408) -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:58 INFO Executor: Finished task 94.0 in stage 24.0 (TID 404). 9201 bytes result sent to driver -26/04/01 06:10:58 INFO TaskSetManager: Starting task 99.0 in stage 24.0 (TID 409) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9428 bytes) -26/04/01 06:10:58 INFO TaskSetManager: Finished task 94.0 in stage 24.0 (TID 404) in 3431 ms on 10.0.0.133 (executor driver) (96/200) -26/04/01 06:10:58 INFO Executor: Running task 99.0 in stage 24.0 (TID 409) -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:10:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:10:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:00 INFO Executor: Finished task 96.0 in stage 24.0 (TID 406). 8470 bytes result sent to driver -26/04/01 06:11:00 INFO TaskSetManager: Starting task 100.0 in stage 24.0 (TID 410) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:00 INFO Executor: Running task 100.0 in stage 24.0 (TID 410) -26/04/01 06:11:00 INFO TaskSetManager: Finished task 96.0 in stage 24.0 (TID 406) in 3368 ms on 10.0.0.133 (executor driver) (97/200) -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:00 INFO Executor: Finished task 97.0 in stage 24.0 (TID 407). 8470 bytes result sent to driver -26/04/01 06:11:00 INFO TaskSetManager: Starting task 101.0 in stage 24.0 (TID 411) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:00 INFO Executor: Running task 101.0 in stage 24.0 (TID 411) -26/04/01 06:11:00 INFO TaskSetManager: Finished task 97.0 in stage 24.0 (TID 407) in 3360 ms on 10.0.0.133 (executor driver) (98/200) -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:01 INFO Executor: Finished task 98.0 in stage 24.0 (TID 408). 8470 bytes result sent to driver -26/04/01 06:11:01 INFO TaskSetManager: Starting task 102.0 in stage 24.0 (TID 412) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:01 INFO Executor: Running task 102.0 in stage 24.0 (TID 412) -26/04/01 06:11:01 INFO TaskSetManager: Finished task 98.0 in stage 24.0 (TID 408) in 3429 ms on 10.0.0.133 (executor driver) (99/200) -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:01 INFO Executor: Finished task 99.0 in stage 24.0 (TID 409). 8470 bytes result sent to driver -26/04/01 06:11:01 INFO TaskSetManager: Starting task 103.0 in stage 24.0 (TID 413) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:01 INFO TaskSetManager: Finished task 99.0 in stage 24.0 (TID 409) in 3430 ms on 10.0.0.133 (executor driver) (100/200) -26/04/01 06:11:01 INFO Executor: Running task 103.0 in stage 24.0 (TID 413) -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:03 INFO Executor: Finished task 100.0 in stage 24.0 (TID 410). 8470 bytes result sent to driver -26/04/01 06:11:03 INFO TaskSetManager: Starting task 104.0 in stage 24.0 (TID 414) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:03 INFO Executor: Running task 104.0 in stage 24.0 (TID 414) -26/04/01 06:11:03 INFO TaskSetManager: Finished task 100.0 in stage 24.0 (TID 410) in 3361 ms on 10.0.0.133 (executor driver) (101/200) -26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:04 INFO Executor: Finished task 101.0 in stage 24.0 (TID 411). 8470 bytes result sent to driver -26/04/01 06:11:04 INFO TaskSetManager: Starting task 105.0 in stage 24.0 (TID 415) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:04 INFO Executor: Running task 105.0 in stage 24.0 (TID 415) -26/04/01 06:11:04 INFO TaskSetManager: Finished task 101.0 in stage 24.0 (TID 411) in 3347 ms on 10.0.0.133 (executor driver) (102/200) -26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:05 INFO Executor: Finished task 102.0 in stage 24.0 (TID 412). 8470 bytes result sent to driver -26/04/01 06:11:05 INFO TaskSetManager: Starting task 106.0 in stage 24.0 (TID 416) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:05 INFO Executor: Running task 106.0 in stage 24.0 (TID 416) -26/04/01 06:11:05 INFO TaskSetManager: Finished task 102.0 in stage 24.0 (TID 412) in 3430 ms on 10.0.0.133 (executor driver) (103/200) -26/04/01 06:11:05 INFO Executor: Finished task 103.0 in stage 24.0 (TID 413). 8470 bytes result sent to driver -26/04/01 06:11:05 INFO TaskSetManager: Starting task 107.0 in stage 24.0 (TID 417) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:05 INFO TaskSetManager: Finished task 103.0 in stage 24.0 (TID 413) in 3427 ms on 10.0.0.133 (executor driver) (104/200) -26/04/01 06:11:05 INFO Executor: Running task 107.0 in stage 24.0 (TID 417) -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:07 INFO Executor: Finished task 104.0 in stage 24.0 (TID 414). 9201 bytes result sent to driver -26/04/01 06:11:07 INFO TaskSetManager: Starting task 108.0 in stage 24.0 (TID 418) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:07 INFO Executor: Running task 108.0 in stage 24.0 (TID 418) -26/04/01 06:11:07 INFO TaskSetManager: Finished task 104.0 in stage 24.0 (TID 414) in 3342 ms on 10.0.0.133 (executor driver) (105/200) -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:07 INFO Executor: Finished task 105.0 in stage 24.0 (TID 415). 9201 bytes result sent to driver -26/04/01 06:11:07 INFO TaskSetManager: Starting task 109.0 in stage 24.0 (TID 419) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:07 INFO Executor: Running task 109.0 in stage 24.0 (TID 419) -26/04/01 06:11:07 INFO TaskSetManager: Finished task 105.0 in stage 24.0 (TID 415) in 3347 ms on 10.0.0.133 (executor driver) (106/200) -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:08 INFO Executor: Finished task 106.0 in stage 24.0 (TID 416). 9201 bytes result sent to driver -26/04/01 06:11:08 INFO TaskSetManager: Starting task 110.0 in stage 24.0 (TID 420) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:08 INFO Executor: Finished task 107.0 in stage 24.0 (TID 417). 9201 bytes result sent to driver -26/04/01 06:11:08 INFO Executor: Running task 110.0 in stage 24.0 (TID 420) -26/04/01 06:11:08 INFO TaskSetManager: Starting task 111.0 in stage 24.0 (TID 421) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:08 INFO Executor: Running task 111.0 in stage 24.0 (TID 421) -26/04/01 06:11:08 INFO TaskSetManager: Finished task 106.0 in stage 24.0 (TID 416) in 3429 ms on 10.0.0.133 (executor driver) (107/200) -26/04/01 06:11:08 INFO TaskSetManager: Finished task 107.0 in stage 24.0 (TID 417) in 3429 ms on 10.0.0.133 (executor driver) (108/200) -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:10 INFO Executor: Finished task 108.0 in stage 24.0 (TID 418). 8470 bytes result sent to driver -26/04/01 06:11:10 INFO TaskSetManager: Starting task 112.0 in stage 24.0 (TID 422) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:10 INFO Executor: Running task 112.0 in stage 24.0 (TID 422) -26/04/01 06:11:10 INFO TaskSetManager: Finished task 108.0 in stage 24.0 (TID 418) in 3353 ms on 10.0.0.133 (executor driver) (109/200) -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:10 INFO Executor: Finished task 109.0 in stage 24.0 (TID 419). 8470 bytes result sent to driver -26/04/01 06:11:10 INFO TaskSetManager: Starting task 113.0 in stage 24.0 (TID 423) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:10 INFO Executor: Running task 113.0 in stage 24.0 (TID 423) -26/04/01 06:11:10 INFO TaskSetManager: Finished task 109.0 in stage 24.0 (TID 419) in 3355 ms on 10.0.0.133 (executor driver) (110/200) -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:12 INFO Executor: Finished task 110.0 in stage 24.0 (TID 420). 8470 bytes result sent to driver -26/04/01 06:11:12 INFO TaskSetManager: Starting task 114.0 in stage 24.0 (TID 424) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:12 INFO Executor: Running task 114.0 in stage 24.0 (TID 424) -26/04/01 06:11:12 INFO TaskSetManager: Finished task 110.0 in stage 24.0 (TID 420) in 3435 ms on 10.0.0.133 (executor driver) (111/200) -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:12 INFO Executor: Finished task 111.0 in stage 24.0 (TID 421). 8470 bytes result sent to driver -26/04/01 06:11:12 INFO TaskSetManager: Starting task 115.0 in stage 24.0 (TID 425) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:12 INFO Executor: Running task 115.0 in stage 24.0 (TID 425) -26/04/01 06:11:12 INFO TaskSetManager: Finished task 111.0 in stage 24.0 (TID 421) in 3443 ms on 10.0.0.133 (executor driver) (112/200) -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:13 INFO Executor: Finished task 112.0 in stage 24.0 (TID 422). 8470 bytes result sent to driver -26/04/01 06:11:13 INFO TaskSetManager: Starting task 116.0 in stage 24.0 (TID 426) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:13 INFO Executor: Running task 116.0 in stage 24.0 (TID 426) -26/04/01 06:11:13 INFO TaskSetManager: Finished task 112.0 in stage 24.0 (TID 422) in 3357 ms on 10.0.0.133 (executor driver) (113/200) -26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:14 INFO Executor: Finished task 113.0 in stage 24.0 (TID 423). 8470 bytes result sent to driver -26/04/01 06:11:14 INFO TaskSetManager: Starting task 117.0 in stage 24.0 (TID 427) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:14 INFO Executor: Running task 117.0 in stage 24.0 (TID 427) -26/04/01 06:11:14 INFO TaskSetManager: Finished task 113.0 in stage 24.0 (TID 423) in 3354 ms on 10.0.0.133 (executor driver) (114/200) -26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:15 INFO Executor: Finished task 114.0 in stage 24.0 (TID 424). 8470 bytes result sent to driver -26/04/01 06:11:15 INFO TaskSetManager: Starting task 118.0 in stage 24.0 (TID 428) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:15 INFO Executor: Running task 118.0 in stage 24.0 (TID 428) -26/04/01 06:11:15 INFO TaskSetManager: Finished task 114.0 in stage 24.0 (TID 424) in 3424 ms on 10.0.0.133 (executor driver) (115/200) -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 6 (1017.0 KiB) non-empty blocks including 6 (1017.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:15 INFO Executor: Finished task 115.0 in stage 24.0 (TID 425). 8470 bytes result sent to driver -26/04/01 06:11:15 INFO TaskSetManager: Starting task 119.0 in stage 24.0 (TID 429) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:15 INFO Executor: Running task 119.0 in stage 24.0 (TID 429) -26/04/01 06:11:15 INFO TaskSetManager: Finished task 115.0 in stage 24.0 (TID 425) in 3420 ms on 10.0.0.133 (executor driver) (116/200) -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 6 (1017.0 KiB) non-empty blocks including 6 (1017.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:17 INFO Executor: Finished task 116.0 in stage 24.0 (TID 426). 9201 bytes result sent to driver -26/04/01 06:11:17 INFO TaskSetManager: Starting task 120.0 in stage 24.0 (TID 430) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:17 INFO Executor: Running task 120.0 in stage 24.0 (TID 430) -26/04/01 06:11:17 INFO TaskSetManager: Finished task 116.0 in stage 24.0 (TID 426) in 3352 ms on 10.0.0.133 (executor driver) (117/200) -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:17 INFO Executor: Finished task 117.0 in stage 24.0 (TID 427). 9201 bytes result sent to driver -26/04/01 06:11:17 INFO TaskSetManager: Starting task 121.0 in stage 24.0 (TID 431) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:17 INFO Executor: Running task 121.0 in stage 24.0 (TID 431) -26/04/01 06:11:17 INFO TaskSetManager: Finished task 117.0 in stage 24.0 (TID 427) in 3373 ms on 10.0.0.133 (executor driver) (118/200) -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:19 INFO Executor: Finished task 118.0 in stage 24.0 (TID 428). 9201 bytes result sent to driver -26/04/01 06:11:19 INFO TaskSetManager: Starting task 122.0 in stage 24.0 (TID 432) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:19 INFO Executor: Running task 122.0 in stage 24.0 (TID 432) -26/04/01 06:11:19 INFO Executor: Finished task 119.0 in stage 24.0 (TID 429). 9201 bytes result sent to driver -26/04/01 06:11:19 INFO TaskSetManager: Starting task 123.0 in stage 24.0 (TID 433) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:19 INFO Executor: Running task 123.0 in stage 24.0 (TID 433) -26/04/01 06:11:19 INFO TaskSetManager: Finished task 118.0 in stage 24.0 (TID 428) in 3427 ms on 10.0.0.133 (executor driver) (119/200) -26/04/01 06:11:19 INFO TaskSetManager: Finished task 119.0 in stage 24.0 (TID 429) in 3422 ms on 10.0.0.133 (executor driver) (120/200) -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:20 INFO Executor: Finished task 120.0 in stage 24.0 (TID 430). 8470 bytes result sent to driver -26/04/01 06:11:20 INFO TaskSetManager: Starting task 124.0 in stage 24.0 (TID 434) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:20 INFO Executor: Running task 124.0 in stage 24.0 (TID 434) -26/04/01 06:11:20 INFO TaskSetManager: Finished task 120.0 in stage 24.0 (TID 430) in 3359 ms on 10.0.0.133 (executor driver) (121/200) -26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:21 INFO Executor: Finished task 121.0 in stage 24.0 (TID 431). 8470 bytes result sent to driver -26/04/01 06:11:21 INFO TaskSetManager: Starting task 125.0 in stage 24.0 (TID 435) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:21 INFO TaskSetManager: Finished task 121.0 in stage 24.0 (TID 431) in 3351 ms on 10.0.0.133 (executor driver) (122/200) -26/04/01 06:11:21 INFO Executor: Running task 125.0 in stage 24.0 (TID 435) -26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:22 INFO Executor: Finished task 122.0 in stage 24.0 (TID 432). 8470 bytes result sent to driver -26/04/01 06:11:22 INFO TaskSetManager: Starting task 126.0 in stage 24.0 (TID 436) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:22 INFO Executor: Running task 126.0 in stage 24.0 (TID 436) -26/04/01 06:11:22 INFO TaskSetManager: Finished task 122.0 in stage 24.0 (TID 432) in 3424 ms on 10.0.0.133 (executor driver) (123/200) -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:22 INFO Executor: Finished task 123.0 in stage 24.0 (TID 433). 8470 bytes result sent to driver -26/04/01 06:11:22 INFO TaskSetManager: Starting task 127.0 in stage 24.0 (TID 437) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:22 INFO Executor: Running task 127.0 in stage 24.0 (TID 437) -26/04/01 06:11:22 INFO TaskSetManager: Finished task 123.0 in stage 24.0 (TID 433) in 3426 ms on 10.0.0.133 (executor driver) (124/200) -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:23 INFO Executor: Finished task 124.0 in stage 24.0 (TID 434). 8470 bytes result sent to driver -26/04/01 06:11:23 INFO TaskSetManager: Starting task 128.0 in stage 24.0 (TID 438) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:23 INFO Executor: Running task 128.0 in stage 24.0 (TID 438) -26/04/01 06:11:23 INFO TaskSetManager: Finished task 124.0 in stage 24.0 (TID 434) in 3341 ms on 10.0.0.133 (executor driver) (125/200) -26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:24 INFO Executor: Finished task 125.0 in stage 24.0 (TID 435). 8470 bytes result sent to driver -26/04/01 06:11:24 INFO TaskSetManager: Starting task 129.0 in stage 24.0 (TID 439) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:24 INFO Executor: Running task 129.0 in stage 24.0 (TID 439) -26/04/01 06:11:24 INFO TaskSetManager: Finished task 125.0 in stage 24.0 (TID 435) in 3350 ms on 10.0.0.133 (executor driver) (126/200) -26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:25 INFO Executor: Finished task 126.0 in stage 24.0 (TID 436). 9201 bytes result sent to driver -26/04/01 06:11:25 INFO TaskSetManager: Starting task 130.0 in stage 24.0 (TID 440) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:25 INFO Executor: Running task 130.0 in stage 24.0 (TID 440) -26/04/01 06:11:25 INFO TaskSetManager: Finished task 126.0 in stage 24.0 (TID 436) in 3437 ms on 10.0.0.133 (executor driver) (127/200) -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:25 INFO Executor: Finished task 127.0 in stage 24.0 (TID 437). 9201 bytes result sent to driver -26/04/01 06:11:25 INFO TaskSetManager: Starting task 131.0 in stage 24.0 (TID 441) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:25 INFO Executor: Running task 131.0 in stage 24.0 (TID 441) -26/04/01 06:11:25 INFO TaskSetManager: Finished task 127.0 in stage 24.0 (TID 437) in 3435 ms on 10.0.0.133 (executor driver) (128/200) -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:27 INFO Executor: Finished task 128.0 in stage 24.0 (TID 438). 9201 bytes result sent to driver -26/04/01 06:11:27 INFO TaskSetManager: Starting task 132.0 in stage 24.0 (TID 442) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:27 INFO Executor: Running task 132.0 in stage 24.0 (TID 442) -26/04/01 06:11:27 INFO TaskSetManager: Finished task 128.0 in stage 24.0 (TID 438) in 3379 ms on 10.0.0.133 (executor driver) (129/200) -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:27 INFO Executor: Finished task 129.0 in stage 24.0 (TID 439). 9201 bytes result sent to driver -26/04/01 06:11:27 INFO TaskSetManager: Starting task 133.0 in stage 24.0 (TID 443) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:27 INFO Executor: Running task 133.0 in stage 24.0 (TID 443) -26/04/01 06:11:27 INFO TaskSetManager: Finished task 129.0 in stage 24.0 (TID 439) in 3364 ms on 10.0.0.133 (executor driver) (130/200) -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:29 INFO Executor: Finished task 131.0 in stage 24.0 (TID 441). 8470 bytes result sent to driver -26/04/01 06:11:29 INFO TaskSetManager: Starting task 134.0 in stage 24.0 (TID 444) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:29 INFO Executor: Running task 134.0 in stage 24.0 (TID 444) -26/04/01 06:11:29 INFO TaskSetManager: Finished task 131.0 in stage 24.0 (TID 441) in 3426 ms on 10.0.0.133 (executor driver) (131/200) -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:29 INFO Executor: Finished task 130.0 in stage 24.0 (TID 440). 8470 bytes result sent to driver -26/04/01 06:11:29 INFO TaskSetManager: Starting task 135.0 in stage 24.0 (TID 445) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:29 INFO Executor: Running task 135.0 in stage 24.0 (TID 445) -26/04/01 06:11:29 INFO TaskSetManager: Finished task 130.0 in stage 24.0 (TID 440) in 3441 ms on 10.0.0.133 (executor driver) (132/200) -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:30 INFO Executor: Finished task 132.0 in stage 24.0 (TID 442). 8470 bytes result sent to driver -26/04/01 06:11:30 INFO TaskSetManager: Starting task 136.0 in stage 24.0 (TID 446) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:30 INFO Executor: Running task 136.0 in stage 24.0 (TID 446) -26/04/01 06:11:30 INFO TaskSetManager: Finished task 132.0 in stage 24.0 (TID 442) in 3370 ms on 10.0.0.133 (executor driver) (133/200) -26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:31 INFO Executor: Finished task 133.0 in stage 24.0 (TID 443). 8470 bytes result sent to driver -26/04/01 06:11:31 INFO TaskSetManager: Starting task 137.0 in stage 24.0 (TID 447) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:31 INFO Executor: Running task 137.0 in stage 24.0 (TID 447) -26/04/01 06:11:31 INFO TaskSetManager: Finished task 133.0 in stage 24.0 (TID 443) in 3355 ms on 10.0.0.133 (executor driver) (134/200) -26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:32 INFO Executor: Finished task 135.0 in stage 24.0 (TID 445). 8470 bytes result sent to driver -26/04/01 06:11:32 INFO TaskSetManager: Starting task 138.0 in stage 24.0 (TID 448) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:32 INFO TaskSetManager: Finished task 135.0 in stage 24.0 (TID 445) in 3386 ms on 10.0.0.133 (executor driver) (135/200) -26/04/01 06:11:32 INFO Executor: Running task 138.0 in stage 24.0 (TID 448) -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:32 INFO Executor: Finished task 134.0 in stage 24.0 (TID 444). 8470 bytes result sent to driver -26/04/01 06:11:32 INFO TaskSetManager: Starting task 139.0 in stage 24.0 (TID 449) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:32 INFO TaskSetManager: Finished task 134.0 in stage 24.0 (TID 444) in 3407 ms on 10.0.0.133 (executor driver) (136/200) -26/04/01 06:11:32 INFO Executor: Running task 139.0 in stage 24.0 (TID 449) -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:34 INFO Executor: Finished task 136.0 in stage 24.0 (TID 446). 8470 bytes result sent to driver -26/04/01 06:11:34 INFO TaskSetManager: Starting task 140.0 in stage 24.0 (TID 450) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:34 INFO Executor: Running task 140.0 in stage 24.0 (TID 450) -26/04/01 06:11:34 INFO TaskSetManager: Finished task 136.0 in stage 24.0 (TID 446) in 3367 ms on 10.0.0.133 (executor driver) (137/200) -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:34 INFO Executor: Finished task 137.0 in stage 24.0 (TID 447). 8470 bytes result sent to driver -26/04/01 06:11:34 INFO TaskSetManager: Starting task 141.0 in stage 24.0 (TID 451) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:34 INFO Executor: Running task 141.0 in stage 24.0 (TID 451) -26/04/01 06:11:34 INFO TaskSetManager: Finished task 137.0 in stage 24.0 (TID 447) in 3365 ms on 10.0.0.133 (executor driver) (138/200) -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:36 INFO Executor: Finished task 138.0 in stage 24.0 (TID 448). 9201 bytes result sent to driver -26/04/01 06:11:36 INFO TaskSetManager: Starting task 142.0 in stage 24.0 (TID 452) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:36 INFO Executor: Running task 142.0 in stage 24.0 (TID 452) -26/04/01 06:11:36 INFO TaskSetManager: Finished task 138.0 in stage 24.0 (TID 448) in 3415 ms on 10.0.0.133 (executor driver) (139/200) -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:36 INFO Executor: Finished task 139.0 in stage 24.0 (TID 449). 9201 bytes result sent to driver -26/04/01 06:11:36 INFO TaskSetManager: Starting task 143.0 in stage 24.0 (TID 453) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:36 INFO Executor: Running task 143.0 in stage 24.0 (TID 453) -26/04/01 06:11:36 INFO TaskSetManager: Finished task 139.0 in stage 24.0 (TID 449) in 3410 ms on 10.0.0.133 (executor driver) (140/200) -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:37 INFO Executor: Finished task 140.0 in stage 24.0 (TID 450). 9201 bytes result sent to driver -26/04/01 06:11:37 INFO TaskSetManager: Starting task 144.0 in stage 24.0 (TID 454) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:37 INFO Executor: Running task 144.0 in stage 24.0 (TID 454) -26/04/01 06:11:37 INFO TaskSetManager: Finished task 140.0 in stage 24.0 (TID 450) in 3385 ms on 10.0.0.133 (executor driver) (141/200) -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:37 INFO Executor: Finished task 141.0 in stage 24.0 (TID 451). 9201 bytes result sent to driver -26/04/01 06:11:37 INFO TaskSetManager: Starting task 145.0 in stage 24.0 (TID 455) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:37 INFO Executor: Running task 145.0 in stage 24.0 (TID 455) -26/04/01 06:11:37 INFO TaskSetManager: Finished task 141.0 in stage 24.0 (TID 451) in 3403 ms on 10.0.0.133 (executor driver) (142/200) -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:39 INFO Executor: Finished task 143.0 in stage 24.0 (TID 453). 8470 bytes result sent to driver -26/04/01 06:11:39 INFO TaskSetManager: Starting task 146.0 in stage 24.0 (TID 456) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:39 INFO TaskSetManager: Finished task 143.0 in stage 24.0 (TID 453) in 3436 ms on 10.0.0.133 (executor driver) (143/200) -26/04/01 06:11:39 INFO Executor: Running task 146.0 in stage 24.0 (TID 456) -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:39 INFO Executor: Finished task 142.0 in stage 24.0 (TID 452). 8470 bytes result sent to driver -26/04/01 06:11:39 INFO TaskSetManager: Starting task 147.0 in stage 24.0 (TID 457) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:39 INFO Executor: Running task 147.0 in stage 24.0 (TID 457) -26/04/01 06:11:39 INFO TaskSetManager: Finished task 142.0 in stage 24.0 (TID 452) in 3441 ms on 10.0.0.133 (executor driver) (144/200) -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:40 INFO Executor: Finished task 144.0 in stage 24.0 (TID 454). 8470 bytes result sent to driver -26/04/01 06:11:40 INFO TaskSetManager: Starting task 148.0 in stage 24.0 (TID 458) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:40 INFO Executor: Running task 148.0 in stage 24.0 (TID 458) -26/04/01 06:11:40 INFO TaskSetManager: Finished task 144.0 in stage 24.0 (TID 454) in 3388 ms on 10.0.0.133 (executor driver) (145/200) -26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:41 INFO Executor: Finished task 145.0 in stage 24.0 (TID 455). 8470 bytes result sent to driver -26/04/01 06:11:41 INFO TaskSetManager: Starting task 149.0 in stage 24.0 (TID 459) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:41 INFO Executor: Running task 149.0 in stage 24.0 (TID 459) -26/04/01 06:11:41 INFO TaskSetManager: Finished task 145.0 in stage 24.0 (TID 455) in 3366 ms on 10.0.0.133 (executor driver) (146/200) -26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:43 INFO Executor: Finished task 147.0 in stage 24.0 (TID 457). 8470 bytes result sent to driver -26/04/01 06:11:43 INFO TaskSetManager: Starting task 150.0 in stage 24.0 (TID 460) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:43 INFO Executor: Running task 150.0 in stage 24.0 (TID 460) -26/04/01 06:11:43 INFO TaskSetManager: Finished task 147.0 in stage 24.0 (TID 457) in 3409 ms on 10.0.0.133 (executor driver) (147/200) -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:43 INFO Executor: Finished task 146.0 in stage 24.0 (TID 456). 8470 bytes result sent to driver -26/04/01 06:11:43 INFO TaskSetManager: Starting task 151.0 in stage 24.0 (TID 461) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:43 INFO TaskSetManager: Finished task 146.0 in stage 24.0 (TID 456) in 3421 ms on 10.0.0.133 (executor driver) (148/200) -26/04/01 06:11:43 INFO Executor: Running task 151.0 in stage 24.0 (TID 461) -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:44 INFO Executor: Finished task 148.0 in stage 24.0 (TID 458). 8470 bytes result sent to driver -26/04/01 06:11:44 INFO TaskSetManager: Starting task 152.0 in stage 24.0 (TID 462) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:44 INFO Executor: Running task 152.0 in stage 24.0 (TID 462) -26/04/01 06:11:44 INFO TaskSetManager: Finished task 148.0 in stage 24.0 (TID 458) in 3359 ms on 10.0.0.133 (executor driver) (149/200) -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:44 INFO Executor: Finished task 149.0 in stage 24.0 (TID 459). 8470 bytes result sent to driver -26/04/01 06:11:44 INFO TaskSetManager: Starting task 153.0 in stage 24.0 (TID 463) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:44 INFO Executor: Running task 153.0 in stage 24.0 (TID 463) -26/04/01 06:11:44 INFO TaskSetManager: Finished task 149.0 in stage 24.0 (TID 459) in 3354 ms on 10.0.0.133 (executor driver) (150/200) -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:46 INFO Executor: Finished task 151.0 in stage 24.0 (TID 461). 9201 bytes result sent to driver -26/04/01 06:11:46 INFO TaskSetManager: Starting task 154.0 in stage 24.0 (TID 464) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:46 INFO Executor: Running task 154.0 in stage 24.0 (TID 464) -26/04/01 06:11:46 INFO TaskSetManager: Finished task 151.0 in stage 24.0 (TID 461) in 3402 ms on 10.0.0.133 (executor driver) (151/200) -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:46 INFO Executor: Finished task 150.0 in stage 24.0 (TID 460). 9201 bytes result sent to driver -26/04/01 06:11:46 INFO TaskSetManager: Starting task 155.0 in stage 24.0 (TID 465) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:46 INFO Executor: Running task 155.0 in stage 24.0 (TID 465) -26/04/01 06:11:46 INFO TaskSetManager: Finished task 150.0 in stage 24.0 (TID 460) in 3422 ms on 10.0.0.133 (executor driver) (152/200) -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:47 INFO Executor: Finished task 152.0 in stage 24.0 (TID 462). 9201 bytes result sent to driver -26/04/01 06:11:47 INFO TaskSetManager: Starting task 156.0 in stage 24.0 (TID 466) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:47 INFO Executor: Running task 156.0 in stage 24.0 (TID 466) -26/04/01 06:11:47 INFO TaskSetManager: Finished task 152.0 in stage 24.0 (TID 462) in 3379 ms on 10.0.0.133 (executor driver) (153/200) -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:47 INFO Executor: Finished task 153.0 in stage 24.0 (TID 463). 9201 bytes result sent to driver -26/04/01 06:11:47 INFO TaskSetManager: Starting task 157.0 in stage 24.0 (TID 467) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:47 INFO Executor: Running task 157.0 in stage 24.0 (TID 467) -26/04/01 06:11:47 INFO TaskSetManager: Finished task 153.0 in stage 24.0 (TID 463) in 3358 ms on 10.0.0.133 (executor driver) (154/200) -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:49 INFO Executor: Finished task 155.0 in stage 24.0 (TID 465). 8470 bytes result sent to driver -26/04/01 06:11:49 INFO TaskSetManager: Starting task 158.0 in stage 24.0 (TID 468) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:49 INFO Executor: Running task 158.0 in stage 24.0 (TID 468) -26/04/01 06:11:49 INFO TaskSetManager: Finished task 155.0 in stage 24.0 (TID 465) in 3456 ms on 10.0.0.133 (executor driver) (155/200) -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:49 INFO Executor: Finished task 154.0 in stage 24.0 (TID 464). 8470 bytes result sent to driver -26/04/01 06:11:49 INFO TaskSetManager: Starting task 159.0 in stage 24.0 (TID 469) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:49 INFO Executor: Running task 159.0 in stage 24.0 (TID 469) -26/04/01 06:11:49 INFO TaskSetManager: Finished task 154.0 in stage 24.0 (TID 464) in 3475 ms on 10.0.0.133 (executor driver) (156/200) -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:50 INFO Executor: Finished task 156.0 in stage 24.0 (TID 466). 8470 bytes result sent to driver -26/04/01 06:11:50 INFO TaskSetManager: Starting task 160.0 in stage 24.0 (TID 470) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:50 INFO Executor: Running task 160.0 in stage 24.0 (TID 470) -26/04/01 06:11:50 INFO TaskSetManager: Finished task 156.0 in stage 24.0 (TID 466) in 3361 ms on 10.0.0.133 (executor driver) (157/200) -26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:51 INFO Executor: Finished task 157.0 in stage 24.0 (TID 467). 8470 bytes result sent to driver -26/04/01 06:11:51 INFO TaskSetManager: Starting task 161.0 in stage 24.0 (TID 471) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:51 INFO Executor: Running task 161.0 in stage 24.0 (TID 471) -26/04/01 06:11:51 INFO TaskSetManager: Finished task 157.0 in stage 24.0 (TID 467) in 3354 ms on 10.0.0.133 (executor driver) (158/200) -26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:53 INFO Executor: Finished task 159.0 in stage 24.0 (TID 469). 8470 bytes result sent to driver -26/04/01 06:11:53 INFO TaskSetManager: Starting task 162.0 in stage 24.0 (TID 472) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:53 INFO Executor: Running task 162.0 in stage 24.0 (TID 472) -26/04/01 06:11:53 INFO TaskSetManager: Finished task 159.0 in stage 24.0 (TID 469) in 3401 ms on 10.0.0.133 (executor driver) (159/200) -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:53 INFO Executor: Finished task 158.0 in stage 24.0 (TID 468). 8470 bytes result sent to driver -26/04/01 06:11:53 INFO TaskSetManager: Starting task 163.0 in stage 24.0 (TID 473) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:53 INFO Executor: Running task 163.0 in stage 24.0 (TID 473) -26/04/01 06:11:53 INFO TaskSetManager: Finished task 158.0 in stage 24.0 (TID 468) in 3422 ms on 10.0.0.133 (executor driver) (160/200) -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:54 INFO Executor: Finished task 160.0 in stage 24.0 (TID 470). 8470 bytes result sent to driver -26/04/01 06:11:54 INFO TaskSetManager: Starting task 164.0 in stage 24.0 (TID 474) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:54 INFO Executor: Running task 164.0 in stage 24.0 (TID 474) -26/04/01 06:11:54 INFO TaskSetManager: Finished task 160.0 in stage 24.0 (TID 470) in 3348 ms on 10.0.0.133 (executor driver) (161/200) -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:54 INFO Executor: Finished task 161.0 in stage 24.0 (TID 471). 8470 bytes result sent to driver -26/04/01 06:11:54 INFO TaskSetManager: Starting task 165.0 in stage 24.0 (TID 475) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:54 INFO Executor: Running task 165.0 in stage 24.0 (TID 475) -26/04/01 06:11:54 INFO TaskSetManager: Finished task 161.0 in stage 24.0 (TID 471) in 3352 ms on 10.0.0.133 (executor driver) (162/200) -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:56 INFO Executor: Finished task 163.0 in stage 24.0 (TID 473). 9201 bytes result sent to driver -26/04/01 06:11:56 INFO TaskSetManager: Starting task 166.0 in stage 24.0 (TID 476) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:56 INFO Executor: Running task 166.0 in stage 24.0 (TID 476) -26/04/01 06:11:56 INFO TaskSetManager: Finished task 163.0 in stage 24.0 (TID 473) in 3404 ms on 10.0.0.133 (executor driver) (163/200) -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:56 INFO Executor: Finished task 162.0 in stage 24.0 (TID 472). 9201 bytes result sent to driver -26/04/01 06:11:56 INFO TaskSetManager: Starting task 167.0 in stage 24.0 (TID 477) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:56 INFO Executor: Running task 167.0 in stage 24.0 (TID 477) -26/04/01 06:11:56 INFO TaskSetManager: Finished task 162.0 in stage 24.0 (TID 472) in 3420 ms on 10.0.0.133 (executor driver) (164/200) -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:57 INFO Executor: Finished task 164.0 in stage 24.0 (TID 474). 9201 bytes result sent to driver -26/04/01 06:11:57 INFO TaskSetManager: Starting task 168.0 in stage 24.0 (TID 478) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:57 INFO Executor: Running task 168.0 in stage 24.0 (TID 478) -26/04/01 06:11:57 INFO TaskSetManager: Finished task 164.0 in stage 24.0 (TID 474) in 3345 ms on 10.0.0.133 (executor driver) (165/200) -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:57 INFO Executor: Finished task 165.0 in stage 24.0 (TID 475). 9201 bytes result sent to driver -26/04/01 06:11:57 INFO TaskSetManager: Starting task 169.0 in stage 24.0 (TID 479) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9428 bytes) -26/04/01 06:11:57 INFO Executor: Running task 169.0 in stage 24.0 (TID 479) -26/04/01 06:11:57 INFO TaskSetManager: Finished task 165.0 in stage 24.0 (TID 475) in 3348 ms on 10.0.0.133 (executor driver) (166/200) -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:11:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:11:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:11:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:00 INFO Executor: Finished task 166.0 in stage 24.0 (TID 476). 8470 bytes result sent to driver -26/04/01 06:12:00 INFO TaskSetManager: Starting task 170.0 in stage 24.0 (TID 480) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:00 INFO Executor: Running task 170.0 in stage 24.0 (TID 480) -26/04/01 06:12:00 INFO TaskSetManager: Finished task 166.0 in stage 24.0 (TID 476) in 3452 ms on 10.0.0.133 (executor driver) (167/200) -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:00 INFO Executor: Finished task 167.0 in stage 24.0 (TID 477). 8470 bytes result sent to driver -26/04/01 06:12:00 INFO TaskSetManager: Starting task 171.0 in stage 24.0 (TID 481) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:00 INFO Executor: Running task 171.0 in stage 24.0 (TID 481) -26/04/01 06:12:00 INFO TaskSetManager: Finished task 167.0 in stage 24.0 (TID 477) in 3453 ms on 10.0.0.133 (executor driver) (168/200) -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:00 INFO Executor: Finished task 168.0 in stage 24.0 (TID 478). 8470 bytes result sent to driver -26/04/01 06:12:00 INFO TaskSetManager: Starting task 172.0 in stage 24.0 (TID 482) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:00 INFO Executor: Running task 172.0 in stage 24.0 (TID 482) -26/04/01 06:12:00 INFO TaskSetManager: Finished task 168.0 in stage 24.0 (TID 478) in 3403 ms on 10.0.0.133 (executor driver) (169/200) -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:01 INFO Executor: Finished task 169.0 in stage 24.0 (TID 479). 8470 bytes result sent to driver -26/04/01 06:12:01 INFO TaskSetManager: Starting task 173.0 in stage 24.0 (TID 483) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:01 INFO Executor: Running task 173.0 in stage 24.0 (TID 483) -26/04/01 06:12:01 INFO TaskSetManager: Finished task 169.0 in stage 24.0 (TID 479) in 3387 ms on 10.0.0.133 (executor driver) (170/200) -26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:03 INFO Executor: Finished task 171.0 in stage 24.0 (TID 481). 8470 bytes result sent to driver -26/04/01 06:12:03 INFO TaskSetManager: Starting task 174.0 in stage 24.0 (TID 484) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:03 INFO Executor: Running task 174.0 in stage 24.0 (TID 484) -26/04/01 06:12:03 INFO TaskSetManager: Finished task 171.0 in stage 24.0 (TID 481) in 3420 ms on 10.0.0.133 (executor driver) (171/200) -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:03 INFO Executor: Finished task 170.0 in stage 24.0 (TID 480). 8470 bytes result sent to driver -26/04/01 06:12:03 INFO TaskSetManager: Starting task 175.0 in stage 24.0 (TID 485) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:03 INFO Executor: Running task 175.0 in stage 24.0 (TID 485) -26/04/01 06:12:03 INFO TaskSetManager: Finished task 170.0 in stage 24.0 (TID 480) in 3432 ms on 10.0.0.133 (executor driver) (172/200) -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:04 INFO Executor: Finished task 172.0 in stage 24.0 (TID 482). 8470 bytes result sent to driver -26/04/01 06:12:04 INFO TaskSetManager: Starting task 176.0 in stage 24.0 (TID 486) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:04 INFO Executor: Running task 176.0 in stage 24.0 (TID 486) -26/04/01 06:12:04 INFO TaskSetManager: Finished task 172.0 in stage 24.0 (TID 482) in 3423 ms on 10.0.0.133 (executor driver) (173/200) -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:04 INFO Executor: Finished task 173.0 in stage 24.0 (TID 483). 8470 bytes result sent to driver -26/04/01 06:12:04 INFO TaskSetManager: Starting task 177.0 in stage 24.0 (TID 487) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:04 INFO Executor: Running task 177.0 in stage 24.0 (TID 487) -26/04/01 06:12:04 INFO TaskSetManager: Finished task 173.0 in stage 24.0 (TID 483) in 3371 ms on 10.0.0.133 (executor driver) (174/200) -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:07 INFO Executor: Finished task 175.0 in stage 24.0 (TID 485). 9201 bytes result sent to driver -26/04/01 06:12:07 INFO TaskSetManager: Starting task 178.0 in stage 24.0 (TID 488) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:07 INFO TaskSetManager: Finished task 175.0 in stage 24.0 (TID 485) in 3420 ms on 10.0.0.133 (executor driver) (175/200) -26/04/01 06:12:07 INFO Executor: Running task 178.0 in stage 24.0 (TID 488) -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:07 INFO Executor: Finished task 174.0 in stage 24.0 (TID 484). 9201 bytes result sent to driver -26/04/01 06:12:07 INFO TaskSetManager: Starting task 179.0 in stage 24.0 (TID 489) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:07 INFO Executor: Running task 179.0 in stage 24.0 (TID 489) -26/04/01 06:12:07 INFO TaskSetManager: Finished task 174.0 in stage 24.0 (TID 484) in 3438 ms on 10.0.0.133 (executor driver) (176/200) -26/04/01 06:12:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:07 INFO Executor: Finished task 176.0 in stage 24.0 (TID 486). 9201 bytes result sent to driver -26/04/01 06:12:07 INFO TaskSetManager: Starting task 180.0 in stage 24.0 (TID 490) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:07 INFO Executor: Running task 180.0 in stage 24.0 (TID 490) -26/04/01 06:12:07 INFO TaskSetManager: Finished task 176.0 in stage 24.0 (TID 486) in 3429 ms on 10.0.0.133 (executor driver) (177/200) -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:08 INFO Executor: Finished task 177.0 in stage 24.0 (TID 487). 9201 bytes result sent to driver -26/04/01 06:12:08 INFO TaskSetManager: Starting task 181.0 in stage 24.0 (TID 491) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:08 INFO Executor: Running task 181.0 in stage 24.0 (TID 491) -26/04/01 06:12:08 INFO TaskSetManager: Finished task 177.0 in stage 24.0 (TID 487) in 3370 ms on 10.0.0.133 (executor driver) (178/200) -26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:10 INFO Executor: Finished task 179.0 in stage 24.0 (TID 489). 8470 bytes result sent to driver -26/04/01 06:12:10 INFO TaskSetManager: Starting task 182.0 in stage 24.0 (TID 492) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:10 INFO Executor: Running task 182.0 in stage 24.0 (TID 492) -26/04/01 06:12:10 INFO TaskSetManager: Finished task 179.0 in stage 24.0 (TID 489) in 3423 ms on 10.0.0.133 (executor driver) (179/200) -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:10 INFO Executor: Finished task 178.0 in stage 24.0 (TID 488). 8470 bytes result sent to driver -26/04/01 06:12:10 INFO TaskSetManager: Starting task 183.0 in stage 24.0 (TID 493) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:10 INFO Executor: Running task 183.0 in stage 24.0 (TID 493) -26/04/01 06:12:10 INFO TaskSetManager: Finished task 178.0 in stage 24.0 (TID 488) in 3435 ms on 10.0.0.133 (executor driver) (180/200) -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:11 INFO Executor: Finished task 180.0 in stage 24.0 (TID 490). 8470 bytes result sent to driver -26/04/01 06:12:11 INFO TaskSetManager: Starting task 184.0 in stage 24.0 (TID 494) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:11 INFO Executor: Running task 184.0 in stage 24.0 (TID 494) -26/04/01 06:12:11 INFO TaskSetManager: Finished task 180.0 in stage 24.0 (TID 490) in 3420 ms on 10.0.0.133 (executor driver) (181/200) -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:11 INFO Executor: Finished task 181.0 in stage 24.0 (TID 491). 8470 bytes result sent to driver -26/04/01 06:12:11 INFO TaskSetManager: Starting task 185.0 in stage 24.0 (TID 495) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:11 INFO Executor: Running task 185.0 in stage 24.0 (TID 495) -26/04/01 06:12:11 INFO TaskSetManager: Finished task 181.0 in stage 24.0 (TID 491) in 3373 ms on 10.0.0.133 (executor driver) (182/200) -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:13 INFO Executor: Finished task 183.0 in stage 24.0 (TID 493). 8470 bytes result sent to driver -26/04/01 06:12:13 INFO TaskSetManager: Starting task 186.0 in stage 24.0 (TID 496) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:13 INFO Executor: Running task 186.0 in stage 24.0 (TID 496) -26/04/01 06:12:13 INFO TaskSetManager: Finished task 183.0 in stage 24.0 (TID 493) in 3428 ms on 10.0.0.133 (executor driver) (183/200) -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:13 INFO Executor: Finished task 182.0 in stage 24.0 (TID 492). 8470 bytes result sent to driver -26/04/01 06:12:13 INFO TaskSetManager: Starting task 187.0 in stage 24.0 (TID 497) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:13 INFO TaskSetManager: Finished task 182.0 in stage 24.0 (TID 492) in 3435 ms on 10.0.0.133 (executor driver) (184/200) -26/04/01 06:12:13 INFO Executor: Running task 187.0 in stage 24.0 (TID 497) -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:14 INFO Executor: Finished task 184.0 in stage 24.0 (TID 494). 8470 bytes result sent to driver -26/04/01 06:12:14 INFO TaskSetManager: Starting task 188.0 in stage 24.0 (TID 498) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:14 INFO Executor: Running task 188.0 in stage 24.0 (TID 498) -26/04/01 06:12:14 INFO TaskSetManager: Finished task 184.0 in stage 24.0 (TID 494) in 3373 ms on 10.0.0.133 (executor driver) (185/200) -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:14 INFO Executor: Finished task 185.0 in stage 24.0 (TID 495). 8470 bytes result sent to driver -26/04/01 06:12:14 INFO TaskSetManager: Starting task 189.0 in stage 24.0 (TID 499) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:14 INFO TaskSetManager: Finished task 185.0 in stage 24.0 (TID 495) in 3384 ms on 10.0.0.133 (executor driver) (186/200) -26/04/01 06:12:14 INFO Executor: Running task 189.0 in stage 24.0 (TID 499) -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:17 INFO Executor: Finished task 187.0 in stage 24.0 (TID 497). 9201 bytes result sent to driver -26/04/01 06:12:17 INFO TaskSetManager: Starting task 190.0 in stage 24.0 (TID 500) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:17 INFO Executor: Running task 190.0 in stage 24.0 (TID 500) -26/04/01 06:12:17 INFO TaskSetManager: Finished task 187.0 in stage 24.0 (TID 497) in 3431 ms on 10.0.0.133 (executor driver) (187/200) -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:17 INFO Executor: Finished task 186.0 in stage 24.0 (TID 496). 9201 bytes result sent to driver -26/04/01 06:12:17 INFO TaskSetManager: Starting task 191.0 in stage 24.0 (TID 501) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:17 INFO Executor: Running task 191.0 in stage 24.0 (TID 501) -26/04/01 06:12:17 INFO TaskSetManager: Finished task 186.0 in stage 24.0 (TID 496) in 3450 ms on 10.0.0.133 (executor driver) (188/200) -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:17 INFO Executor: Finished task 188.0 in stage 24.0 (TID 498). 9201 bytes result sent to driver -26/04/01 06:12:17 INFO TaskSetManager: Starting task 192.0 in stage 24.0 (TID 502) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:17 INFO Executor: Running task 192.0 in stage 24.0 (TID 502) -26/04/01 06:12:17 INFO TaskSetManager: Finished task 188.0 in stage 24.0 (TID 498) in 3344 ms on 10.0.0.133 (executor driver) (189/200) -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:18 INFO Executor: Finished task 189.0 in stage 24.0 (TID 499). 9201 bytes result sent to driver -26/04/01 06:12:18 INFO TaskSetManager: Starting task 193.0 in stage 24.0 (TID 503) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:18 INFO Executor: Running task 193.0 in stage 24.0 (TID 503) -26/04/01 06:12:18 INFO TaskSetManager: Finished task 189.0 in stage 24.0 (TID 499) in 3357 ms on 10.0.0.133 (executor driver) (190/200) -26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:20 INFO Executor: Finished task 191.0 in stage 24.0 (TID 501). 8470 bytes result sent to driver -26/04/01 06:12:20 INFO TaskSetManager: Starting task 194.0 in stage 24.0 (TID 504) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:20 INFO Executor: Running task 194.0 in stage 24.0 (TID 504) -26/04/01 06:12:20 INFO TaskSetManager: Finished task 191.0 in stage 24.0 (TID 501) in 3399 ms on 10.0.0.133 (executor driver) (191/200) -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:20 INFO Executor: Finished task 190.0 in stage 24.0 (TID 500). 8470 bytes result sent to driver -26/04/01 06:12:20 INFO TaskSetManager: Starting task 195.0 in stage 24.0 (TID 505) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:20 INFO Executor: Running task 195.0 in stage 24.0 (TID 505) -26/04/01 06:12:20 INFO TaskSetManager: Finished task 190.0 in stage 24.0 (TID 500) in 3423 ms on 10.0.0.133 (executor driver) (192/200) -26/04/01 06:12:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:21 INFO Executor: Finished task 192.0 in stage 24.0 (TID 502). 8470 bytes result sent to driver -26/04/01 06:12:21 INFO TaskSetManager: Starting task 196.0 in stage 24.0 (TID 506) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:21 INFO Executor: Running task 196.0 in stage 24.0 (TID 506) -26/04/01 06:12:21 INFO TaskSetManager: Finished task 192.0 in stage 24.0 (TID 502) in 3359 ms on 10.0.0.133 (executor driver) (193/200) -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:21 INFO Executor: Finished task 193.0 in stage 24.0 (TID 503). 8470 bytes result sent to driver -26/04/01 06:12:21 INFO TaskSetManager: Starting task 197.0 in stage 24.0 (TID 507) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:21 INFO Executor: Running task 197.0 in stage 24.0 (TID 507) -26/04/01 06:12:21 INFO TaskSetManager: Finished task 193.0 in stage 24.0 (TID 503) in 3362 ms on 10.0.0.133 (executor driver) (194/200) -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:24 INFO Executor: Finished task 195.0 in stage 24.0 (TID 505). 8470 bytes result sent to driver -26/04/01 06:12:24 INFO TaskSetManager: Starting task 198.0 in stage 24.0 (TID 508) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:24 INFO Executor: Running task 198.0 in stage 24.0 (TID 508) -26/04/01 06:12:24 INFO TaskSetManager: Finished task 195.0 in stage 24.0 (TID 505) in 3416 ms on 10.0.0.133 (executor driver) (195/200) -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:24 INFO Executor: Finished task 194.0 in stage 24.0 (TID 504). 8470 bytes result sent to driver -26/04/01 06:12:24 INFO TaskSetManager: Starting task 199.0 in stage 24.0 (TID 509) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9428 bytes) -26/04/01 06:12:24 INFO TaskSetManager: Finished task 194.0 in stage 24.0 (TID 504) in 3439 ms on 10.0.0.133 (executor driver) (196/200) -26/04/01 06:12:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:24 INFO Executor: Running task 199.0 in stage 24.0 (TID 509) -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:24 INFO Executor: Finished task 196.0 in stage 24.0 (TID 506). 8470 bytes result sent to driver -26/04/01 06:12:24 INFO TaskSetManager: Finished task 196.0 in stage 24.0 (TID 506) in 3363 ms on 10.0.0.133 (executor driver) (197/200) -26/04/01 06:12:24 INFO Executor: Finished task 197.0 in stage 24.0 (TID 507). 8470 bytes result sent to driver -26/04/01 06:12:24 INFO TaskSetManager: Finished task 197.0 in stage 24.0 (TID 507) in 3351 ms on 10.0.0.133 (executor driver) (198/200) -26/04/01 06:12:27 INFO Executor: Finished task 199.0 in stage 24.0 (TID 509). 9201 bytes result sent to driver -26/04/01 06:12:27 INFO TaskSetManager: Finished task 199.0 in stage 24.0 (TID 509) in 3184 ms on 10.0.0.133 (executor driver) (199/200) -26/04/01 06:12:27 INFO Executor: Finished task 198.0 in stage 24.0 (TID 508). 9201 bytes result sent to driver -26/04/01 06:12:27 INFO TaskSetManager: Finished task 198.0 in stage 24.0 (TID 508) in 3202 ms on 10.0.0.133 (executor driver) (200/200) -26/04/01 06:12:27 INFO TaskSchedulerImpl: Removed TaskSet 24.0, whose tasks have all completed, from pool -26/04/01 06:12:27 INFO DAGScheduler: ShuffleMapStage 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 172.132 s -26/04/01 06:12:27 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:12:27 INFO DAGScheduler: running: Set() -26/04/01 06:12:27 INFO DAGScheduler: waiting: Set() -26/04/01 06:12:27 INFO DAGScheduler: failed: Set() -26/04/01 06:12:27 INFO ShufflePartitionsUtil: For shuffle(6, 4), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 06:12:27 INFO DAGScheduler: Registering RDD 54 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 7 -26/04/01 06:12:27 INFO DAGScheduler: Got map stage job 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 25 output partitions -26/04/01 06:12:27 INFO DAGScheduler: Final stage: ShuffleMapStage 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:12:27 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 30, ShuffleMapStage 29) -26/04/01 06:12:27 INFO DAGScheduler: Missing parents: List() -26/04/01 06:12:27 INFO DAGScheduler: Submitting ShuffleMapStage 31 (MapPartitionsRDD[54] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:12:27 INFO MemoryStore: Block broadcast_27 stored as values in memory (estimated size 39.3 KiB, free 8.6 GiB) -26/04/01 06:12:27 INFO MemoryStore: Block broadcast_27_piece0 stored as bytes in memory (estimated size 12.3 KiB, free 8.6 GiB) -26/04/01 06:12:27 INFO BlockManagerInfo: Added broadcast_27_piece0 in memory on 10.0.0.133:58259 (size: 12.3 KiB, free: 8.6 GiB) -26/04/01 06:12:27 INFO SparkContext: Created broadcast 27 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:12:27 INFO DAGScheduler: Submitting 25 missing tasks from ShuffleMapStage 31 (MapPartitionsRDD[54] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 06:12:27 INFO TaskSchedulerImpl: Adding task set 31.0 with 25 tasks resource profile 0 -26/04/01 06:12:27 INFO TaskSetManager: Starting task 0.0 in stage 31.0 (TID 510) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:27 INFO TaskSetManager: Starting task 1.0 in stage 31.0 (TID 511) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:27 INFO TaskSetManager: Starting task 2.0 in stage 31.0 (TID 512) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:27 INFO TaskSetManager: Starting task 3.0 in stage 31.0 (TID 513) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:27 INFO Executor: Running task 0.0 in stage 31.0 (TID 510) -26/04/01 06:12:27 INFO Executor: Running task 2.0 in stage 31.0 (TID 512) -26/04/01 06:12:27 INFO Executor: Running task 3.0 in stage 31.0 (TID 513) -26/04/01 06:12:27 INFO Executor: Running task 1.0 in stage 31.0 (TID 511) -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (320.9 KiB) non-empty blocks including 4 (320.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (320.9 KiB) non-empty blocks including 4 (320.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:36 INFO Executor: Finished task 3.0 in stage 31.0 (TID 513). 11071 bytes result sent to driver -26/04/01 06:12:36 INFO TaskSetManager: Starting task 4.0 in stage 31.0 (TID 514) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:36 INFO TaskSetManager: Finished task 3.0 in stage 31.0 (TID 513) in 8867 ms on 10.0.0.133 (executor driver) (1/25) -26/04/01 06:12:36 INFO Executor: Running task 4.0 in stage 31.0 (TID 514) -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:36 INFO Executor: Finished task 1.0 in stage 31.0 (TID 511). 11071 bytes result sent to driver -26/04/01 06:12:36 INFO TaskSetManager: Starting task 5.0 in stage 31.0 (TID 515) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:36 INFO Executor: Running task 5.0 in stage 31.0 (TID 515) -26/04/01 06:12:36 INFO TaskSetManager: Finished task 1.0 in stage 31.0 (TID 511) in 8885 ms on 10.0.0.133 (executor driver) (2/25) -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:36 INFO Executor: Finished task 0.0 in stage 31.0 (TID 510). 11071 bytes result sent to driver -26/04/01 06:12:36 INFO TaskSetManager: Starting task 6.0 in stage 31.0 (TID 516) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:36 INFO Executor: Running task 6.0 in stage 31.0 (TID 516) -26/04/01 06:12:36 INFO TaskSetManager: Finished task 0.0 in stage 31.0 (TID 510) in 8900 ms on 10.0.0.133 (executor driver) (3/25) -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO Executor: Finished task 2.0 in stage 31.0 (TID 512). 11071 bytes result sent to driver -26/04/01 06:12:36 INFO TaskSetManager: Starting task 7.0 in stage 31.0 (TID 517) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:36 INFO TaskSetManager: Finished task 2.0 in stage 31.0 (TID 512) in 8905 ms on 10.0.0.133 (executor driver) (4/25) -26/04/01 06:12:36 INFO Executor: Running task 7.0 in stage 31.0 (TID 517) -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:45 INFO Executor: Finished task 4.0 in stage 31.0 (TID 514). 10340 bytes result sent to driver -26/04/01 06:12:45 INFO TaskSetManager: Starting task 8.0 in stage 31.0 (TID 518) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:45 INFO Executor: Running task 8.0 in stage 31.0 (TID 518) -26/04/01 06:12:45 INFO TaskSetManager: Finished task 4.0 in stage 31.0 (TID 514) in 8783 ms on 10.0.0.133 (executor driver) (5/25) -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:45 INFO Executor: Finished task 5.0 in stage 31.0 (TID 515). 10340 bytes result sent to driver -26/04/01 06:12:45 INFO TaskSetManager: Starting task 9.0 in stage 31.0 (TID 519) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:45 INFO Executor: Running task 9.0 in stage 31.0 (TID 519) -26/04/01 06:12:45 INFO TaskSetManager: Finished task 5.0 in stage 31.0 (TID 515) in 8814 ms on 10.0.0.133 (executor driver) (6/25) -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (317.2 KiB) non-empty blocks including 4 (317.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (317.2 KiB) non-empty blocks including 4 (317.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:45 INFO Executor: Finished task 6.0 in stage 31.0 (TID 516). 10340 bytes result sent to driver -26/04/01 06:12:45 INFO Executor: Finished task 7.0 in stage 31.0 (TID 517). 10340 bytes result sent to driver -26/04/01 06:12:45 INFO TaskSetManager: Starting task 10.0 in stage 31.0 (TID 520) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:45 INFO Executor: Running task 10.0 in stage 31.0 (TID 520) -26/04/01 06:12:45 INFO TaskSetManager: Starting task 11.0 in stage 31.0 (TID 521) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:45 INFO TaskSetManager: Finished task 6.0 in stage 31.0 (TID 516) in 8837 ms on 10.0.0.133 (executor driver) (7/25) -26/04/01 06:12:45 INFO Executor: Running task 11.0 in stage 31.0 (TID 521) -26/04/01 06:12:45 INFO TaskSetManager: Finished task 7.0 in stage 31.0 (TID 517) in 8832 ms on 10.0.0.133 (executor driver) (8/25) -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:53 INFO Executor: Finished task 8.0 in stage 31.0 (TID 518). 11071 bytes result sent to driver -26/04/01 06:12:53 INFO TaskSetManager: Starting task 12.0 in stage 31.0 (TID 522) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:53 INFO Executor: Running task 12.0 in stage 31.0 (TID 522) -26/04/01 06:12:53 INFO TaskSetManager: Finished task 8.0 in stage 31.0 (TID 518) in 8774 ms on 10.0.0.133 (executor driver) (9/25) -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:53 INFO Executor: Finished task 9.0 in stage 31.0 (TID 519). 11071 bytes result sent to driver -26/04/01 06:12:53 INFO TaskSetManager: Starting task 13.0 in stage 31.0 (TID 523) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:53 INFO TaskSetManager: Finished task 9.0 in stage 31.0 (TID 519) in 8745 ms on 10.0.0.133 (executor driver) (10/25) -26/04/01 06:12:53 INFO Executor: Running task 13.0 in stage 31.0 (TID 523) -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (322.7 KiB) non-empty blocks including 4 (322.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (322.7 KiB) non-empty blocks including 4 (322.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:53 INFO Executor: Finished task 11.0 in stage 31.0 (TID 521). 11071 bytes result sent to driver -26/04/01 06:12:53 INFO TaskSetManager: Starting task 14.0 in stage 31.0 (TID 524) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:53 INFO Executor: Running task 14.0 in stage 31.0 (TID 524) -26/04/01 06:12:53 INFO TaskSetManager: Finished task 11.0 in stage 31.0 (TID 521) in 8755 ms on 10.0.0.133 (executor driver) (11/25) -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO Executor: Finished task 10.0 in stage 31.0 (TID 520). 11071 bytes result sent to driver -26/04/01 06:12:53 INFO TaskSetManager: Starting task 15.0 in stage 31.0 (TID 525) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9719 bytes) -26/04/01 06:12:53 INFO Executor: Running task 15.0 in stage 31.0 (TID 525) -26/04/01 06:12:53 INFO TaskSetManager: Finished task 10.0 in stage 31.0 (TID 520) in 8763 ms on 10.0.0.133 (executor driver) (12/25) -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:12:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:12:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:02 INFO Executor: Finished task 12.0 in stage 31.0 (TID 522). 11071 bytes result sent to driver -26/04/01 06:13:02 INFO TaskSetManager: Starting task 16.0 in stage 31.0 (TID 526) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9719 bytes) -26/04/01 06:13:02 INFO Executor: Running task 16.0 in stage 31.0 (TID 526) -26/04/01 06:13:02 INFO TaskSetManager: Finished task 12.0 in stage 31.0 (TID 522) in 8748 ms on 10.0.0.133 (executor driver) (13/25) -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:02 INFO Executor: Finished task 13.0 in stage 31.0 (TID 523). 11071 bytes result sent to driver -26/04/01 06:13:02 INFO TaskSetManager: Starting task 17.0 in stage 31.0 (TID 527) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9719 bytes) -26/04/01 06:13:02 INFO TaskSetManager: Finished task 13.0 in stage 31.0 (TID 523) in 8774 ms on 10.0.0.133 (executor driver) (14/25) -26/04/01 06:13:02 INFO Executor: Running task 17.0 in stage 31.0 (TID 527) -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:02 INFO Executor: Finished task 15.0 in stage 31.0 (TID 525). 11071 bytes result sent to driver -26/04/01 06:13:02 INFO TaskSetManager: Starting task 18.0 in stage 31.0 (TID 528) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9719 bytes) -26/04/01 06:13:02 INFO Executor: Running task 18.0 in stage 31.0 (TID 528) -26/04/01 06:13:02 INFO TaskSetManager: Finished task 15.0 in stage 31.0 (TID 525) in 8795 ms on 10.0.0.133 (executor driver) (15/25) -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:02 INFO Executor: Finished task 14.0 in stage 31.0 (TID 524). 11071 bytes result sent to driver -26/04/01 06:13:02 INFO TaskSetManager: Starting task 19.0 in stage 31.0 (TID 529) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9719 bytes) -26/04/01 06:13:02 INFO TaskSetManager: Finished task 14.0 in stage 31.0 (TID 524) in 8816 ms on 10.0.0.133 (executor driver) (16/25) -26/04/01 06:13:02 INFO Executor: Running task 19.0 in stage 31.0 (TID 529) -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:11 INFO Executor: Finished task 16.0 in stage 31.0 (TID 526). 11071 bytes result sent to driver -26/04/01 06:13:11 INFO TaskSetManager: Starting task 20.0 in stage 31.0 (TID 530) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9719 bytes) -26/04/01 06:13:11 INFO Executor: Running task 20.0 in stage 31.0 (TID 530) -26/04/01 06:13:11 INFO TaskSetManager: Finished task 16.0 in stage 31.0 (TID 526) in 8752 ms on 10.0.0.133 (executor driver) (17/25) -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:11 INFO Executor: Finished task 17.0 in stage 31.0 (TID 527). 11071 bytes result sent to driver -26/04/01 06:13:11 INFO TaskSetManager: Starting task 21.0 in stage 31.0 (TID 531) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9719 bytes) -26/04/01 06:13:11 INFO Executor: Running task 21.0 in stage 31.0 (TID 531) -26/04/01 06:13:11 INFO TaskSetManager: Finished task 17.0 in stage 31.0 (TID 527) in 8756 ms on 10.0.0.133 (executor driver) (18/25) -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:11 INFO Executor: Finished task 18.0 in stage 31.0 (TID 528). 11071 bytes result sent to driver -26/04/01 06:13:11 INFO TaskSetManager: Starting task 22.0 in stage 31.0 (TID 532) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9719 bytes) -26/04/01 06:13:11 INFO TaskSetManager: Finished task 18.0 in stage 31.0 (TID 528) in 8700 ms on 10.0.0.133 (executor driver) (19/25) -26/04/01 06:13:11 INFO Executor: Running task 22.0 in stage 31.0 (TID 532) -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:11 INFO Executor: Finished task 19.0 in stage 31.0 (TID 529). 11071 bytes result sent to driver -26/04/01 06:13:11 INFO TaskSetManager: Starting task 23.0 in stage 31.0 (TID 533) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9719 bytes) -26/04/01 06:13:11 INFO Executor: Running task 23.0 in stage 31.0 (TID 533) -26/04/01 06:13:11 INFO TaskSetManager: Finished task 19.0 in stage 31.0 (TID 529) in 8726 ms on 10.0.0.133 (executor driver) (20/25) -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (322.8 KiB) non-empty blocks including 4 (322.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Getting 4 (322.8 KiB) non-empty blocks including 4 (322.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:20 INFO Executor: Finished task 20.0 in stage 31.0 (TID 530). 11071 bytes result sent to driver -26/04/01 06:13:20 INFO TaskSetManager: Starting task 24.0 in stage 31.0 (TID 534) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9719 bytes) -26/04/01 06:13:20 INFO Executor: Running task 24.0 in stage 31.0 (TID 534) -26/04/01 06:13:20 INFO TaskSetManager: Finished task 20.0 in stage 31.0 (TID 530) in 8794 ms on 10.0.0.133 (executor driver) (21/25) -26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:20 INFO Executor: Finished task 21.0 in stage 31.0 (TID 531). 11071 bytes result sent to driver -26/04/01 06:13:20 INFO TaskSetManager: Finished task 21.0 in stage 31.0 (TID 531) in 8760 ms on 10.0.0.133 (executor driver) (22/25) -26/04/01 06:13:20 INFO Executor: Finished task 22.0 in stage 31.0 (TID 532). 11071 bytes result sent to driver -26/04/01 06:13:20 INFO TaskSetManager: Finished task 22.0 in stage 31.0 (TID 532) in 8754 ms on 10.0.0.133 (executor driver) (23/25) -26/04/01 06:13:20 INFO Executor: Finished task 23.0 in stage 31.0 (TID 533). 11071 bytes result sent to driver -26/04/01 06:13:20 INFO TaskSetManager: Finished task 23.0 in stage 31.0 (TID 533) in 8731 ms on 10.0.0.133 (executor driver) (24/25) -26/04/01 06:13:26 INFO Executor: Finished task 24.0 in stage 31.0 (TID 534). 11071 bytes result sent to driver -26/04/01 06:13:26 INFO TaskSetManager: Finished task 24.0 in stage 31.0 (TID 534) in 6142 ms on 10.0.0.133 (executor driver) (25/25) -26/04/01 06:13:26 INFO TaskSchedulerImpl: Removed TaskSet 31.0, whose tasks have all completed, from pool -26/04/01 06:13:26 INFO DAGScheduler: ShuffleMapStage 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 58.860 s -26/04/01 06:13:26 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:13:26 INFO DAGScheduler: running: Set() -26/04/01 06:13:26 INFO DAGScheduler: waiting: Set() -26/04/01 06:13:26 INFO DAGScheduler: failed: Set() -26/04/01 06:13:26 INFO ShufflePartitionsUtil: For shuffle(7), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 06:13:26 INFO CodeGenerator: Code generated in 56.085125 ms -26/04/01 06:13:26 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:13:26 INFO DAGScheduler: Got job 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:13:26 INFO DAGScheduler: Final stage: ResultStage 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:13:26 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 38) -26/04/01 06:13:26 INFO DAGScheduler: Missing parents: List() -26/04/01 06:13:26 INFO DAGScheduler: Submitting ResultStage 39 (MapPartitionsRDD[59] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:13:26 INFO MemoryStore: Block broadcast_28 stored as values in memory (estimated size 22.7 KiB, free 8.6 GiB) -26/04/01 06:13:26 INFO MemoryStore: Block broadcast_28_piece0 stored as bytes in memory (estimated size 8.4 KiB, free 8.6 GiB) -26/04/01 06:13:26 INFO BlockManagerInfo: Added broadcast_28_piece0 in memory on 10.0.0.133:58259 (size: 8.4 KiB, free: 8.6 GiB) -26/04/01 06:13:26 INFO SparkContext: Created broadcast 28 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:13:26 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 39 (MapPartitionsRDD[59] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:13:26 INFO TaskSchedulerImpl: Adding task set 39.0 with 1 tasks resource profile 0 -26/04/01 06:13:26 INFO TaskSetManager: Starting task 0.0 in stage 39.0 (TID 535) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) -26/04/01 06:13:26 INFO Executor: Running task 0.0 in stage 39.0 (TID 535) -26/04/01 06:13:26 INFO ShuffleBlockFetcherIterator: Getting 25 (54.3 KiB) non-empty blocks including 25 (54.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:26 INFO CodeGenerator: Code generated in 5.687333 ms -26/04/01 06:13:26 INFO Executor: Finished task 0.0 in stage 39.0 (TID 535). 5177 bytes result sent to driver -26/04/01 06:13:26 INFO TaskSetManager: Finished task 0.0 in stage 39.0 (TID 535) in 48 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:13:26 INFO TaskSchedulerImpl: Removed TaskSet 39.0, whose tasks have all completed, from pool -26/04/01 06:13:26 INFO DAGScheduler: ResultStage 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.049 s -26/04/01 06:13:26 INFO DAGScheduler: Job 19 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:13:26 INFO TaskSchedulerImpl: Killing all running tasks in stage 39: Stage finished -26/04/01 06:13:26 INFO DAGScheduler: Job 19 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.051501 s -26/04/01 06:13:26 INFO DAGScheduler: Registering RDD 60 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 8 -26/04/01 06:13:26 INFO DAGScheduler: Got map stage job 20 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:13:26 INFO DAGScheduler: Final stage: ShuffleMapStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:13:26 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 46) -26/04/01 06:13:26 INFO DAGScheduler: Missing parents: List() -26/04/01 06:13:26 INFO DAGScheduler: Submitting ShuffleMapStage 47 (MapPartitionsRDD[60] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:13:26 INFO MemoryStore: Block broadcast_29 stored as values in memory (estimated size 25.0 KiB, free 8.6 GiB) -26/04/01 06:13:26 INFO MemoryStore: Block broadcast_29_piece0 stored as bytes in memory (estimated size 9.3 KiB, free 8.6 GiB) -26/04/01 06:13:26 INFO BlockManagerInfo: Added broadcast_29_piece0 in memory on 10.0.0.133:58259 (size: 9.3 KiB, free: 8.6 GiB) -26/04/01 06:13:26 INFO SparkContext: Created broadcast 29 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:13:26 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 47 (MapPartitionsRDD[60] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:13:26 INFO TaskSchedulerImpl: Adding task set 47.0 with 1 tasks resource profile 0 -26/04/01 06:13:26 INFO TaskSetManager: Starting task 0.0 in stage 47.0 (TID 536) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9404 bytes) -26/04/01 06:13:26 INFO Executor: Running task 0.0 in stage 47.0 (TID 536) -26/04/01 06:13:26 INFO CodeGenerator: Code generated in 3.263208 ms -26/04/01 06:13:26 INFO ShuffleBlockFetcherIterator: Getting 25 (54.3 KiB) non-empty blocks including 25 (54.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:26 INFO Executor: Finished task 0.0 in stage 47.0 (TID 536). 6566 bytes result sent to driver -26/04/01 06:13:26 INFO TaskSetManager: Finished task 0.0 in stage 47.0 (TID 536) in 47 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:13:26 INFO TaskSchedulerImpl: Removed TaskSet 47.0, whose tasks have all completed, from pool -26/04/01 06:13:26 INFO DAGScheduler: ShuffleMapStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.051 s -26/04/01 06:13:26 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:13:26 INFO DAGScheduler: running: Set() -26/04/01 06:13:26 INFO DAGScheduler: waiting: Set() -26/04/01 06:13:26 INFO DAGScheduler: failed: Set() -26/04/01 06:13:26 INFO ShufflePartitionsUtil: For shuffle(8), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 06:13:26 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:13:26 INFO DAGScheduler: Got job 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:13:26 INFO DAGScheduler: Final stage: ResultStage 56 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:13:26 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 55) -26/04/01 06:13:26 INFO DAGScheduler: Missing parents: List() -26/04/01 06:13:26 INFO DAGScheduler: Submitting ResultStage 56 (MapPartitionsRDD[64] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:13:26 INFO MemoryStore: Block broadcast_30 stored as values in memory (estimated size 18.6 KiB, free 8.6 GiB) -26/04/01 06:13:26 INFO MemoryStore: Block broadcast_30_piece0 stored as bytes in memory (estimated size 8.8 KiB, free 8.6 GiB) -26/04/01 06:13:26 INFO BlockManagerInfo: Added broadcast_30_piece0 in memory on 10.0.0.133:58259 (size: 8.8 KiB, free: 8.6 GiB) -26/04/01 06:13:26 INFO SparkContext: Created broadcast 30 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:13:26 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 56 (MapPartitionsRDD[64] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:13:26 INFO TaskSchedulerImpl: Adding task set 56.0 with 1 tasks resource profile 0 -26/04/01 06:13:26 INFO TaskSetManager: Starting task 0.0 in stage 56.0 (TID 537) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) -26/04/01 06:13:26 INFO Executor: Running task 0.0 in stage 56.0 (TID 537) -26/04/01 06:13:26 INFO ShuffleBlockFetcherIterator: Getting 1 (1620.0 B) non-empty blocks including 1 (1620.0 B) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:13:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:13:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=1024 MB -26/04/01 06:13:26 INFO Executor: Finished task 0.0 in stage 56.0 (TID 537). 5280 bytes result sent to driver -26/04/01 06:13:26 INFO TaskSetManager: Finished task 0.0 in stage 56.0 (TID 537) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:13:26 INFO TaskSchedulerImpl: Removed TaskSet 56.0, whose tasks have all completed, from pool -26/04/01 06:13:26 INFO DAGScheduler: ResultStage 56 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.007 s -26/04/01 06:13:26 INFO DAGScheduler: Job 21 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:13:26 INFO TaskSchedulerImpl: Killing all running tasks in stage 56: Stage finished -26/04/01 06:13:26 INFO DAGScheduler: Job 21 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.008645 s -26/04/01 06:13:26 INFO SparkContext: SparkContext is stopping with exitCode 0. -26/04/01 06:13:26 INFO CometDriverPlugin: CometDriverPlugin shutdown -26/04/01 06:13:26 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! -26/04/01 06:13:26 INFO MemoryStore: MemoryStore cleared -26/04/01 06:13:26 INFO BlockManager: BlockManager stopped -26/04/01 06:13:26 INFO BlockManagerMaster: BlockManagerMaster stopped -26/04/01 06:13:26 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! -26/04/01 06:13:26 INFO SparkContext: Successfully stopped SparkContext -26/04/01 06:13:27 INFO ShutdownHookManager: Shutdown hook called -26/04/01 06:13:27 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-73a4ef89-b8ea-48f7-92a1-8593182866f3/pyspark-c6973fe6-e3dd-4d80-9cfd-2e99d29d47fe -26/04/01 06:13:27 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-74acaa95-d245-4d4b-ab22-d5a168d8ad37 -26/04/01 06:13:27 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-73a4ef89-b8ea-48f7-92a1-8593182866f3 - 488.48 real 2097.74 user 21.12 sys - 5803589632 maximum resident set size - 0 average shared memory size - 0 average unshared data size - 0 average unshared stack size - 380771 page reclaims - 36 page faults - 0 swaps - 0 block input operations - 0 block output operations - 1276 messages sent - 2361 messages received - 17 signals received - 36202 voluntary context switches - 773327 involuntary context switches - 33328818155221 instructions retired - 7704033477817 cycles elapsed - 2546407040 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.log b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.log deleted file mode 100644 index 0dec6c4c4b..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.log +++ /dev/null @@ -1,281 +0,0 @@ -Registering table customer from /opt/tpch/sf100/customer -Registering table lineitem from /opt/tpch/sf100/lineitem -Registering table nation from /opt/tpch/sf100/nation -Registering table orders from /opt/tpch/sf100/orders -Registering table part from /opt/tpch/sf100/part -Registering table partsupp from /opt/tpch/sf100/partsupp -Registering table region from /opt/tpch/sf100/region -Registering table supplier from /opt/tpch/sf100/supplier - -============================================================ -Starting iteration 1 of 1 -============================================================ - -Running query 9 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q9.sql -Executing: -- CometBench-H query 9 derived from TPC-H query 9 under the terms of the TPC Fair Use Policy. --- TP... -== Physical Plan == -AdaptiveSparkPlan (46) -+- CometSort (45) - +- CometExchange (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (35) - : +- CometSortMergeJoin (34) - : :- CometSort (29) - : : +- CometExchange (28) - : : +- CometProject (27) - : : +- CometSortMergeJoin (26) - : : :- CometSort (21) - : : : +- CometExchange (20) - : : : +- CometProject (19) - : : : +- CometSortMergeJoin (18) - : : : :- CometSort (13) - : : : : +- CometExchange (12) - : : : : +- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometFilter (7) - : : : : +- CometNativeScan parquet (6) - : : : +- CometSort (17) - : : : +- CometExchange (16) - : : : +- CometFilter (15) - : : : +- CometNativeScan parquet (14) - : : +- CometSort (25) - : : +- CometExchange (24) - : : +- CometFilter (23) - : : +- CometNativeScan parquet (22) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometFilter (31) - : +- CometNativeScan parquet (30) - +- CometBroadcastExchange (38) - +- CometFilter (37) - +- CometNativeScan parquet (36) - - -(1) CometNativeScan parquet -Output [2]: [p_partkey#74L, p_name#75] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/part] -PushedFilters: [IsNotNull(p_name), StringContains(p_name,moccasin), IsNotNull(p_partkey)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [p_partkey#74L, p_name#75] -Condition : ((isnotnull(p_name#75) AND Contains(p_name#75, moccasin)) AND isnotnull(p_partkey#74L)) - -(3) CometProject -Input [2]: [p_partkey#74L, p_name#75] -Arguments: [p_partkey#74L], [p_partkey#74L] - -(4) CometExchange -Input [1]: [p_partkey#74L] -Arguments: hashpartitioning(p_partkey#74L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=376] - -(5) CometSort -Input [1]: [p_partkey#74L] -Arguments: [p_partkey#74L], [p_partkey#74L ASC NULLS FIRST] - -(6) CometNativeScan parquet -Output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] -PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] -ReadSchema: struct - -(7) CometFilter -Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Condition : ((isnotnull(l_partkey#17L) AND isnotnull(l_suppkey#18L)) AND isnotnull(l_orderkey#16L)) - -(8) CometExchange -Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: hashpartitioning(l_partkey#17L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=380] - -(9) CometSort -Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22], [l_partkey#17L ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [1]: [p_partkey#74L] -Right output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: [p_partkey#74L], [l_partkey#17L], Inner - -(11) CometProject -Input [7]: [p_partkey#74L, l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22], [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] - -(12) CometExchange -Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: hashpartitioning(l_suppkey#18L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=384] - -(13) CometSort -Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22], [l_suppkey#18L ASC NULLS FIRST] - -(14) CometNativeScan parquet -Output [2]: [s_suppkey#108L, s_nationkey#111L] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/supplier] -PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Condition : (isnotnull(s_suppkey#108L) AND isnotnull(s_nationkey#111L)) - -(16) CometExchange -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: hashpartitioning(s_suppkey#108L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=388] - -(17) CometSort -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: [s_suppkey#108L, s_nationkey#111L], [s_suppkey#108L ASC NULLS FIRST] - -(18) CometSortMergeJoin -Left output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Right output [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: [l_suppkey#18L], [s_suppkey#108L], Inner - -(19) CometProject -Input [8]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_suppkey#108L, s_nationkey#111L] -Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L], [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] - -(20) CometExchange -Input [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] -Arguments: hashpartitioning(l_suppkey#18L, l_partkey#17L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=392] - -(21) CometSort -Input [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] -Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L], [l_suppkey#18L ASC NULLS FIRST, l_partkey#17L ASC NULLS FIRST] - -(22) CometNativeScan parquet -Output [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/partsupp] -PushedFilters: [IsNotNull(ps_suppkey), IsNotNull(ps_partkey)] -ReadSchema: struct - -(23) CometFilter -Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Condition : (isnotnull(ps_suppkey#93L) AND isnotnull(ps_partkey#92L)) - -(24) CometExchange -Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Arguments: hashpartitioning(ps_suppkey#93L, ps_partkey#92L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=396] - -(25) CometSort -Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Arguments: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95], [ps_suppkey#93L ASC NULLS FIRST, ps_partkey#92L ASC NULLS FIRST] - -(26) CometSortMergeJoin -Left output [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] -Right output [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Arguments: [l_suppkey#18L, l_partkey#17L], [ps_suppkey#93L, ps_partkey#92L], Inner - -(27) CometProject -Input [10]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Arguments: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95], [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] - -(28) CometExchange -Input [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] -Arguments: hashpartitioning(l_orderkey#16L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=400] - -(29) CometSort -Input [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] -Arguments: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95], [l_orderkey#16L ASC NULLS FIRST] - -(30) CometNativeScan parquet -Output [2]: [o_orderkey#56L, o_orderdate#60] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/orders] -PushedFilters: [IsNotNull(o_orderkey)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [o_orderkey#56L, o_orderdate#60] -Condition : isnotnull(o_orderkey#56L) - -(32) CometExchange -Input [2]: [o_orderkey#56L, o_orderdate#60] -Arguments: hashpartitioning(o_orderkey#56L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=404] - -(33) CometSort -Input [2]: [o_orderkey#56L, o_orderdate#60] -Arguments: [o_orderkey#56L, o_orderdate#60], [o_orderkey#56L ASC NULLS FIRST] - -(34) CometSortMergeJoin -Left output [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] -Right output [2]: [o_orderkey#56L, o_orderdate#60] -Arguments: [l_orderkey#16L], [o_orderkey#56L], Inner - -(35) CometProject -Input [8]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderkey#56L, o_orderdate#60] -Arguments: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60], [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60] - -(36) CometNativeScan parquet -Output [2]: [n_nationkey#48L, n_name#49] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/nation] -PushedFilters: [IsNotNull(n_nationkey)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [n_nationkey#48L, n_name#49] -Condition : isnotnull(n_nationkey#48L) - -(38) CometBroadcastExchange -Input [2]: [n_nationkey#48L, n_name#49] -Arguments: [n_nationkey#48L, n_name#49] - -(39) CometBroadcastHashJoin -Left output [6]: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60] -Right output [2]: [n_nationkey#48L, n_name#49] -Arguments: [s_nationkey#111L], [n_nationkey#48L], Inner, BuildRight - -(40) CometProject -Input [8]: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60, n_nationkey#48L, n_name#49] -Arguments: [nation#122, o_year#123, amount#124], [n_name#49 AS nation#122, year(o_orderdate#60) AS o_year#123, ((l_extendedprice#21 * (1 - l_discount#22)) - (ps_supplycost#95 * l_quantity#20)) AS amount#124] - -(41) CometHashAggregate -Input [3]: [nation#122, o_year#123, amount#124] -Keys [2]: [nation#122, o_year#123] -Functions [1]: [partial_sum(amount#124)] - -(42) CometExchange -Input [4]: [nation#122, o_year#123, sum#132, isEmpty#133] -Arguments: hashpartitioning(nation#122, o_year#123, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=413] - -(43) CometHashAggregate -Input [4]: [nation#122, o_year#123, sum#132, isEmpty#133] -Keys [2]: [nation#122, o_year#123] -Functions [1]: [sum(amount#124)] - -(44) CometExchange -Input [3]: [nation#122, o_year#123, sum_profit#125] -Arguments: rangepartitioning(nation#122 ASC NULLS FIRST, o_year#123 DESC NULLS LAST, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=415] - -(45) CometSort -Input [3]: [nation#122, o_year#123, sum_profit#125] -Arguments: [nation#122, o_year#123, sum_profit#125], [nation#122 ASC NULLS FIRST, o_year#123 DESC NULLS LAST] - -(46) AdaptiveSparkPlan -Output [3]: [nation#122, o_year#123, sum_profit#125] -Arguments: isFinalPlan=false - - -Query 9 returned 175 rows, hash=fff4ba3023e74505f304fa3243cbeeb6 -Query 9 took 375.36 seconds - -Iteration 1 took 375.36 seconds - -Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/comet-offheap4g-q9-tpch-1775054750948.json diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.time b/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.time deleted file mode 100644 index 7aff1629d6..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.time +++ /dev/null @@ -1,6872 +0,0 @@ -26/04/01 08:39:30 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) -26/04/01 08:39:30 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address -26/04/01 08:39:30 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable -26/04/01 08:39:31 INFO SparkContext: Running Spark version 3.5.8 -26/04/01 08:39:31 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 08:39:31 INFO SparkContext: Java version 17.0.17 -26/04/01 08:39:31 INFO ResourceUtils: ============================================================== -26/04/01 08:39:31 INFO ResourceUtils: No custom resources configured for spark.driver. -26/04/01 08:39:31 INFO ResourceUtils: ============================================================== -26/04/01 08:39:31 INFO SparkContext: Submitted application: comet-offheap4g-q9 benchmark derived from tpch -26/04/01 08:39:31 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 4096, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) -26/04/01 08:39:31 INFO ResourceProfile: Limiting resource is cpu -26/04/01 08:39:31 INFO ResourceProfileManager: Added ResourceProfile id: 0 -26/04/01 08:39:31 INFO SecurityManager: Changing view acls to: andy -26/04/01 08:39:31 INFO SecurityManager: Changing modify acls to: andy -26/04/01 08:39:31 INFO SecurityManager: Changing view acls groups to: -26/04/01 08:39:31 INFO SecurityManager: Changing modify acls groups to: -26/04/01 08:39:31 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY -26/04/01 08:39:31 INFO Utils: Successfully started service 'sparkDriver' on port 59301. -26/04/01 08:39:31 INFO SparkEnv: Registering MapOutputTracker -26/04/01 08:39:31 INFO SparkEnv: Registering BlockManagerMaster -26/04/01 08:39:31 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information -26/04/01 08:39:31 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up -26/04/01 08:39:31 INFO SparkEnv: Registering BlockManagerMasterHeartbeat -26/04/01 08:39:31 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-bde69578-45f2-4d2d-97b9-bf70e14e1e93 -26/04/01 08:39:31 INFO MemoryStore: MemoryStore started with capacity 8.6 GiB -26/04/01 08:39:31 INFO SparkEnv: Registering OutputCommitCoordinator -26/04/01 08:39:31 INFO SparkContext: Added JAR file:///Users/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar at spark://10.0.0.133:59301/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775054371206 -26/04/01 08:39:31 INFO CometDriverPlugin: CometDriverPlugin init -26/04/01 08:39:31 INFO CometDriverPlugin: Setting spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions -26/04/01 08:39:31 INFO CometDriverPlugin: Comet metrics reporting is disabled. Set spark.comet.metrics.enabled=true to enable. -26/04/01 08:39:31 INFO CometDriverPlugin: Comet is running in unified memory mode and sharing off-heap memory with Spark -26/04/01 08:39:31 INFO DriverPluginContainer: Initialized driver component for plugin org.apache.spark.CometPlugin. -26/04/01 08:39:31 INFO Executor: Starting executor ID driver on host 10.0.0.133 -26/04/01 08:39:31 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 08:39:31 INFO Executor: Java version 17.0.17 -26/04/01 08:39:31 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' -26/04/01 08:39:31 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@5a3b465a for default. -26/04/01 08:39:31 INFO Executor: Fetching spark://10.0.0.133:59301/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775054371206 -26/04/01 08:39:31 INFO TransportClientFactory: Successfully created connection to /10.0.0.133:59301 after 8 ms (0 ms spent in bootstraps) -26/04/01 08:39:31 INFO Utils: Fetching spark://10.0.0.133:59301/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-cae9fc0e-ad8f-4143-bb25-84be77849f6a/userFiles-42cf6306-f997-4b97-99fd-212819ed4a9d/fetchFileTemp5223428367132204682.tmp -26/04/01 08:39:31 INFO Executor: Adding file:/private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-cae9fc0e-ad8f-4143-bb25-84be77849f6a/userFiles-42cf6306-f997-4b97-99fd-212819ed4a9d/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to class loader default -26/04/01 08:39:31 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 59303. -26/04/01 08:39:31 INFO NettyBlockTransferService: Server created on 10.0.0.133:59303 -26/04/01 08:39:31 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy -26/04/01 08:39:31 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 59303, None) -26/04/01 08:39:31 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:59303 with 8.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 59303, None) -26/04/01 08:39:31 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 59303, None) -26/04/01 08:39:31 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 59303, None) -26/04/01 08:39:31 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. -26/04/01 08:39:31 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. -26/04/01 08:39:31 INFO InMemoryFileIndex: It took 17 ms to list leaf files for 1 paths. -26/04/01 08:39:32 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:39:32 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:39:32 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:39:32 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:39:32 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:32 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:39:32 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 08:39:32 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 08:39:32 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:32 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:32 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:39:32 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 -26/04/01 08:39:32 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 08:39:32 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) -26/04/01 08:39:32 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 2041 bytes result sent to driver -26/04/01 08:39:32 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 127 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:39:32 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool -26/04/01 08:39:32 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.400 s -26/04/01 08:39:32 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:39:32 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished -26/04/01 08:39:32 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.411248 s -26/04/01 08:39:35 INFO NativeBase: Couldn't locate log file from either COMET_CONF_DIR or comet.log.file.path. Using default log configuration with INFO log level which emits to stderr -26/04/01 08:39:35 INFO core/src/lib.rs: Comet native library version 0.15.0 initialized -26/04/01 08:39:35 INFO NativeBase: Setting system property arrow.enable_unsafe_memory_access to true -26/04/01 08:39:35 INFO NativeBase: Setting system property arrow.enable_null_check_for_get to false -26/04/01 08:39:35 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. -26/04/01 08:39:35 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:39:35 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:39:35 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:39:35 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:39:35 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:35 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:39:35 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 08:39:35 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 08:39:35 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:35 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:35 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:39:35 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 -26/04/01 08:39:35 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 08:39:35 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) -26/04/01 08:39:35 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2344 bytes result sent to driver -26/04/01 08:39:35 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 17 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:39:35 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool -26/04/01 08:39:35 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.025 s -26/04/01 08:39:35 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:39:35 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished -26/04/01 08:39:35 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.027290 s -26/04/01 08:39:35 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 08:39:35 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:39:35 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:39:35 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:39:35 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:39:35 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:35 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:39:35 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 08:39:35 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 08:39:35 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:35 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:35 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:39:35 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 -26/04/01 08:39:35 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 08:39:35 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) -26/04/01 08:39:35 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver -26/04/01 08:39:35 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 7 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:39:35 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool -26/04/01 08:39:35 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s -26/04/01 08:39:35 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:39:35 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished -26/04/01 08:39:35 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.015439 s -26/04/01 08:39:35 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. -26/04/01 08:39:35 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:39:35 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:39:35 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:39:35 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:39:35 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:35 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:39:35 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 08:39:35 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 08:39:35 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:35 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:35 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:39:35 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 -26/04/01 08:39:35 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 08:39:35 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) -26/04/01 08:39:35 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver -26/04/01 08:39:35 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 7 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:39:35 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool -26/04/01 08:39:35 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s -26/04/01 08:39:35 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:39:35 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished -26/04/01 08:39:35 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.015823 s -26/04/01 08:39:35 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. -26/04/01 08:39:35 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:39:35 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:39:35 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:39:35 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:39:35 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:35 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:39:35 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 08:39:35 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 08:39:35 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:35 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:35 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:39:35 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 -26/04/01 08:39:35 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9299 bytes) -26/04/01 08:39:35 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) -26/04/01 08:39:35 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver -26/04/01 08:39:35 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:39:35 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool -26/04/01 08:39:35 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s -26/04/01 08:39:35 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:39:35 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished -26/04/01 08:39:35 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.014553 s -26/04/01 08:39:35 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 08:39:35 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:39:35 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:39:35 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:39:35 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:39:35 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:35 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:39:35 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 08:39:35 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 08:39:35 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:35 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:35 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:39:35 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 -26/04/01 08:39:35 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 08:39:35 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) -26/04/01 08:39:35 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver -26/04/01 08:39:35 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:39:35 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool -26/04/01 08:39:35 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 08:39:35 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:39:35 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished -26/04/01 08:39:35 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.014503 s -26/04/01 08:39:35 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. -26/04/01 08:39:35 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:39:35 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:39:35 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:39:35 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:39:35 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:35 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:39:35 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 08:39:35 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 08:39:35 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:35 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:35 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:39:35 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 -26/04/01 08:39:35 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 08:39:35 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) -26/04/01 08:39:35 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver -26/04/01 08:39:35 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:39:35 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool -26/04/01 08:39:35 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.011 s -26/04/01 08:39:35 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:39:35 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished -26/04/01 08:39:35 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.012746 s -26/04/01 08:39:35 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 08:39:35 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:39:35 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:39:35 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:39:35 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:39:35 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:35 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:39:35 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 104.8 KiB, free 8.6 GiB) -26/04/01 08:39:35 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 8.6 GiB) -26/04/01 08:39:35 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:59303 (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:35 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:35 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:39:35 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 -26/04/01 08:39:35 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 08:39:35 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) -26/04/01 08:39:35 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver -26/04/01 08:39:35 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:39:35 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool -26/04/01 08:39:35 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.012 s -26/04/01 08:39:35 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:39:35 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished -26/04/01 08:39:35 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.013823 s -26/04/01 08:39:35 INFO FileSourceStrategy: Pushed Filters: IsNotNull(p_name),StringContains(p_name,moccasin),IsNotNull(p_partkey) -26/04/01 08:39:35 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(p_name#75),Contains(p_name#75, moccasin),isnotnull(p_partkey#74L) -26/04/01 08:39:35 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_partkey),IsNotNull(l_suppkey),IsNotNull(l_orderkey) -26/04/01 08:39:35 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_partkey#17L),isnotnull(l_suppkey#18L),isnotnull(l_orderkey#16L) -26/04/01 08:39:35 INFO FileSourceStrategy: Pushed Filters: IsNotNull(s_suppkey),IsNotNull(s_nationkey) -26/04/01 08:39:35 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(s_suppkey#108L),isnotnull(s_nationkey#111L) -26/04/01 08:39:35 INFO FileSourceStrategy: Pushed Filters: IsNotNull(ps_suppkey),IsNotNull(ps_partkey) -26/04/01 08:39:35 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(ps_suppkey#93L),isnotnull(ps_partkey#92L) -26/04/01 08:39:35 INFO FileSourceStrategy: Pushed Filters: IsNotNull(o_orderkey) -26/04/01 08:39:35 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(o_orderkey#56L) -26/04/01 08:39:35 INFO FileSourceStrategy: Pushed Filters: IsNotNull(n_nationkey) -26/04/01 08:39:35 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(n_nationkey#48L) -26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:35 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:59303 in memory (size: 38.0 KiB, free: 8.6 GiB) -26/04/01 08:39:35 INFO CometScanExec: Planning scan with bin packing, max size: 102898790 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:35 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:35 INFO CometScanExec: Planning scan with bin packing, max size: 27960063 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:35 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:35 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:35 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:36 INFO CometScanExec: Planning scan with bin packing, max size: 102898790 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:36 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:59303 (size: 35.4 KiB, free: 8.6 GiB) -26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:59303 (size: 35.4 KiB, free: 8.6 GiB) -26/04/01 08:39:36 INFO SparkContext: Created broadcast 9 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:39:36 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:39:36 INFO FileSourceScanExec: Planning scan with bin packing, max size: 102898790 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:36 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:36 INFO DAGScheduler: Registering RDD 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 -26/04/01 08:39:36 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:36 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 8 output partitions -26/04/01 08:39:36 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:39:36 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:39:36 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:36 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[21] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:39:36 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 20.5 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 8.1 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:59303 (size: 8.1 KiB, free: 8.6 GiB) -26/04/01 08:39:36 INFO SparkContext: Created broadcast 11 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:36 INFO DAGScheduler: Submitting 8 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[21] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7)) -26/04/01 08:39:36 INFO TaskSchedulerImpl: Adding task set 8.0 with 8 tasks resource profile 0 -26/04/01 08:39:36 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 08:39:36 INFO DAGScheduler: Final stage: ResultStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:39:36 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:39:36 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:36 INFO DAGScheduler: Submitting ResultStage 9 (MapPartitionsRDD[20] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9429 bytes) -26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:59303 (size: 35.4 KiB, free: 8.6 GiB) -26/04/01 08:39:36 INFO SparkContext: Created broadcast 10 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:39:36 INFO TaskSetManager: Starting task 1.0 in stage 8.0 (TID 9) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9431 bytes) -26/04/01 08:39:36 INFO TaskSetManager: Starting task 2.0 in stage 8.0 (TID 10) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9431 bytes) -26/04/01 08:39:36 INFO TaskSetManager: Starting task 3.0 in stage 8.0 (TID 11) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9431 bytes) -26/04/01 08:39:36 INFO TaskSetManager: Starting task 4.0 in stage 8.0 (TID 12) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9430 bytes) -26/04/01 08:39:36 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 10.8 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO TaskSetManager: Starting task 5.0 in stage 8.0 (TID 13) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9432 bytes) -26/04/01 08:39:36 INFO TaskSetManager: Starting task 6.0 in stage 8.0 (TID 14) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9431 bytes) -26/04/01 08:39:36 INFO TaskSetManager: Starting task 7.0 in stage 8.0 (TID 15) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9432 bytes) -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 5.3 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) -26/04/01 08:39:36 INFO Executor: Running task 1.0 in stage 8.0 (TID 9) -26/04/01 08:39:36 INFO Executor: Running task 2.0 in stage 8.0 (TID 10) -26/04/01 08:39:36 INFO Executor: Running task 3.0 in stage 8.0 (TID 11) -26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:59303 (size: 5.3 KiB, free: 8.6 GiB) -26/04/01 08:39:36 INFO Executor: Running task 4.0 in stage 8.0 (TID 12) -26/04/01 08:39:36 INFO Executor: Running task 5.0 in stage 8.0 (TID 13) -26/04/01 08:39:36 INFO SparkContext: Created broadcast 12 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:36 INFO Executor: Running task 6.0 in stage 8.0 (TID 14) -26/04/01 08:39:36 INFO Executor: Running task 7.0 in stage 8.0 (TID 15) -26/04/01 08:39:36 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 9 (MapPartitionsRDD[20] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:39:36 INFO TaskSchedulerImpl: Adding task set 9.0 with 1 tasks resource profile 0 -26/04/01 08:39:36 INFO DAGScheduler: Registering RDD 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 -26/04/01 08:39:36 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions -26/04/01 08:39:36 INFO DAGScheduler: Final stage: ShuffleMapStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:39:36 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:39:36 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:36 INFO CometScanExec: Planning scan with bin packing, max size: 27960063 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:36 INFO DAGScheduler: Submitting ShuffleMapStage 10 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_13 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_14 stored as values in memory (estimated size 20.6 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_14_piece0 stored as bytes in memory (estimated size 8.2 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_14_piece0 in memory on 10.0.0.133:59303 (size: 8.2 KiB, free: 8.6 GiB) -26/04/01 08:39:36 INFO SparkContext: Created broadcast 14 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:36 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 10 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:39:36 INFO TaskSchedulerImpl: Adding task set 10.0 with 208 tasks resource profile 0 -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_13_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_13_piece0 in memory on 10.0.0.133:59303 (size: 35.4 KiB, free: 8.6 GiB) -26/04/01 08:39:36 INFO SparkContext: Created broadcast 13 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:39:36 INFO FileSourceScanExec: Planning scan with bin packing, max size: 27960063 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:36 INFO DAGScheduler: Registering RDD 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 2 -26/04/01 08:39:36 INFO DAGScheduler: Got map stage job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 8 output partitions -26/04/01 08:39:36 INFO DAGScheduler: Final stage: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:39:36 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:39:36 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:36 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:36 INFO DAGScheduler: Submitting ShuffleMapStage 11 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_15 stored as values in memory (estimated size 18.0 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_15_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_15_piece0 in memory on 10.0.0.133:59303 (size: 7.9 KiB, free: 8.6 GiB) -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_16 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO SparkContext: Created broadcast 15 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:36 INFO DAGScheduler: Submitting 8 missing tasks from ShuffleMapStage 11 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7)) -26/04/01 08:39:36 INFO TaskSchedulerImpl: Adding task set 11.0 with 8 tasks resource profile 0 -26/04/01 08:39:36 INFO BaseAllocator: Debug mode disabled. Enable with the VM option -Darrow.memory.debug.allocator=true. -26/04/01 08:39:36 INFO DefaultAllocationManagerOption: allocation manager type not specified, using netty as the default type -26/04/01 08:39:36 INFO CheckAllocator: Using DefaultAllocationManager at memory/unsafe/DefaultAllocationManagerFactory.class -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_16_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_16_piece0 in memory on 10.0.0.133:59303 (size: 35.4 KiB, free: 8.6 GiB) -26/04/01 08:39:36 INFO SparkContext: Created broadcast 16 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:39:36 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:36 INFO DAGScheduler: Registering RDD 30 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 3 -26/04/01 08:39:36 INFO DAGScheduler: Got map stage job 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 37 output partitions -26/04/01 08:39:36 INFO DAGScheduler: Final stage: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:39:36 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:39:36 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:36 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:36 INFO DAGScheduler: Submitting ShuffleMapStage 12 (MapPartitionsRDD[30] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_17 stored as values in memory (estimated size 18.2 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_17_piece0 stored as bytes in memory (estimated size 8.0 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_17_piece0 in memory on 10.0.0.133:59303 (size: 8.0 KiB, free: 8.6 GiB) -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_18 stored as values in memory (estimated size 203.0 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:36 INFO SparkContext: Created broadcast 17 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:36 INFO DAGScheduler: Submitting 37 missing tasks from ShuffleMapStage 12 (MapPartitionsRDD[30] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:39:36 INFO TaskSchedulerImpl: Adding task set 12.0 with 37 tasks resource profile 0 -26/04/01 08:39:36 INFO core/src/execution/jni_api.rs: Comet tokio runtime: using spark.executor.cores=8 worker threads -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_18_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_18_piece0 in memory on 10.0.0.133:59303 (size: 35.4 KiB, free: 8.6 GiB) -26/04/01 08:39:36 INFO SparkContext: Created broadcast 18 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:39:36 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:39:36 INFO DAGScheduler: Registering RDD 33 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 4 -26/04/01 08:39:36 INFO DAGScheduler: Got map stage job 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 64 output partitions -26/04/01 08:39:36 INFO DAGScheduler: Final stage: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:39:36 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:39:36 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:36 INFO DAGScheduler: Submitting ShuffleMapStage 13 (MapPartitionsRDD[33] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_19 stored as values in memory (estimated size 16.6 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO MemoryStore: Block broadcast_19_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 8.6 GiB) -26/04/01 08:39:36 INFO BlockManagerInfo: Added broadcast_19_piece0 in memory on 10.0.0.133:59303 (size: 7.9 KiB, free: 8.6 GiB) -26/04/01 08:39:36 INFO SparkContext: Created broadcast 19 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:36 INFO DAGScheduler: Submitting 64 missing tasks from ShuffleMapStage 13 (MapPartitionsRDD[33] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:39:36 INFO TaskSchedulerImpl: Adding task set 13.0 with 64 tasks resource profile 0 -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO Executor: Finished task 7.0 in stage 8.0 (TID 15). 6761 bytes result sent to driver -26/04/01 08:39:38 INFO Executor: Finished task 3.0 in stage 8.0 (TID 11). 6761 bytes result sent to driver -26/04/01 08:39:38 INFO Executor: Finished task 5.0 in stage 8.0 (TID 13). 6761 bytes result sent to driver -26/04/01 08:39:38 INFO Executor: Finished task 1.0 in stage 8.0 (TID 9). 6761 bytes result sent to driver -26/04/01 08:39:38 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 6761 bytes result sent to driver -26/04/01 08:39:38 INFO TaskSetManager: Starting task 0.0 in stage 9.0 (TID 16) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:39:38 INFO Executor: Running task 0.0 in stage 9.0 (TID 16) -26/04/01 08:39:38 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 17) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:39:38 INFO Executor: Running task 0.0 in stage 10.0 (TID 17) -26/04/01 08:39:38 INFO TaskSetManager: Starting task 1.0 in stage 10.0 (TID 18) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:39:38 INFO TaskSetManager: Finished task 7.0 in stage 8.0 (TID 15) in 2439 ms on 10.0.0.133 (executor driver) (1/8) -26/04/01 08:39:38 INFO TaskSetManager: Finished task 1.0 in stage 8.0 (TID 9) in 2440 ms on 10.0.0.133 (executor driver) (2/8) -26/04/01 08:39:38 INFO Executor: Finished task 4.0 in stage 8.0 (TID 12). 6761 bytes result sent to driver -26/04/01 08:39:38 INFO Executor: Running task 1.0 in stage 10.0 (TID 18) -26/04/01 08:39:38 INFO Executor: Finished task 2.0 in stage 8.0 (TID 10). 6761 bytes result sent to driver -26/04/01 08:39:38 INFO TaskSetManager: Starting task 2.0 in stage 10.0 (TID 19) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:39:38 INFO Executor: Finished task 6.0 in stage 8.0 (TID 14). 6761 bytes result sent to driver -26/04/01 08:39:38 INFO Executor: Running task 2.0 in stage 10.0 (TID 19) -26/04/01 08:39:38 INFO TaskSetManager: Finished task 3.0 in stage 8.0 (TID 11) in 2441 ms on 10.0.0.133 (executor driver) (3/8) -26/04/01 08:39:38 INFO TaskSetManager: Starting task 3.0 in stage 10.0 (TID 20) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:39:38 INFO Executor: Running task 3.0 in stage 10.0 (TID 20) -26/04/01 08:39:38 INFO TaskSetManager: Starting task 4.0 in stage 10.0 (TID 21) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:39:38 INFO TaskSetManager: Finished task 5.0 in stage 8.0 (TID 13) in 2442 ms on 10.0.0.133 (executor driver) (4/8) -26/04/01 08:39:38 INFO Executor: Running task 4.0 in stage 10.0 (TID 21) -26/04/01 08:39:38 INFO TaskSetManager: Starting task 5.0 in stage 10.0 (TID 22) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:39:38 INFO Executor: Running task 5.0 in stage 10.0 (TID 22) -26/04/01 08:39:38 INFO TaskSetManager: Starting task 6.0 in stage 10.0 (TID 23) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:39:38 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 2443 ms on 10.0.0.133 (executor driver) (5/8) -26/04/01 08:39:38 INFO Executor: Running task 6.0 in stage 10.0 (TID 23) -26/04/01 08:39:38 INFO TaskSetManager: Finished task 4.0 in stage 8.0 (TID 12) in 2443 ms on 10.0.0.133 (executor driver) (6/8) -26/04/01 08:39:38 INFO TaskSetManager: Finished task 2.0 in stage 8.0 (TID 10) in 2443 ms on 10.0.0.133 (executor driver) (7/8) -26/04/01 08:39:38 INFO TaskSetManager: Finished task 6.0 in stage 8.0 (TID 14) in 2443 ms on 10.0.0.133 (executor driver) (8/8) -26/04/01 08:39:38 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool -26/04/01 08:39:38 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.453 s -26/04/01 08:39:38 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:39:38 INFO DAGScheduler: running: Set(ShuffleMapStage 12, ResultStage 9, ShuffleMapStage 13, ShuffleMapStage 10, ShuffleMapStage 11) -26/04/01 08:39:38 INFO DAGScheduler: waiting: Set() -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO DAGScheduler: failed: Set() -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO ShufflePartitionsUtil: For shuffle(0), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 08:39:38 INFO Executor: Finished task 0.0 in stage 9.0 (TID 16). 4762 bytes result sent to driver -26/04/01 08:39:38 INFO TaskSetManager: Starting task 7.0 in stage 10.0 (TID 24) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:39:38 INFO Executor: Running task 7.0 in stage 10.0 (TID 24) -26/04/01 08:39:38 INFO TaskSetManager: Finished task 0.0 in stage 9.0 (TID 16) in 40 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:39:38 INFO TaskSchedulerImpl: Removed TaskSet 9.0, whose tasks have all completed, from pool -26/04/01 08:39:38 INFO DAGScheduler: ResultStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.479 s -26/04/01 08:39:38 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:39:38 INFO TaskSchedulerImpl: Killing all running tasks in stage 9: Stage finished -26/04/01 08:39:38 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 2.487185 s -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:38 INFO Utils: Coalesced 1 broadcast batches into 1 (25 rows) -26/04/01 08:39:38 INFO MemoryStore: Block broadcast_20 stored as values in memory (estimated size 880.0 B, free 8.6 GiB) -26/04/01 08:39:38 INFO MemoryStore: Block broadcast_20_piece0 stored as bytes in memory (estimated size 831.0 B, free 8.6 GiB) -26/04/01 08:39:38 INFO BlockManagerInfo: Added broadcast_20_piece0 in memory on 10.0.0.133:59303 (size: 831.0 B, free: 8.6 GiB) -26/04/01 08:39:38 INFO SparkContext: Created broadcast 20 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:39:38 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:39:38 INFO DAGScheduler: Got job 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions -26/04/01 08:39:38 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:39:38 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 14) -26/04/01 08:39:38 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:38 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[35] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:39:38 INFO MemoryStore: Block broadcast_21 stored as values in memory (estimated size 14.5 KiB, free 8.6 GiB) -26/04/01 08:39:38 INFO MemoryStore: Block broadcast_21_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 8.6 GiB) -26/04/01 08:39:38 INFO BlockManagerInfo: Added broadcast_21_piece0 in memory on 10.0.0.133:59303 (size: 7.9 KiB, free: 8.6 GiB) -26/04/01 08:39:38 INFO SparkContext: Created broadcast 21 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:38 INFO DAGScheduler: Submitting 200 missing tasks from ResultStage 15 (MapPartitionsRDD[35] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:39:38 INFO TaskSchedulerImpl: Adding task set 15.0 with 200 tasks resource profile 0 -26/04/01 08:39:46 INFO Executor: Finished task 4.0 in stage 10.0 (TID 21). 6649 bytes result sent to driver -26/04/01 08:39:46 INFO TaskSetManager: Starting task 8.0 in stage 10.0 (TID 25) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:39:46 INFO Executor: Finished task 3.0 in stage 10.0 (TID 20). 6649 bytes result sent to driver -26/04/01 08:39:46 INFO Executor: Running task 8.0 in stage 10.0 (TID 25) -26/04/01 08:39:46 INFO TaskSetManager: Finished task 4.0 in stage 10.0 (TID 21) in 7994 ms on 10.0.0.133 (executor driver) (1/208) -26/04/01 08:39:46 INFO TaskSetManager: Starting task 9.0 in stage 10.0 (TID 26) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:39:46 INFO TaskSetManager: Finished task 3.0 in stage 10.0 (TID 20) in 7995 ms on 10.0.0.133 (executor driver) (2/208) -26/04/01 08:39:46 INFO Executor: Running task 9.0 in stage 10.0 (TID 26) -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:46 INFO Executor: Finished task 0.0 in stage 10.0 (TID 17). 6649 bytes result sent to driver -26/04/01 08:39:46 INFO TaskSetManager: Starting task 10.0 in stage 10.0 (TID 27) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:46 INFO Executor: Running task 10.0 in stage 10.0 (TID 27) -26/04/01 08:39:46 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 17) in 8001 ms on 10.0.0.133 (executor driver) (3/208) -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:46 INFO Executor: Finished task 1.0 in stage 10.0 (TID 18). 6649 bytes result sent to driver -26/04/01 08:39:46 INFO TaskSetManager: Starting task 11.0 in stage 10.0 (TID 28) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:39:46 INFO Executor: Running task 11.0 in stage 10.0 (TID 28) -26/04/01 08:39:46 INFO TaskSetManager: Finished task 1.0 in stage 10.0 (TID 18) in 8005 ms on 10.0.0.133 (executor driver) (4/208) -26/04/01 08:39:46 INFO Executor: Finished task 5.0 in stage 10.0 (TID 22). 6649 bytes result sent to driver -26/04/01 08:39:46 INFO TaskSetManager: Starting task 12.0 in stage 10.0 (TID 29) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:39:46 INFO Executor: Running task 12.0 in stage 10.0 (TID 29) -26/04/01 08:39:46 INFO TaskSetManager: Finished task 5.0 in stage 10.0 (TID 22) in 8004 ms on 10.0.0.133 (executor driver) (5/208) -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:46 INFO Executor: Finished task 6.0 in stage 10.0 (TID 23). 6649 bytes result sent to driver -26/04/01 08:39:46 INFO TaskSetManager: Starting task 13.0 in stage 10.0 (TID 30) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:39:46 INFO TaskSetManager: Finished task 6.0 in stage 10.0 (TID 23) in 8019 ms on 10.0.0.133 (executor driver) (6/208) -26/04/01 08:39:46 INFO Executor: Running task 13.0 in stage 10.0 (TID 30) -26/04/01 08:39:46 INFO Executor: Finished task 2.0 in stage 10.0 (TID 19). 6649 bytes result sent to driver -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:46 INFO TaskSetManager: Starting task 14.0 in stage 10.0 (TID 31) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:39:46 INFO TaskSetManager: Finished task 2.0 in stage 10.0 (TID 19) in 8023 ms on 10.0.0.133 (executor driver) (7/208) -26/04/01 08:39:46 INFO Executor: Running task 14.0 in stage 10.0 (TID 31) -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:46 INFO Executor: Finished task 7.0 in stage 10.0 (TID 24). 6649 bytes result sent to driver -26/04/01 08:39:46 INFO TaskSetManager: Starting task 15.0 in stage 10.0 (TID 32) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:39:46 INFO Executor: Running task 15.0 in stage 10.0 (TID 32) -26/04/01 08:39:46 INFO TaskSetManager: Finished task 7.0 in stage 10.0 (TID 24) in 7990 ms on 10.0.0.133 (executor driver) (8/208) -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO Executor: Finished task 9.0 in stage 10.0 (TID 26). 6606 bytes result sent to driver -26/04/01 08:39:54 INFO TaskSetManager: Starting task 16.0 in stage 10.0 (TID 33) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:39:54 INFO TaskSetManager: Finished task 9.0 in stage 10.0 (TID 26) in 7941 ms on 10.0.0.133 (executor driver) (9/208) -26/04/01 08:39:54 INFO Executor: Running task 16.0 in stage 10.0 (TID 33) -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO Executor: Finished task 12.0 in stage 10.0 (TID 29). 6563 bytes result sent to driver -26/04/01 08:39:54 INFO TaskSetManager: Starting task 17.0 in stage 10.0 (TID 34) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:39:54 INFO Executor: Running task 17.0 in stage 10.0 (TID 34) -26/04/01 08:39:54 INFO TaskSetManager: Finished task 12.0 in stage 10.0 (TID 29) in 7947 ms on 10.0.0.133 (executor driver) (10/208) -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO Executor: Finished task 10.0 in stage 10.0 (TID 27). 6606 bytes result sent to driver -26/04/01 08:39:54 INFO TaskSetManager: Starting task 18.0 in stage 10.0 (TID 35) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:39:54 INFO TaskSetManager: Finished task 10.0 in stage 10.0 (TID 27) in 7959 ms on 10.0.0.133 (executor driver) (11/208) -26/04/01 08:39:54 INFO Executor: Running task 18.0 in stage 10.0 (TID 35) -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO Executor: Finished task 15.0 in stage 10.0 (TID 32). 6606 bytes result sent to driver -26/04/01 08:39:54 INFO TaskSetManager: Starting task 19.0 in stage 10.0 (TID 36) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:39:54 INFO TaskSetManager: Finished task 15.0 in stage 10.0 (TID 32) in 7934 ms on 10.0.0.133 (executor driver) (12/208) -26/04/01 08:39:54 INFO Executor: Running task 19.0 in stage 10.0 (TID 36) -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO Executor: Finished task 14.0 in stage 10.0 (TID 31). 6563 bytes result sent to driver -26/04/01 08:39:54 INFO TaskSetManager: Starting task 20.0 in stage 10.0 (TID 37) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:39:54 INFO Executor: Running task 20.0 in stage 10.0 (TID 37) -26/04/01 08:39:54 INFO TaskSetManager: Finished task 14.0 in stage 10.0 (TID 31) in 7956 ms on 10.0.0.133 (executor driver) (13/208) -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO Executor: Finished task 8.0 in stage 10.0 (TID 25). 6606 bytes result sent to driver -26/04/01 08:39:54 INFO TaskSetManager: Starting task 21.0 in stage 10.0 (TID 38) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:39:54 INFO TaskSetManager: Finished task 8.0 in stage 10.0 (TID 25) in 7986 ms on 10.0.0.133 (executor driver) (14/208) -26/04/01 08:39:54 INFO Executor: Running task 21.0 in stage 10.0 (TID 38) -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO Executor: Finished task 13.0 in stage 10.0 (TID 30). 6606 bytes result sent to driver -26/04/01 08:39:54 INFO TaskSetManager: Starting task 22.0 in stage 10.0 (TID 39) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:39:54 INFO TaskSetManager: Finished task 13.0 in stage 10.0 (TID 30) in 7979 ms on 10.0.0.133 (executor driver) (15/208) -26/04/01 08:39:54 INFO Executor: Running task 22.0 in stage 10.0 (TID 39) -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO Executor: Finished task 11.0 in stage 10.0 (TID 28). 6606 bytes result sent to driver -26/04/01 08:39:54 INFO TaskSetManager: Starting task 23.0 in stage 10.0 (TID 40) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:39:54 INFO Executor: Running task 23.0 in stage 10.0 (TID 40) -26/04/01 08:39:54 INFO TaskSetManager: Finished task 11.0 in stage 10.0 (TID 28) in 8018 ms on 10.0.0.133 (executor driver) (16/208) -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:39:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO Executor: Finished task 16.0 in stage 10.0 (TID 33). 6563 bytes result sent to driver -26/04/01 08:40:02 INFO TaskSetManager: Starting task 24.0 in stage 10.0 (TID 41) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:40:02 INFO TaskSetManager: Finished task 16.0 in stage 10.0 (TID 33) in 7950 ms on 10.0.0.133 (executor driver) (17/208) -26/04/01 08:40:02 INFO Executor: Running task 24.0 in stage 10.0 (TID 41) -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO Executor: Finished task 17.0 in stage 10.0 (TID 34). 6563 bytes result sent to driver -26/04/01 08:40:02 INFO TaskSetManager: Starting task 25.0 in stage 10.0 (TID 42) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:02 INFO TaskSetManager: Finished task 17.0 in stage 10.0 (TID 34) in 7958 ms on 10.0.0.133 (executor driver) (18/208) -26/04/01 08:40:02 INFO Executor: Running task 25.0 in stage 10.0 (TID 42) -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO Executor: Finished task 18.0 in stage 10.0 (TID 35). 6563 bytes result sent to driver -26/04/01 08:40:02 INFO TaskSetManager: Starting task 26.0 in stage 10.0 (TID 43) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:02 INFO TaskSetManager: Finished task 18.0 in stage 10.0 (TID 35) in 7960 ms on 10.0.0.133 (executor driver) (19/208) -26/04/01 08:40:02 INFO Executor: Running task 26.0 in stage 10.0 (TID 43) -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO Executor: Finished task 19.0 in stage 10.0 (TID 36). 6563 bytes result sent to driver -26/04/01 08:40:02 INFO TaskSetManager: Starting task 27.0 in stage 10.0 (TID 44) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:02 INFO Executor: Running task 27.0 in stage 10.0 (TID 44) -26/04/01 08:40:02 INFO TaskSetManager: Finished task 19.0 in stage 10.0 (TID 36) in 7960 ms on 10.0.0.133 (executor driver) (20/208) -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO Executor: Finished task 21.0 in stage 10.0 (TID 38). 6563 bytes result sent to driver -26/04/01 08:40:02 INFO TaskSetManager: Starting task 28.0 in stage 10.0 (TID 45) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:02 INFO Executor: Finished task 20.0 in stage 10.0 (TID 37). 6563 bytes result sent to driver -26/04/01 08:40:02 INFO Executor: Running task 28.0 in stage 10.0 (TID 45) -26/04/01 08:40:02 INFO TaskSetManager: Finished task 21.0 in stage 10.0 (TID 38) in 7975 ms on 10.0.0.133 (executor driver) (21/208) -26/04/01 08:40:02 INFO TaskSetManager: Starting task 29.0 in stage 10.0 (TID 46) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:02 INFO TaskSetManager: Finished task 20.0 in stage 10.0 (TID 37) in 7977 ms on 10.0.0.133 (executor driver) (22/208) -26/04/01 08:40:02 INFO Executor: Running task 29.0 in stage 10.0 (TID 46) -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO Executor: Finished task 22.0 in stage 10.0 (TID 39). 6563 bytes result sent to driver -26/04/01 08:40:02 INFO TaskSetManager: Starting task 30.0 in stage 10.0 (TID 47) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:40:02 INFO TaskSetManager: Finished task 22.0 in stage 10.0 (TID 39) in 7976 ms on 10.0.0.133 (executor driver) (23/208) -26/04/01 08:40:02 INFO Executor: Running task 30.0 in stage 10.0 (TID 47) -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO Executor: Finished task 23.0 in stage 10.0 (TID 40). 6563 bytes result sent to driver -26/04/01 08:40:02 INFO TaskSetManager: Starting task 31.0 in stage 10.0 (TID 48) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:02 INFO Executor: Running task 31.0 in stage 10.0 (TID 48) -26/04/01 08:40:02 INFO TaskSetManager: Finished task 23.0 in stage 10.0 (TID 40) in 7970 ms on 10.0.0.133 (executor driver) (24/208) -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO Executor: Finished task 25.0 in stage 10.0 (TID 42). 6563 bytes result sent to driver -26/04/01 08:40:10 INFO TaskSetManager: Starting task 32.0 in stage 10.0 (TID 49) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:10 INFO Executor: Running task 32.0 in stage 10.0 (TID 49) -26/04/01 08:40:10 INFO TaskSetManager: Finished task 25.0 in stage 10.0 (TID 42) in 7860 ms on 10.0.0.133 (executor driver) (25/208) -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO Executor: Finished task 24.0 in stage 10.0 (TID 41). 6563 bytes result sent to driver -26/04/01 08:40:10 INFO TaskSetManager: Starting task 33.0 in stage 10.0 (TID 50) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:10 INFO Executor: Running task 33.0 in stage 10.0 (TID 50) -26/04/01 08:40:10 INFO TaskSetManager: Finished task 24.0 in stage 10.0 (TID 41) in 7963 ms on 10.0.0.133 (executor driver) (26/208) -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO Executor: Finished task 27.0 in stage 10.0 (TID 44). 6563 bytes result sent to driver -26/04/01 08:40:10 INFO TaskSetManager: Starting task 34.0 in stage 10.0 (TID 51) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:10 INFO Executor: Running task 34.0 in stage 10.0 (TID 51) -26/04/01 08:40:10 INFO TaskSetManager: Finished task 27.0 in stage 10.0 (TID 44) in 7938 ms on 10.0.0.133 (executor driver) (27/208) -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO Executor: Finished task 28.0 in stage 10.0 (TID 45). 6563 bytes result sent to driver -26/04/01 08:40:10 INFO TaskSetManager: Starting task 35.0 in stage 10.0 (TID 52) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:10 INFO TaskSetManager: Finished task 28.0 in stage 10.0 (TID 45) in 7921 ms on 10.0.0.133 (executor driver) (28/208) -26/04/01 08:40:10 INFO Executor: Running task 35.0 in stage 10.0 (TID 52) -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO Executor: Finished task 26.0 in stage 10.0 (TID 43). 6563 bytes result sent to driver -26/04/01 08:40:10 INFO TaskSetManager: Starting task 36.0 in stage 10.0 (TID 53) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:40:10 INFO Executor: Running task 36.0 in stage 10.0 (TID 53) -26/04/01 08:40:10 INFO TaskSetManager: Finished task 26.0 in stage 10.0 (TID 43) in 7971 ms on 10.0.0.133 (executor driver) (29/208) -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO Executor: Finished task 29.0 in stage 10.0 (TID 46). 6563 bytes result sent to driver -26/04/01 08:40:10 INFO TaskSetManager: Starting task 37.0 in stage 10.0 (TID 54) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:10 INFO TaskSetManager: Finished task 29.0 in stage 10.0 (TID 46) in 7942 ms on 10.0.0.133 (executor driver) (30/208) -26/04/01 08:40:10 INFO Executor: Running task 37.0 in stage 10.0 (TID 54) -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO Executor: Finished task 30.0 in stage 10.0 (TID 47). 6563 bytes result sent to driver -26/04/01 08:40:10 INFO TaskSetManager: Starting task 38.0 in stage 10.0 (TID 55) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:10 INFO Executor: Running task 38.0 in stage 10.0 (TID 55) -26/04/01 08:40:10 INFO TaskSetManager: Finished task 30.0 in stage 10.0 (TID 47) in 7955 ms on 10.0.0.133 (executor driver) (31/208) -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO Executor: Finished task 31.0 in stage 10.0 (TID 48). 6563 bytes result sent to driver -26/04/01 08:40:10 INFO TaskSetManager: Starting task 39.0 in stage 10.0 (TID 56) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:10 INFO Executor: Running task 39.0 in stage 10.0 (TID 56) -26/04/01 08:40:10 INFO TaskSetManager: Finished task 31.0 in stage 10.0 (TID 48) in 7951 ms on 10.0.0.133 (executor driver) (32/208) -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO Executor: Finished task 32.0 in stage 10.0 (TID 49). 6606 bytes result sent to driver -26/04/01 08:40:18 INFO TaskSetManager: Starting task 40.0 in stage 10.0 (TID 57) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:18 INFO Executor: Running task 40.0 in stage 10.0 (TID 57) -26/04/01 08:40:18 INFO TaskSetManager: Finished task 32.0 in stage 10.0 (TID 49) in 7967 ms on 10.0.0.133 (executor driver) (33/208) -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO Executor: Finished task 34.0 in stage 10.0 (TID 51). 6606 bytes result sent to driver -26/04/01 08:40:18 INFO TaskSetManager: Starting task 41.0 in stage 10.0 (TID 58) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:18 INFO TaskSetManager: Finished task 34.0 in stage 10.0 (TID 51) in 7944 ms on 10.0.0.133 (executor driver) (34/208) -26/04/01 08:40:18 INFO Executor: Running task 41.0 in stage 10.0 (TID 58) -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO Executor: Finished task 33.0 in stage 10.0 (TID 50). 6606 bytes result sent to driver -26/04/01 08:40:18 INFO TaskSetManager: Starting task 42.0 in stage 10.0 (TID 59) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:40:18 INFO TaskSetManager: Finished task 33.0 in stage 10.0 (TID 50) in 7983 ms on 10.0.0.133 (executor driver) (35/208) -26/04/01 08:40:18 INFO Executor: Running task 42.0 in stage 10.0 (TID 59) -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO Executor: Finished task 35.0 in stage 10.0 (TID 52). 6606 bytes result sent to driver -26/04/01 08:40:18 INFO TaskSetManager: Starting task 43.0 in stage 10.0 (TID 60) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:40:18 INFO TaskSetManager: Finished task 35.0 in stage 10.0 (TID 52) in 7956 ms on 10.0.0.133 (executor driver) (36/208) -26/04/01 08:40:18 INFO Executor: Running task 43.0 in stage 10.0 (TID 60) -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO Executor: Finished task 36.0 in stage 10.0 (TID 53). 6606 bytes result sent to driver -26/04/01 08:40:18 INFO TaskSetManager: Starting task 44.0 in stage 10.0 (TID 61) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:18 INFO TaskSetManager: Finished task 36.0 in stage 10.0 (TID 53) in 7976 ms on 10.0.0.133 (executor driver) (37/208) -26/04/01 08:40:18 INFO Executor: Running task 44.0 in stage 10.0 (TID 61) -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO Executor: Finished task 39.0 in stage 10.0 (TID 56). 6606 bytes result sent to driver -26/04/01 08:40:18 INFO TaskSetManager: Starting task 45.0 in stage 10.0 (TID 62) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:18 INFO Executor: Running task 45.0 in stage 10.0 (TID 62) -26/04/01 08:40:18 INFO TaskSetManager: Finished task 39.0 in stage 10.0 (TID 56) in 7936 ms on 10.0.0.133 (executor driver) (38/208) -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO Executor: Finished task 38.0 in stage 10.0 (TID 55). 6606 bytes result sent to driver -26/04/01 08:40:18 INFO TaskSetManager: Starting task 46.0 in stage 10.0 (TID 63) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:18 INFO TaskSetManager: Finished task 38.0 in stage 10.0 (TID 55) in 7955 ms on 10.0.0.133 (executor driver) (39/208) -26/04/01 08:40:18 INFO Executor: Running task 46.0 in stage 10.0 (TID 63) -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO Executor: Finished task 37.0 in stage 10.0 (TID 54). 6606 bytes result sent to driver -26/04/01 08:40:18 INFO TaskSetManager: Starting task 47.0 in stage 10.0 (TID 64) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:18 INFO TaskSetManager: Finished task 37.0 in stage 10.0 (TID 54) in 8001 ms on 10.0.0.133 (executor driver) (40/208) -26/04/01 08:40:18 INFO Executor: Running task 47.0 in stage 10.0 (TID 64) -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO Executor: Finished task 45.0 in stage 10.0 (TID 62). 6563 bytes result sent to driver -26/04/01 08:40:26 INFO TaskSetManager: Starting task 48.0 in stage 10.0 (TID 65) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:40:26 INFO Executor: Running task 48.0 in stage 10.0 (TID 65) -26/04/01 08:40:26 INFO TaskSetManager: Finished task 45.0 in stage 10.0 (TID 62) in 7649 ms on 10.0.0.133 (executor driver) (41/208) -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO Executor: Finished task 40.0 in stage 10.0 (TID 57). 6563 bytes result sent to driver -26/04/01 08:40:26 INFO TaskSetManager: Starting task 49.0 in stage 10.0 (TID 66) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:26 INFO TaskSetManager: Finished task 40.0 in stage 10.0 (TID 57) in 7972 ms on 10.0.0.133 (executor driver) (42/208) -26/04/01 08:40:26 INFO Executor: Running task 49.0 in stage 10.0 (TID 66) -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO Executor: Finished task 42.0 in stage 10.0 (TID 59). 6563 bytes result sent to driver -26/04/01 08:40:26 INFO TaskSetManager: Starting task 50.0 in stage 10.0 (TID 67) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:26 INFO TaskSetManager: Finished task 42.0 in stage 10.0 (TID 59) in 7920 ms on 10.0.0.133 (executor driver) (43/208) -26/04/01 08:40:26 INFO Executor: Running task 50.0 in stage 10.0 (TID 67) -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO Executor: Finished task 41.0 in stage 10.0 (TID 58). 6563 bytes result sent to driver -26/04/01 08:40:26 INFO TaskSetManager: Starting task 51.0 in stage 10.0 (TID 68) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:26 INFO Executor: Running task 51.0 in stage 10.0 (TID 68) -26/04/01 08:40:26 INFO TaskSetManager: Finished task 41.0 in stage 10.0 (TID 58) in 7969 ms on 10.0.0.133 (executor driver) (44/208) -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO Executor: Finished task 43.0 in stage 10.0 (TID 60). 6563 bytes result sent to driver -26/04/01 08:40:26 INFO TaskSetManager: Starting task 52.0 in stage 10.0 (TID 69) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:26 INFO TaskSetManager: Finished task 43.0 in stage 10.0 (TID 60) in 7948 ms on 10.0.0.133 (executor driver) (45/208) -26/04/01 08:40:26 INFO Executor: Running task 52.0 in stage 10.0 (TID 69) -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO Executor: Finished task 47.0 in stage 10.0 (TID 64). 6563 bytes result sent to driver -26/04/01 08:40:26 INFO TaskSetManager: Starting task 53.0 in stage 10.0 (TID 70) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:26 INFO TaskSetManager: Finished task 47.0 in stage 10.0 (TID 64) in 7952 ms on 10.0.0.133 (executor driver) (46/208) -26/04/01 08:40:26 INFO Executor: Running task 53.0 in stage 10.0 (TID 70) -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO Executor: Finished task 46.0 in stage 10.0 (TID 63). 6563 bytes result sent to driver -26/04/01 08:40:26 INFO TaskSetManager: Starting task 54.0 in stage 10.0 (TID 71) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:40:26 INFO Executor: Running task 54.0 in stage 10.0 (TID 71) -26/04/01 08:40:26 INFO TaskSetManager: Finished task 46.0 in stage 10.0 (TID 63) in 7971 ms on 10.0.0.133 (executor driver) (47/208) -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO Executor: Finished task 44.0 in stage 10.0 (TID 61). 6563 bytes result sent to driver -26/04/01 08:40:26 INFO TaskSetManager: Starting task 55.0 in stage 10.0 (TID 72) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:26 INFO TaskSetManager: Finished task 44.0 in stage 10.0 (TID 61) in 8210 ms on 10.0.0.133 (executor driver) (48/208) -26/04/01 08:40:26 INFO Executor: Running task 55.0 in stage 10.0 (TID 72) -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:33 INFO Executor: Finished task 48.0 in stage 10.0 (TID 65). 6563 bytes result sent to driver -26/04/01 08:40:33 INFO TaskSetManager: Starting task 56.0 in stage 10.0 (TID 73) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:33 INFO TaskSetManager: Finished task 48.0 in stage 10.0 (TID 65) in 7978 ms on 10.0.0.133 (executor driver) (49/208) -26/04/01 08:40:33 INFO Executor: Running task 56.0 in stage 10.0 (TID 73) -26/04/01 08:40:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:34 INFO Executor: Finished task 49.0 in stage 10.0 (TID 66). 6563 bytes result sent to driver -26/04/01 08:40:34 INFO TaskSetManager: Starting task 57.0 in stage 10.0 (TID 74) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:34 INFO TaskSetManager: Finished task 49.0 in stage 10.0 (TID 66) in 7980 ms on 10.0.0.133 (executor driver) (50/208) -26/04/01 08:40:34 INFO Executor: Running task 57.0 in stage 10.0 (TID 74) -26/04/01 08:40:34 INFO Executor: Finished task 50.0 in stage 10.0 (TID 67). 6563 bytes result sent to driver -26/04/01 08:40:34 INFO TaskSetManager: Starting task 58.0 in stage 10.0 (TID 75) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:34 INFO TaskSetManager: Finished task 50.0 in stage 10.0 (TID 67) in 7940 ms on 10.0.0.133 (executor driver) (51/208) -26/04/01 08:40:34 INFO Executor: Running task 58.0 in stage 10.0 (TID 75) -26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:34 INFO Executor: Finished task 51.0 in stage 10.0 (TID 68). 6563 bytes result sent to driver -26/04/01 08:40:34 INFO TaskSetManager: Starting task 59.0 in stage 10.0 (TID 76) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:34 INFO TaskSetManager: Finished task 51.0 in stage 10.0 (TID 68) in 7962 ms on 10.0.0.133 (executor driver) (52/208) -26/04/01 08:40:34 INFO Executor: Running task 59.0 in stage 10.0 (TID 76) -26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:34 INFO Executor: Finished task 52.0 in stage 10.0 (TID 69). 6563 bytes result sent to driver -26/04/01 08:40:34 INFO TaskSetManager: Starting task 60.0 in stage 10.0 (TID 77) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:40:34 INFO Executor: Running task 60.0 in stage 10.0 (TID 77) -26/04/01 08:40:34 INFO TaskSetManager: Finished task 52.0 in stage 10.0 (TID 69) in 7989 ms on 10.0.0.133 (executor driver) (53/208) -26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:34 INFO Executor: Finished task 53.0 in stage 10.0 (TID 70). 6563 bytes result sent to driver -26/04/01 08:40:34 INFO TaskSetManager: Starting task 61.0 in stage 10.0 (TID 78) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:34 INFO TaskSetManager: Finished task 53.0 in stage 10.0 (TID 70) in 7977 ms on 10.0.0.133 (executor driver) (54/208) -26/04/01 08:40:34 INFO Executor: Running task 61.0 in stage 10.0 (TID 78) -26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:34 INFO Executor: Finished task 54.0 in stage 10.0 (TID 71). 6563 bytes result sent to driver -26/04/01 08:40:34 INFO TaskSetManager: Starting task 62.0 in stage 10.0 (TID 79) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:34 INFO TaskSetManager: Finished task 54.0 in stage 10.0 (TID 71) in 7988 ms on 10.0.0.133 (executor driver) (55/208) -26/04/01 08:40:34 INFO Executor: Running task 62.0 in stage 10.0 (TID 79) -26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:34 INFO Executor: Finished task 55.0 in stage 10.0 (TID 72). 6563 bytes result sent to driver -26/04/01 08:40:34 INFO TaskSetManager: Starting task 63.0 in stage 10.0 (TID 80) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:34 INFO TaskSetManager: Finished task 55.0 in stage 10.0 (TID 72) in 7966 ms on 10.0.0.133 (executor driver) (56/208) -26/04/01 08:40:34 INFO Executor: Running task 63.0 in stage 10.0 (TID 80) -26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:41 INFO Executor: Finished task 56.0 in stage 10.0 (TID 73). 6563 bytes result sent to driver -26/04/01 08:40:41 INFO TaskSetManager: Starting task 64.0 in stage 10.0 (TID 81) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:41 INFO TaskSetManager: Finished task 56.0 in stage 10.0 (TID 73) in 7963 ms on 10.0.0.133 (executor driver) (57/208) -26/04/01 08:40:41 INFO Executor: Running task 64.0 in stage 10.0 (TID 81) -26/04/01 08:40:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:42 INFO Executor: Finished task 59.0 in stage 10.0 (TID 76). 6563 bytes result sent to driver -26/04/01 08:40:42 INFO TaskSetManager: Starting task 65.0 in stage 10.0 (TID 82) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:42 INFO TaskSetManager: Finished task 59.0 in stage 10.0 (TID 76) in 7907 ms on 10.0.0.133 (executor driver) (58/208) -26/04/01 08:40:42 INFO Executor: Running task 65.0 in stage 10.0 (TID 82) -26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:42 INFO Executor: Finished task 57.0 in stage 10.0 (TID 74). 6563 bytes result sent to driver -26/04/01 08:40:42 INFO TaskSetManager: Starting task 66.0 in stage 10.0 (TID 83) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:40:42 INFO TaskSetManager: Finished task 57.0 in stage 10.0 (TID 74) in 7958 ms on 10.0.0.133 (executor driver) (59/208) -26/04/01 08:40:42 INFO Executor: Running task 66.0 in stage 10.0 (TID 83) -26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:42 INFO Executor: Finished task 60.0 in stage 10.0 (TID 77). 6563 bytes result sent to driver -26/04/01 08:40:42 INFO TaskSetManager: Starting task 67.0 in stage 10.0 (TID 84) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:40:42 INFO Executor: Running task 67.0 in stage 10.0 (TID 84) -26/04/01 08:40:42 INFO TaskSetManager: Finished task 60.0 in stage 10.0 (TID 77) in 7908 ms on 10.0.0.133 (executor driver) (60/208) -26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:42 INFO Executor: Finished task 58.0 in stage 10.0 (TID 75). 6563 bytes result sent to driver -26/04/01 08:40:42 INFO TaskSetManager: Starting task 68.0 in stage 10.0 (TID 85) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:42 INFO Executor: Running task 68.0 in stage 10.0 (TID 85) -26/04/01 08:40:42 INFO TaskSetManager: Finished task 58.0 in stage 10.0 (TID 75) in 7999 ms on 10.0.0.133 (executor driver) (61/208) -26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:42 INFO Executor: Finished task 61.0 in stage 10.0 (TID 78). 6563 bytes result sent to driver -26/04/01 08:40:42 INFO TaskSetManager: Starting task 69.0 in stage 10.0 (TID 86) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:42 INFO Executor: Running task 69.0 in stage 10.0 (TID 86) -26/04/01 08:40:42 INFO TaskSetManager: Finished task 61.0 in stage 10.0 (TID 78) in 7879 ms on 10.0.0.133 (executor driver) (62/208) -26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:42 INFO Executor: Finished task 62.0 in stage 10.0 (TID 79). 6563 bytes result sent to driver -26/04/01 08:40:42 INFO TaskSetManager: Starting task 70.0 in stage 10.0 (TID 87) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:42 INFO TaskSetManager: Finished task 62.0 in stage 10.0 (TID 79) in 7955 ms on 10.0.0.133 (executor driver) (63/208) -26/04/01 08:40:42 INFO Executor: Running task 70.0 in stage 10.0 (TID 87) -26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:42 INFO Executor: Finished task 63.0 in stage 10.0 (TID 80). 6606 bytes result sent to driver -26/04/01 08:40:42 INFO TaskSetManager: Starting task 71.0 in stage 10.0 (TID 88) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:42 INFO Executor: Running task 71.0 in stage 10.0 (TID 88) -26/04/01 08:40:42 INFO TaskSetManager: Finished task 63.0 in stage 10.0 (TID 80) in 7966 ms on 10.0.0.133 (executor driver) (64/208) -26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:49 INFO Executor: Finished task 64.0 in stage 10.0 (TID 81). 6606 bytes result sent to driver -26/04/01 08:40:49 INFO TaskSetManager: Starting task 72.0 in stage 10.0 (TID 89) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:40:49 INFO TaskSetManager: Finished task 64.0 in stage 10.0 (TID 81) in 7934 ms on 10.0.0.133 (executor driver) (65/208) -26/04/01 08:40:49 INFO Executor: Running task 72.0 in stage 10.0 (TID 89) -26/04/01 08:40:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:50 INFO Executor: Finished task 65.0 in stage 10.0 (TID 82). 6606 bytes result sent to driver -26/04/01 08:40:50 INFO TaskSetManager: Starting task 73.0 in stage 10.0 (TID 90) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:40:50 INFO TaskSetManager: Finished task 65.0 in stage 10.0 (TID 82) in 7957 ms on 10.0.0.133 (executor driver) (66/208) -26/04/01 08:40:50 INFO Executor: Running task 73.0 in stage 10.0 (TID 90) -26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:50 INFO Executor: Finished task 66.0 in stage 10.0 (TID 83). 6606 bytes result sent to driver -26/04/01 08:40:50 INFO TaskSetManager: Starting task 74.0 in stage 10.0 (TID 91) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:50 INFO TaskSetManager: Finished task 66.0 in stage 10.0 (TID 83) in 7956 ms on 10.0.0.133 (executor driver) (67/208) -26/04/01 08:40:50 INFO Executor: Running task 74.0 in stage 10.0 (TID 91) -26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:50 INFO Executor: Finished task 68.0 in stage 10.0 (TID 85). 6606 bytes result sent to driver -26/04/01 08:40:50 INFO TaskSetManager: Starting task 75.0 in stage 10.0 (TID 92) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:50 INFO TaskSetManager: Finished task 68.0 in stage 10.0 (TID 85) in 7918 ms on 10.0.0.133 (executor driver) (68/208) -26/04/01 08:40:50 INFO Executor: Running task 75.0 in stage 10.0 (TID 92) -26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:50 INFO Executor: Finished task 69.0 in stage 10.0 (TID 86). 6606 bytes result sent to driver -26/04/01 08:40:50 INFO TaskSetManager: Starting task 76.0 in stage 10.0 (TID 93) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:50 INFO TaskSetManager: Finished task 69.0 in stage 10.0 (TID 86) in 7907 ms on 10.0.0.133 (executor driver) (69/208) -26/04/01 08:40:50 INFO Executor: Running task 76.0 in stage 10.0 (TID 93) -26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:50 INFO Executor: Finished task 67.0 in stage 10.0 (TID 84). 6606 bytes result sent to driver -26/04/01 08:40:50 INFO TaskSetManager: Starting task 77.0 in stage 10.0 (TID 94) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:50 INFO TaskSetManager: Finished task 67.0 in stage 10.0 (TID 84) in 7954 ms on 10.0.0.133 (executor driver) (70/208) -26/04/01 08:40:50 INFO Executor: Running task 77.0 in stage 10.0 (TID 94) -26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:50 INFO Executor: Finished task 70.0 in stage 10.0 (TID 87). 6606 bytes result sent to driver -26/04/01 08:40:50 INFO TaskSetManager: Starting task 78.0 in stage 10.0 (TID 95) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:40:50 INFO TaskSetManager: Finished task 70.0 in stage 10.0 (TID 87) in 7939 ms on 10.0.0.133 (executor driver) (71/208) -26/04/01 08:40:50 INFO Executor: Running task 78.0 in stage 10.0 (TID 95) -26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:50 INFO Executor: Finished task 71.0 in stage 10.0 (TID 88). 6563 bytes result sent to driver -26/04/01 08:40:50 INFO TaskSetManager: Starting task 79.0 in stage 10.0 (TID 96) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:50 INFO TaskSetManager: Finished task 71.0 in stage 10.0 (TID 88) in 7952 ms on 10.0.0.133 (executor driver) (72/208) -26/04/01 08:40:50 INFO Executor: Running task 79.0 in stage 10.0 (TID 96) -26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:57 INFO Executor: Finished task 72.0 in stage 10.0 (TID 89). 6563 bytes result sent to driver -26/04/01 08:40:57 INFO TaskSetManager: Starting task 80.0 in stage 10.0 (TID 97) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:57 INFO Executor: Running task 80.0 in stage 10.0 (TID 97) -26/04/01 08:40:57 INFO TaskSetManager: Finished task 72.0 in stage 10.0 (TID 89) in 7967 ms on 10.0.0.133 (executor driver) (73/208) -26/04/01 08:40:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:58 INFO Executor: Finished task 76.0 in stage 10.0 (TID 93). 6563 bytes result sent to driver -26/04/01 08:40:58 INFO TaskSetManager: Starting task 81.0 in stage 10.0 (TID 98) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:58 INFO TaskSetManager: Finished task 76.0 in stage 10.0 (TID 93) in 7924 ms on 10.0.0.133 (executor driver) (74/208) -26/04/01 08:40:58 INFO Executor: Running task 81.0 in stage 10.0 (TID 98) -26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:58 INFO Executor: Finished task 75.0 in stage 10.0 (TID 92). 6563 bytes result sent to driver -26/04/01 08:40:58 INFO TaskSetManager: Starting task 82.0 in stage 10.0 (TID 99) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:58 INFO Executor: Running task 82.0 in stage 10.0 (TID 99) -26/04/01 08:40:58 INFO TaskSetManager: Finished task 75.0 in stage 10.0 (TID 92) in 7942 ms on 10.0.0.133 (executor driver) (75/208) -26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:58 INFO Executor: Finished task 73.0 in stage 10.0 (TID 90). 6563 bytes result sent to driver -26/04/01 08:40:58 INFO TaskSetManager: Starting task 83.0 in stage 10.0 (TID 100) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:58 INFO Executor: Running task 83.0 in stage 10.0 (TID 100) -26/04/01 08:40:58 INFO TaskSetManager: Finished task 73.0 in stage 10.0 (TID 90) in 7955 ms on 10.0.0.133 (executor driver) (76/208) -26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:58 INFO Executor: Finished task 74.0 in stage 10.0 (TID 91). 6563 bytes result sent to driver -26/04/01 08:40:58 INFO TaskSetManager: Starting task 84.0 in stage 10.0 (TID 101) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:40:58 INFO TaskSetManager: Finished task 74.0 in stage 10.0 (TID 91) in 7964 ms on 10.0.0.133 (executor driver) (77/208) -26/04/01 08:40:58 INFO Executor: Running task 84.0 in stage 10.0 (TID 101) -26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:58 INFO Executor: Finished task 77.0 in stage 10.0 (TID 94). 6563 bytes result sent to driver -26/04/01 08:40:58 INFO TaskSetManager: Starting task 85.0 in stage 10.0 (TID 102) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:40:58 INFO TaskSetManager: Finished task 77.0 in stage 10.0 (TID 94) in 7940 ms on 10.0.0.133 (executor driver) (78/208) -26/04/01 08:40:58 INFO Executor: Running task 85.0 in stage 10.0 (TID 102) -26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:58 INFO Executor: Finished task 78.0 in stage 10.0 (TID 95). 6563 bytes result sent to driver -26/04/01 08:40:58 INFO TaskSetManager: Starting task 86.0 in stage 10.0 (TID 103) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:58 INFO TaskSetManager: Finished task 78.0 in stage 10.0 (TID 95) in 7968 ms on 10.0.0.133 (executor driver) (79/208) -26/04/01 08:40:58 INFO Executor: Running task 86.0 in stage 10.0 (TID 103) -26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:58 INFO Executor: Finished task 79.0 in stage 10.0 (TID 96). 6563 bytes result sent to driver -26/04/01 08:40:58 INFO TaskSetManager: Starting task 87.0 in stage 10.0 (TID 104) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:40:58 INFO Executor: Running task 87.0 in stage 10.0 (TID 104) -26/04/01 08:40:58 INFO TaskSetManager: Finished task 79.0 in stage 10.0 (TID 96) in 7948 ms on 10.0.0.133 (executor driver) (80/208) -26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:40:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:05 INFO Executor: Finished task 80.0 in stage 10.0 (TID 97). 6563 bytes result sent to driver -26/04/01 08:41:05 INFO TaskSetManager: Starting task 88.0 in stage 10.0 (TID 105) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:05 INFO TaskSetManager: Finished task 80.0 in stage 10.0 (TID 97) in 7948 ms on 10.0.0.133 (executor driver) (81/208) -26/04/01 08:41:05 INFO Executor: Running task 88.0 in stage 10.0 (TID 105) -26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:05 INFO Executor: Finished task 82.0 in stage 10.0 (TID 99). 6563 bytes result sent to driver -26/04/01 08:41:05 INFO TaskSetManager: Starting task 89.0 in stage 10.0 (TID 106) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:05 INFO TaskSetManager: Finished task 82.0 in stage 10.0 (TID 99) in 7943 ms on 10.0.0.133 (executor driver) (82/208) -26/04/01 08:41:05 INFO Executor: Running task 89.0 in stage 10.0 (TID 106) -26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:05 INFO Executor: Finished task 84.0 in stage 10.0 (TID 101). 6563 bytes result sent to driver -26/04/01 08:41:05 INFO TaskSetManager: Starting task 90.0 in stage 10.0 (TID 107) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:41:05 INFO TaskSetManager: Finished task 84.0 in stage 10.0 (TID 101) in 7932 ms on 10.0.0.133 (executor driver) (83/208) -26/04/01 08:41:05 INFO Executor: Running task 90.0 in stage 10.0 (TID 107) -26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:05 INFO Executor: Finished task 81.0 in stage 10.0 (TID 98). 6563 bytes result sent to driver -26/04/01 08:41:05 INFO TaskSetManager: Starting task 91.0 in stage 10.0 (TID 108) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:41:05 INFO TaskSetManager: Finished task 81.0 in stage 10.0 (TID 98) in 7964 ms on 10.0.0.133 (executor driver) (84/208) -26/04/01 08:41:05 INFO Executor: Running task 91.0 in stage 10.0 (TID 108) -26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:06 INFO Executor: Finished task 83.0 in stage 10.0 (TID 100). 6563 bytes result sent to driver -26/04/01 08:41:06 INFO TaskSetManager: Starting task 92.0 in stage 10.0 (TID 109) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:06 INFO Executor: Running task 92.0 in stage 10.0 (TID 109) -26/04/01 08:41:06 INFO TaskSetManager: Finished task 83.0 in stage 10.0 (TID 100) in 7970 ms on 10.0.0.133 (executor driver) (85/208) -26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:06 INFO Executor: Finished task 85.0 in stage 10.0 (TID 102). 6563 bytes result sent to driver -26/04/01 08:41:06 INFO TaskSetManager: Starting task 93.0 in stage 10.0 (TID 110) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:06 INFO TaskSetManager: Finished task 85.0 in stage 10.0 (TID 102) in 7989 ms on 10.0.0.133 (executor driver) (86/208) -26/04/01 08:41:06 INFO Executor: Running task 93.0 in stage 10.0 (TID 110) -26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:06 INFO Executor: Finished task 86.0 in stage 10.0 (TID 103). 6563 bytes result sent to driver -26/04/01 08:41:06 INFO TaskSetManager: Starting task 94.0 in stage 10.0 (TID 111) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:06 INFO TaskSetManager: Finished task 86.0 in stage 10.0 (TID 103) in 7968 ms on 10.0.0.133 (executor driver) (87/208) -26/04/01 08:41:06 INFO Executor: Running task 94.0 in stage 10.0 (TID 111) -26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:06 INFO Executor: Finished task 87.0 in stage 10.0 (TID 104). 6563 bytes result sent to driver -26/04/01 08:41:06 INFO TaskSetManager: Starting task 95.0 in stage 10.0 (TID 112) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:06 INFO TaskSetManager: Finished task 87.0 in stage 10.0 (TID 104) in 7952 ms on 10.0.0.133 (executor driver) (88/208) -26/04/01 08:41:06 INFO Executor: Running task 95.0 in stage 10.0 (TID 112) -26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:13 INFO Executor: Finished task 91.0 in stage 10.0 (TID 108). 6563 bytes result sent to driver -26/04/01 08:41:13 INFO TaskSetManager: Starting task 96.0 in stage 10.0 (TID 113) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:41:13 INFO TaskSetManager: Finished task 91.0 in stage 10.0 (TID 108) in 7664 ms on 10.0.0.133 (executor driver) (89/208) -26/04/01 08:41:13 INFO Executor: Running task 96.0 in stage 10.0 (TID 113) -26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:13 INFO Executor: Finished task 88.0 in stage 10.0 (TID 105). 6563 bytes result sent to driver -26/04/01 08:41:13 INFO TaskSetManager: Starting task 97.0 in stage 10.0 (TID 114) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:41:13 INFO Executor: Running task 97.0 in stage 10.0 (TID 114) -26/04/01 08:41:13 INFO TaskSetManager: Finished task 88.0 in stage 10.0 (TID 105) in 7930 ms on 10.0.0.133 (executor driver) (90/208) -26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:13 INFO Executor: Finished task 89.0 in stage 10.0 (TID 106). 6563 bytes result sent to driver -26/04/01 08:41:13 INFO TaskSetManager: Starting task 98.0 in stage 10.0 (TID 115) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:41:13 INFO TaskSetManager: Finished task 89.0 in stage 10.0 (TID 106) in 7922 ms on 10.0.0.133 (executor driver) (91/208) -26/04/01 08:41:13 INFO Executor: Running task 98.0 in stage 10.0 (TID 115) -26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:13 INFO Executor: Finished task 93.0 in stage 10.0 (TID 110). 6563 bytes result sent to driver -26/04/01 08:41:13 INFO TaskSetManager: Starting task 99.0 in stage 10.0 (TID 116) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:41:13 INFO TaskSetManager: Finished task 93.0 in stage 10.0 (TID 110) in 7904 ms on 10.0.0.133 (executor driver) (92/208) -26/04/01 08:41:13 INFO Executor: Running task 99.0 in stage 10.0 (TID 116) -26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:13 INFO Executor: Finished task 92.0 in stage 10.0 (TID 109). 6563 bytes result sent to driver -26/04/01 08:41:13 INFO TaskSetManager: Starting task 100.0 in stage 10.0 (TID 117) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:41:13 INFO TaskSetManager: Finished task 92.0 in stage 10.0 (TID 109) in 7959 ms on 10.0.0.133 (executor driver) (93/208) -26/04/01 08:41:13 INFO Executor: Running task 100.0 in stage 10.0 (TID 117) -26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:14 INFO Executor: Finished task 94.0 in stage 10.0 (TID 111). 6606 bytes result sent to driver -26/04/01 08:41:14 INFO TaskSetManager: Starting task 101.0 in stage 10.0 (TID 118) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:41:14 INFO TaskSetManager: Finished task 94.0 in stage 10.0 (TID 111) in 7963 ms on 10.0.0.133 (executor driver) (94/208) -26/04/01 08:41:14 INFO Executor: Running task 101.0 in stage 10.0 (TID 118) -26/04/01 08:41:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:14 INFO Executor: Finished task 90.0 in stage 10.0 (TID 107). 6606 bytes result sent to driver -26/04/01 08:41:14 INFO TaskSetManager: Starting task 102.0 in stage 10.0 (TID 119) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:41:14 INFO TaskSetManager: Finished task 90.0 in stage 10.0 (TID 107) in 8226 ms on 10.0.0.133 (executor driver) (95/208) -26/04/01 08:41:14 INFO Executor: Running task 102.0 in stage 10.0 (TID 119) -26/04/01 08:41:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:14 INFO Executor: Finished task 95.0 in stage 10.0 (TID 112). 6606 bytes result sent to driver -26/04/01 08:41:14 INFO TaskSetManager: Starting task 103.0 in stage 10.0 (TID 120) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:41:14 INFO Executor: Running task 103.0 in stage 10.0 (TID 120) -26/04/01 08:41:14 INFO TaskSetManager: Finished task 95.0 in stage 10.0 (TID 112) in 7954 ms on 10.0.0.133 (executor driver) (96/208) -26/04/01 08:41:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:21 INFO Executor: Finished task 96.0 in stage 10.0 (TID 113). 6606 bytes result sent to driver -26/04/01 08:41:21 INFO TaskSetManager: Starting task 104.0 in stage 10.0 (TID 121) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:21 INFO TaskSetManager: Finished task 96.0 in stage 10.0 (TID 113) in 7999 ms on 10.0.0.133 (executor driver) (97/208) -26/04/01 08:41:21 INFO Executor: Running task 104.0 in stage 10.0 (TID 121) -26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:21 INFO Executor: Finished task 97.0 in stage 10.0 (TID 114). 6606 bytes result sent to driver -26/04/01 08:41:21 INFO TaskSetManager: Starting task 105.0 in stage 10.0 (TID 122) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:21 INFO TaskSetManager: Finished task 97.0 in stage 10.0 (TID 114) in 7992 ms on 10.0.0.133 (executor driver) (98/208) -26/04/01 08:41:21 INFO Executor: Running task 105.0 in stage 10.0 (TID 122) -26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:21 INFO Executor: Finished task 98.0 in stage 10.0 (TID 115). 6606 bytes result sent to driver -26/04/01 08:41:21 INFO TaskSetManager: Starting task 106.0 in stage 10.0 (TID 123) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:21 INFO Executor: Running task 106.0 in stage 10.0 (TID 123) -26/04/01 08:41:21 INFO TaskSetManager: Finished task 98.0 in stage 10.0 (TID 115) in 7985 ms on 10.0.0.133 (executor driver) (99/208) -26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:21 INFO Executor: Finished task 99.0 in stage 10.0 (TID 116). 6606 bytes result sent to driver -26/04/01 08:41:21 INFO TaskSetManager: Starting task 107.0 in stage 10.0 (TID 124) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:21 INFO TaskSetManager: Finished task 99.0 in stage 10.0 (TID 116) in 7964 ms on 10.0.0.133 (executor driver) (100/208) -26/04/01 08:41:21 INFO Executor: Running task 107.0 in stage 10.0 (TID 124) -26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:21 INFO Executor: Finished task 100.0 in stage 10.0 (TID 117). 6606 bytes result sent to driver -26/04/01 08:41:21 INFO TaskSetManager: Starting task 108.0 in stage 10.0 (TID 125) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:41:21 INFO Executor: Running task 108.0 in stage 10.0 (TID 125) -26/04/01 08:41:21 INFO TaskSetManager: Finished task 100.0 in stage 10.0 (TID 117) in 8000 ms on 10.0.0.133 (executor driver) (101/208) -26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:22 INFO Executor: Finished task 101.0 in stage 10.0 (TID 118). 6563 bytes result sent to driver -26/04/01 08:41:22 INFO TaskSetManager: Starting task 109.0 in stage 10.0 (TID 126) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:41:22 INFO TaskSetManager: Finished task 101.0 in stage 10.0 (TID 118) in 8013 ms on 10.0.0.133 (executor driver) (102/208) -26/04/01 08:41:22 INFO Executor: Running task 109.0 in stage 10.0 (TID 126) -26/04/01 08:41:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:22 INFO Executor: Finished task 102.0 in stage 10.0 (TID 119). 6563 bytes result sent to driver -26/04/01 08:41:22 INFO TaskSetManager: Starting task 110.0 in stage 10.0 (TID 127) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:41:22 INFO TaskSetManager: Finished task 102.0 in stage 10.0 (TID 119) in 7955 ms on 10.0.0.133 (executor driver) (103/208) -26/04/01 08:41:22 INFO Executor: Running task 110.0 in stage 10.0 (TID 127) -26/04/01 08:41:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:22 INFO Executor: Finished task 103.0 in stage 10.0 (TID 120). 6563 bytes result sent to driver -26/04/01 08:41:22 INFO TaskSetManager: Starting task 111.0 in stage 10.0 (TID 128) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:41:22 INFO TaskSetManager: Finished task 103.0 in stage 10.0 (TID 120) in 8005 ms on 10.0.0.133 (executor driver) (104/208) -26/04/01 08:41:22 INFO Executor: Running task 111.0 in stage 10.0 (TID 128) -26/04/01 08:41:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:29 INFO Executor: Finished task 104.0 in stage 10.0 (TID 121). 6563 bytes result sent to driver -26/04/01 08:41:29 INFO TaskSetManager: Starting task 112.0 in stage 10.0 (TID 129) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:41:29 INFO TaskSetManager: Finished task 104.0 in stage 10.0 (TID 121) in 7957 ms on 10.0.0.133 (executor driver) (105/208) -26/04/01 08:41:29 INFO Executor: Running task 112.0 in stage 10.0 (TID 129) -26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:29 INFO Executor: Finished task 105.0 in stage 10.0 (TID 122). 6563 bytes result sent to driver -26/04/01 08:41:29 INFO TaskSetManager: Starting task 113.0 in stage 10.0 (TID 130) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:41:29 INFO TaskSetManager: Finished task 105.0 in stage 10.0 (TID 122) in 7957 ms on 10.0.0.133 (executor driver) (106/208) -26/04/01 08:41:29 INFO Executor: Running task 113.0 in stage 10.0 (TID 130) -26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:29 INFO Executor: Finished task 107.0 in stage 10.0 (TID 124). 6563 bytes result sent to driver -26/04/01 08:41:29 INFO TaskSetManager: Starting task 114.0 in stage 10.0 (TID 131) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:41:29 INFO TaskSetManager: Finished task 107.0 in stage 10.0 (TID 124) in 7949 ms on 10.0.0.133 (executor driver) (107/208) -26/04/01 08:41:29 INFO Executor: Running task 114.0 in stage 10.0 (TID 131) -26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:29 INFO Executor: Finished task 106.0 in stage 10.0 (TID 123). 6563 bytes result sent to driver -26/04/01 08:41:29 INFO TaskSetManager: Starting task 115.0 in stage 10.0 (TID 132) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:41:29 INFO TaskSetManager: Finished task 106.0 in stage 10.0 (TID 123) in 8001 ms on 10.0.0.133 (executor driver) (108/208) -26/04/01 08:41:29 INFO Executor: Running task 115.0 in stage 10.0 (TID 132) -26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:29 INFO Executor: Finished task 108.0 in stage 10.0 (TID 125). 6563 bytes result sent to driver -26/04/01 08:41:29 INFO TaskSetManager: Starting task 116.0 in stage 10.0 (TID 133) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:29 INFO TaskSetManager: Finished task 108.0 in stage 10.0 (TID 125) in 7953 ms on 10.0.0.133 (executor driver) (109/208) -26/04/01 08:41:29 INFO Executor: Running task 116.0 in stage 10.0 (TID 133) -26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:30 INFO Executor: Finished task 109.0 in stage 10.0 (TID 126). 6563 bytes result sent to driver -26/04/01 08:41:30 INFO TaskSetManager: Starting task 117.0 in stage 10.0 (TID 134) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:30 INFO Executor: Running task 117.0 in stage 10.0 (TID 134) -26/04/01 08:41:30 INFO TaskSetManager: Finished task 109.0 in stage 10.0 (TID 126) in 7943 ms on 10.0.0.133 (executor driver) (110/208) -26/04/01 08:41:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:30 INFO Executor: Finished task 110.0 in stage 10.0 (TID 127). 6563 bytes result sent to driver -26/04/01 08:41:30 INFO TaskSetManager: Starting task 118.0 in stage 10.0 (TID 135) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:30 INFO Executor: Running task 118.0 in stage 10.0 (TID 135) -26/04/01 08:41:30 INFO TaskSetManager: Finished task 110.0 in stage 10.0 (TID 127) in 7976 ms on 10.0.0.133 (executor driver) (111/208) -26/04/01 08:41:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:30 INFO Executor: Finished task 111.0 in stage 10.0 (TID 128). 6563 bytes result sent to driver -26/04/01 08:41:30 INFO TaskSetManager: Starting task 119.0 in stage 10.0 (TID 136) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:30 INFO TaskSetManager: Finished task 111.0 in stage 10.0 (TID 128) in 7974 ms on 10.0.0.133 (executor driver) (112/208) -26/04/01 08:41:30 INFO Executor: Running task 119.0 in stage 10.0 (TID 136) -26/04/01 08:41:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:37 INFO Executor: Finished task 112.0 in stage 10.0 (TID 129). 6563 bytes result sent to driver -26/04/01 08:41:37 INFO TaskSetManager: Starting task 120.0 in stage 10.0 (TID 137) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:41:37 INFO TaskSetManager: Finished task 112.0 in stage 10.0 (TID 129) in 7995 ms on 10.0.0.133 (executor driver) (113/208) -26/04/01 08:41:37 INFO Executor: Running task 120.0 in stage 10.0 (TID 137) -26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:37 INFO Executor: Finished task 113.0 in stage 10.0 (TID 130). 6563 bytes result sent to driver -26/04/01 08:41:37 INFO TaskSetManager: Starting task 121.0 in stage 10.0 (TID 138) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:41:37 INFO TaskSetManager: Finished task 113.0 in stage 10.0 (TID 130) in 7981 ms on 10.0.0.133 (executor driver) (114/208) -26/04/01 08:41:37 INFO Executor: Running task 121.0 in stage 10.0 (TID 138) -26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:37 INFO Executor: Finished task 114.0 in stage 10.0 (TID 131). 6563 bytes result sent to driver -26/04/01 08:41:37 INFO TaskSetManager: Starting task 122.0 in stage 10.0 (TID 139) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:37 INFO TaskSetManager: Finished task 114.0 in stage 10.0 (TID 131) in 7991 ms on 10.0.0.133 (executor driver) (115/208) -26/04/01 08:41:37 INFO Executor: Running task 122.0 in stage 10.0 (TID 139) -26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:37 INFO Executor: Finished task 115.0 in stage 10.0 (TID 132). 6563 bytes result sent to driver -26/04/01 08:41:37 INFO TaskSetManager: Starting task 123.0 in stage 10.0 (TID 140) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:37 INFO TaskSetManager: Finished task 115.0 in stage 10.0 (TID 132) in 7978 ms on 10.0.0.133 (executor driver) (116/208) -26/04/01 08:41:37 INFO Executor: Running task 123.0 in stage 10.0 (TID 140) -26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:37 INFO Executor: Finished task 116.0 in stage 10.0 (TID 133). 6563 bytes result sent to driver -26/04/01 08:41:37 INFO TaskSetManager: Starting task 124.0 in stage 10.0 (TID 141) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:37 INFO TaskSetManager: Finished task 116.0 in stage 10.0 (TID 133) in 7968 ms on 10.0.0.133 (executor driver) (117/208) -26/04/01 08:41:37 INFO Executor: Running task 124.0 in stage 10.0 (TID 141) -26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:38 INFO Executor: Finished task 117.0 in stage 10.0 (TID 134). 6563 bytes result sent to driver -26/04/01 08:41:38 INFO TaskSetManager: Starting task 125.0 in stage 10.0 (TID 142) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:38 INFO TaskSetManager: Finished task 117.0 in stage 10.0 (TID 134) in 7972 ms on 10.0.0.133 (executor driver) (118/208) -26/04/01 08:41:38 INFO Executor: Running task 125.0 in stage 10.0 (TID 142) -26/04/01 08:41:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:38 INFO Executor: Finished task 118.0 in stage 10.0 (TID 135). 6563 bytes result sent to driver -26/04/01 08:41:38 INFO TaskSetManager: Starting task 126.0 in stage 10.0 (TID 143) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:41:38 INFO Executor: Running task 126.0 in stage 10.0 (TID 143) -26/04/01 08:41:38 INFO TaskSetManager: Finished task 118.0 in stage 10.0 (TID 135) in 7967 ms on 10.0.0.133 (executor driver) (119/208) -26/04/01 08:41:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:38 INFO Executor: Finished task 119.0 in stage 10.0 (TID 136). 6563 bytes result sent to driver -26/04/01 08:41:38 INFO TaskSetManager: Starting task 127.0 in stage 10.0 (TID 144) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:41:38 INFO TaskSetManager: Finished task 119.0 in stage 10.0 (TID 136) in 7987 ms on 10.0.0.133 (executor driver) (120/208) -26/04/01 08:41:38 INFO Executor: Running task 127.0 in stage 10.0 (TID 144) -26/04/01 08:41:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:45 INFO Executor: Finished task 120.0 in stage 10.0 (TID 137). 6563 bytes result sent to driver -26/04/01 08:41:45 INFO TaskSetManager: Starting task 128.0 in stage 10.0 (TID 145) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:45 INFO TaskSetManager: Finished task 120.0 in stage 10.0 (TID 137) in 7976 ms on 10.0.0.133 (executor driver) (121/208) -26/04/01 08:41:45 INFO Executor: Running task 128.0 in stage 10.0 (TID 145) -26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:45 INFO Executor: Finished task 121.0 in stage 10.0 (TID 138). 6649 bytes result sent to driver -26/04/01 08:41:45 INFO TaskSetManager: Starting task 129.0 in stage 10.0 (TID 146) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:45 INFO TaskSetManager: Finished task 121.0 in stage 10.0 (TID 138) in 7953 ms on 10.0.0.133 (executor driver) (122/208) -26/04/01 08:41:45 INFO Executor: Running task 129.0 in stage 10.0 (TID 146) -26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:45 INFO Executor: Finished task 122.0 in stage 10.0 (TID 139). 6606 bytes result sent to driver -26/04/01 08:41:45 INFO TaskSetManager: Starting task 130.0 in stage 10.0 (TID 147) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:45 INFO TaskSetManager: Finished task 122.0 in stage 10.0 (TID 139) in 7934 ms on 10.0.0.133 (executor driver) (123/208) -26/04/01 08:41:45 INFO Executor: Running task 130.0 in stage 10.0 (TID 147) -26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:45 INFO Executor: Finished task 123.0 in stage 10.0 (TID 140). 6606 bytes result sent to driver -26/04/01 08:41:45 INFO TaskSetManager: Starting task 131.0 in stage 10.0 (TID 148) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:45 INFO TaskSetManager: Finished task 123.0 in stage 10.0 (TID 140) in 7930 ms on 10.0.0.133 (executor driver) (124/208) -26/04/01 08:41:45 INFO Executor: Running task 131.0 in stage 10.0 (TID 148) -26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:45 INFO Executor: Finished task 124.0 in stage 10.0 (TID 141). 6606 bytes result sent to driver -26/04/01 08:41:45 INFO TaskSetManager: Starting task 132.0 in stage 10.0 (TID 149) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:41:45 INFO Executor: Running task 132.0 in stage 10.0 (TID 149) -26/04/01 08:41:45 INFO TaskSetManager: Finished task 124.0 in stage 10.0 (TID 141) in 7962 ms on 10.0.0.133 (executor driver) (125/208) -26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:46 INFO Executor: Finished task 125.0 in stage 10.0 (TID 142). 6606 bytes result sent to driver -26/04/01 08:41:46 INFO TaskSetManager: Starting task 133.0 in stage 10.0 (TID 150) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:41:46 INFO TaskSetManager: Finished task 125.0 in stage 10.0 (TID 142) in 8083 ms on 10.0.0.133 (executor driver) (126/208) -26/04/01 08:41:46 INFO Executor: Running task 133.0 in stage 10.0 (TID 150) -26/04/01 08:41:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:46 INFO Executor: Finished task 127.0 in stage 10.0 (TID 144). 6606 bytes result sent to driver -26/04/01 08:41:46 INFO TaskSetManager: Starting task 134.0 in stage 10.0 (TID 151) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:46 INFO Executor: Running task 134.0 in stage 10.0 (TID 151) -26/04/01 08:41:46 INFO TaskSetManager: Finished task 127.0 in stage 10.0 (TID 144) in 7968 ms on 10.0.0.133 (executor driver) (127/208) -26/04/01 08:41:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:46 INFO Executor: Finished task 126.0 in stage 10.0 (TID 143). 6606 bytes result sent to driver -26/04/01 08:41:46 INFO TaskSetManager: Starting task 135.0 in stage 10.0 (TID 152) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:46 INFO TaskSetManager: Finished task 126.0 in stage 10.0 (TID 143) in 8172 ms on 10.0.0.133 (executor driver) (128/208) -26/04/01 08:41:46 INFO Executor: Running task 135.0 in stage 10.0 (TID 152) -26/04/01 08:41:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:53 INFO Executor: Finished task 128.0 in stage 10.0 (TID 145). 6606 bytes result sent to driver -26/04/01 08:41:53 INFO TaskSetManager: Starting task 136.0 in stage 10.0 (TID 153) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:53 INFO TaskSetManager: Finished task 128.0 in stage 10.0 (TID 145) in 7947 ms on 10.0.0.133 (executor driver) (129/208) -26/04/01 08:41:53 INFO Executor: Running task 136.0 in stage 10.0 (TID 153) -26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:53 INFO Executor: Finished task 129.0 in stage 10.0 (TID 146). 6563 bytes result sent to driver -26/04/01 08:41:53 INFO TaskSetManager: Starting task 137.0 in stage 10.0 (TID 154) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:53 INFO TaskSetManager: Finished task 129.0 in stage 10.0 (TID 146) in 7978 ms on 10.0.0.133 (executor driver) (130/208) -26/04/01 08:41:53 INFO Executor: Running task 137.0 in stage 10.0 (TID 154) -26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:53 INFO Executor: Finished task 130.0 in stage 10.0 (TID 147). 6563 bytes result sent to driver -26/04/01 08:41:53 INFO TaskSetManager: Starting task 138.0 in stage 10.0 (TID 155) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:41:53 INFO TaskSetManager: Finished task 130.0 in stage 10.0 (TID 147) in 7950 ms on 10.0.0.133 (executor driver) (131/208) -26/04/01 08:41:53 INFO Executor: Running task 138.0 in stage 10.0 (TID 155) -26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:53 INFO Executor: Finished task 131.0 in stage 10.0 (TID 148). 6563 bytes result sent to driver -26/04/01 08:41:53 INFO TaskSetManager: Starting task 139.0 in stage 10.0 (TID 156) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:41:53 INFO TaskSetManager: Finished task 131.0 in stage 10.0 (TID 148) in 7957 ms on 10.0.0.133 (executor driver) (132/208) -26/04/01 08:41:53 INFO Executor: Running task 139.0 in stage 10.0 (TID 156) -26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:53 INFO Executor: Finished task 132.0 in stage 10.0 (TID 149). 6563 bytes result sent to driver -26/04/01 08:41:53 INFO TaskSetManager: Starting task 140.0 in stage 10.0 (TID 157) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:53 INFO TaskSetManager: Finished task 132.0 in stage 10.0 (TID 149) in 7951 ms on 10.0.0.133 (executor driver) (133/208) -26/04/01 08:41:53 INFO Executor: Running task 140.0 in stage 10.0 (TID 157) -26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:54 INFO Executor: Finished task 133.0 in stage 10.0 (TID 150). 6563 bytes result sent to driver -26/04/01 08:41:54 INFO TaskSetManager: Starting task 141.0 in stage 10.0 (TID 158) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:54 INFO TaskSetManager: Finished task 133.0 in stage 10.0 (TID 150) in 7933 ms on 10.0.0.133 (executor driver) (134/208) -26/04/01 08:41:54 INFO Executor: Running task 141.0 in stage 10.0 (TID 158) -26/04/01 08:41:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:54 INFO Executor: Finished task 134.0 in stage 10.0 (TID 151). 6563 bytes result sent to driver -26/04/01 08:41:54 INFO TaskSetManager: Starting task 142.0 in stage 10.0 (TID 159) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:54 INFO TaskSetManager: Finished task 134.0 in stage 10.0 (TID 151) in 7884 ms on 10.0.0.133 (executor driver) (135/208) -26/04/01 08:41:54 INFO Executor: Running task 142.0 in stage 10.0 (TID 159) -26/04/01 08:41:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:54 INFO Executor: Finished task 135.0 in stage 10.0 (TID 152). 6563 bytes result sent to driver -26/04/01 08:41:54 INFO TaskSetManager: Starting task 143.0 in stage 10.0 (TID 160) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:41:54 INFO TaskSetManager: Finished task 135.0 in stage 10.0 (TID 152) in 7951 ms on 10.0.0.133 (executor driver) (136/208) -26/04/01 08:41:54 INFO Executor: Running task 143.0 in stage 10.0 (TID 160) -26/04/01 08:41:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:41:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:01 INFO Executor: Finished task 136.0 in stage 10.0 (TID 153). 6563 bytes result sent to driver -26/04/01 08:42:01 INFO TaskSetManager: Starting task 144.0 in stage 10.0 (TID 161) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:42:01 INFO TaskSetManager: Finished task 136.0 in stage 10.0 (TID 153) in 7945 ms on 10.0.0.133 (executor driver) (137/208) -26/04/01 08:42:01 INFO Executor: Running task 144.0 in stage 10.0 (TID 161) -26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:01 INFO Executor: Finished task 137.0 in stage 10.0 (TID 154). 6563 bytes result sent to driver -26/04/01 08:42:01 INFO TaskSetManager: Starting task 145.0 in stage 10.0 (TID 162) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:42:01 INFO TaskSetManager: Finished task 137.0 in stage 10.0 (TID 154) in 7962 ms on 10.0.0.133 (executor driver) (138/208) -26/04/01 08:42:01 INFO Executor: Running task 145.0 in stage 10.0 (TID 162) -26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:01 INFO Executor: Finished task 139.0 in stage 10.0 (TID 156). 6563 bytes result sent to driver -26/04/01 08:42:01 INFO TaskSetManager: Starting task 146.0 in stage 10.0 (TID 163) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:01 INFO TaskSetManager: Finished task 139.0 in stage 10.0 (TID 156) in 7940 ms on 10.0.0.133 (executor driver) (139/208) -26/04/01 08:42:01 INFO Executor: Running task 146.0 in stage 10.0 (TID 163) -26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:01 INFO Executor: Finished task 138.0 in stage 10.0 (TID 155). 6563 bytes result sent to driver -26/04/01 08:42:01 INFO TaskSetManager: Starting task 147.0 in stage 10.0 (TID 164) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:01 INFO TaskSetManager: Finished task 138.0 in stage 10.0 (TID 155) in 7965 ms on 10.0.0.133 (executor driver) (140/208) -26/04/01 08:42:01 INFO Executor: Running task 147.0 in stage 10.0 (TID 164) -26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:01 INFO Executor: Finished task 140.0 in stage 10.0 (TID 157). 6563 bytes result sent to driver -26/04/01 08:42:01 INFO TaskSetManager: Starting task 148.0 in stage 10.0 (TID 165) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:01 INFO TaskSetManager: Finished task 140.0 in stage 10.0 (TID 157) in 7956 ms on 10.0.0.133 (executor driver) (141/208) -26/04/01 08:42:01 INFO Executor: Running task 148.0 in stage 10.0 (TID 165) -26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:02 INFO Executor: Finished task 141.0 in stage 10.0 (TID 158). 6563 bytes result sent to driver -26/04/01 08:42:02 INFO TaskSetManager: Starting task 149.0 in stage 10.0 (TID 166) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:02 INFO TaskSetManager: Finished task 141.0 in stage 10.0 (TID 158) in 7945 ms on 10.0.0.133 (executor driver) (142/208) -26/04/01 08:42:02 INFO Executor: Running task 149.0 in stage 10.0 (TID 166) -26/04/01 08:42:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:02 INFO Executor: Finished task 142.0 in stage 10.0 (TID 159). 6563 bytes result sent to driver -26/04/01 08:42:02 INFO TaskSetManager: Starting task 150.0 in stage 10.0 (TID 167) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:42:02 INFO Executor: Running task 150.0 in stage 10.0 (TID 167) -26/04/01 08:42:02 INFO TaskSetManager: Finished task 142.0 in stage 10.0 (TID 159) in 7920 ms on 10.0.0.133 (executor driver) (143/208) -26/04/01 08:42:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:02 INFO Executor: Finished task 143.0 in stage 10.0 (TID 160). 6563 bytes result sent to driver -26/04/01 08:42:02 INFO TaskSetManager: Starting task 151.0 in stage 10.0 (TID 168) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:02 INFO Executor: Running task 151.0 in stage 10.0 (TID 168) -26/04/01 08:42:02 INFO TaskSetManager: Finished task 143.0 in stage 10.0 (TID 160) in 7924 ms on 10.0.0.133 (executor driver) (144/208) -26/04/01 08:42:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:09 INFO Executor: Finished task 144.0 in stage 10.0 (TID 161). 6563 bytes result sent to driver -26/04/01 08:42:09 INFO TaskSetManager: Starting task 152.0 in stage 10.0 (TID 169) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:09 INFO TaskSetManager: Finished task 144.0 in stage 10.0 (TID 161) in 8031 ms on 10.0.0.133 (executor driver) (145/208) -26/04/01 08:42:09 INFO Executor: Running task 152.0 in stage 10.0 (TID 169) -26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:09 INFO Executor: Finished task 145.0 in stage 10.0 (TID 162). 6563 bytes result sent to driver -26/04/01 08:42:09 INFO TaskSetManager: Starting task 153.0 in stage 10.0 (TID 170) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:09 INFO TaskSetManager: Finished task 145.0 in stage 10.0 (TID 162) in 8011 ms on 10.0.0.133 (executor driver) (146/208) -26/04/01 08:42:09 INFO Executor: Running task 153.0 in stage 10.0 (TID 170) -26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:09 INFO Executor: Finished task 147.0 in stage 10.0 (TID 164). 6563 bytes result sent to driver -26/04/01 08:42:09 INFO TaskSetManager: Starting task 154.0 in stage 10.0 (TID 171) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:09 INFO TaskSetManager: Finished task 147.0 in stage 10.0 (TID 164) in 7995 ms on 10.0.0.133 (executor driver) (147/208) -26/04/01 08:42:09 INFO Executor: Running task 154.0 in stage 10.0 (TID 171) -26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:09 INFO Executor: Finished task 146.0 in stage 10.0 (TID 163). 6563 bytes result sent to driver -26/04/01 08:42:09 INFO TaskSetManager: Starting task 155.0 in stage 10.0 (TID 172) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:09 INFO TaskSetManager: Finished task 146.0 in stage 10.0 (TID 163) in 8012 ms on 10.0.0.133 (executor driver) (148/208) -26/04/01 08:42:09 INFO Executor: Running task 155.0 in stage 10.0 (TID 172) -26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:09 INFO Executor: Finished task 148.0 in stage 10.0 (TID 165). 6563 bytes result sent to driver -26/04/01 08:42:09 INFO TaskSetManager: Starting task 156.0 in stage 10.0 (TID 173) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:42:09 INFO TaskSetManager: Finished task 148.0 in stage 10.0 (TID 165) in 7977 ms on 10.0.0.133 (executor driver) (149/208) -26/04/01 08:42:09 INFO Executor: Running task 156.0 in stage 10.0 (TID 173) -26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:10 INFO Executor: Finished task 149.0 in stage 10.0 (TID 166). 6563 bytes result sent to driver -26/04/01 08:42:10 INFO TaskSetManager: Starting task 157.0 in stage 10.0 (TID 174) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:10 INFO TaskSetManager: Finished task 149.0 in stage 10.0 (TID 166) in 7996 ms on 10.0.0.133 (executor driver) (150/208) -26/04/01 08:42:10 INFO Executor: Running task 157.0 in stage 10.0 (TID 174) -26/04/01 08:42:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:10 INFO Executor: Finished task 150.0 in stage 10.0 (TID 167). 6563 bytes result sent to driver -26/04/01 08:42:10 INFO TaskSetManager: Starting task 158.0 in stage 10.0 (TID 175) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:10 INFO TaskSetManager: Finished task 150.0 in stage 10.0 (TID 167) in 7975 ms on 10.0.0.133 (executor driver) (151/208) -26/04/01 08:42:10 INFO Executor: Running task 158.0 in stage 10.0 (TID 175) -26/04/01 08:42:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:10 INFO Executor: Finished task 151.0 in stage 10.0 (TID 168). 6563 bytes result sent to driver -26/04/01 08:42:10 INFO TaskSetManager: Starting task 159.0 in stage 10.0 (TID 176) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:10 INFO TaskSetManager: Finished task 151.0 in stage 10.0 (TID 168) in 8009 ms on 10.0.0.133 (executor driver) (152/208) -26/04/01 08:42:10 INFO Executor: Running task 159.0 in stage 10.0 (TID 176) -26/04/01 08:42:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:17 INFO Executor: Finished task 152.0 in stage 10.0 (TID 169). 6606 bytes result sent to driver -26/04/01 08:42:17 INFO TaskSetManager: Starting task 160.0 in stage 10.0 (TID 177) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:17 INFO TaskSetManager: Finished task 152.0 in stage 10.0 (TID 169) in 8030 ms on 10.0.0.133 (executor driver) (153/208) -26/04/01 08:42:17 INFO Executor: Running task 160.0 in stage 10.0 (TID 177) -26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:17 INFO Executor: Finished task 153.0 in stage 10.0 (TID 170). 6606 bytes result sent to driver -26/04/01 08:42:17 INFO TaskSetManager: Starting task 161.0 in stage 10.0 (TID 178) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:17 INFO TaskSetManager: Finished task 153.0 in stage 10.0 (TID 170) in 8009 ms on 10.0.0.133 (executor driver) (154/208) -26/04/01 08:42:17 INFO Executor: Running task 161.0 in stage 10.0 (TID 178) -26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:17 INFO Executor: Finished task 154.0 in stage 10.0 (TID 171). 6606 bytes result sent to driver -26/04/01 08:42:17 INFO TaskSetManager: Starting task 162.0 in stage 10.0 (TID 179) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:42:17 INFO TaskSetManager: Finished task 154.0 in stage 10.0 (TID 171) in 8046 ms on 10.0.0.133 (executor driver) (155/208) -26/04/01 08:42:17 INFO Executor: Running task 162.0 in stage 10.0 (TID 179) -26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:17 INFO Executor: Finished task 155.0 in stage 10.0 (TID 172). 6606 bytes result sent to driver -26/04/01 08:42:17 INFO TaskSetManager: Starting task 163.0 in stage 10.0 (TID 180) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:42:17 INFO TaskSetManager: Finished task 155.0 in stage 10.0 (TID 172) in 8048 ms on 10.0.0.133 (executor driver) (156/208) -26/04/01 08:42:17 INFO Executor: Running task 163.0 in stage 10.0 (TID 180) -26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:17 INFO Executor: Finished task 156.0 in stage 10.0 (TID 173). 6606 bytes result sent to driver -26/04/01 08:42:17 INFO TaskSetManager: Starting task 164.0 in stage 10.0 (TID 181) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:17 INFO TaskSetManager: Finished task 156.0 in stage 10.0 (TID 173) in 8036 ms on 10.0.0.133 (executor driver) (157/208) -26/04/01 08:42:17 INFO Executor: Running task 164.0 in stage 10.0 (TID 181) -26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:18 INFO Executor: Finished task 157.0 in stage 10.0 (TID 174). 6606 bytes result sent to driver -26/04/01 08:42:18 INFO TaskSetManager: Starting task 165.0 in stage 10.0 (TID 182) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:18 INFO TaskSetManager: Finished task 157.0 in stage 10.0 (TID 174) in 8078 ms on 10.0.0.133 (executor driver) (158/208) -26/04/01 08:42:18 INFO Executor: Running task 165.0 in stage 10.0 (TID 182) -26/04/01 08:42:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:18 INFO Executor: Finished task 158.0 in stage 10.0 (TID 175). 6606 bytes result sent to driver -26/04/01 08:42:18 INFO TaskSetManager: Starting task 166.0 in stage 10.0 (TID 183) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:18 INFO TaskSetManager: Finished task 158.0 in stage 10.0 (TID 175) in 8103 ms on 10.0.0.133 (executor driver) (159/208) -26/04/01 08:42:18 INFO Executor: Running task 166.0 in stage 10.0 (TID 183) -26/04/01 08:42:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:18 INFO Executor: Finished task 159.0 in stage 10.0 (TID 176). 6606 bytes result sent to driver -26/04/01 08:42:18 INFO TaskSetManager: Starting task 167.0 in stage 10.0 (TID 184) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:18 INFO Executor: Running task 167.0 in stage 10.0 (TID 184) -26/04/01 08:42:18 INFO TaskSetManager: Finished task 159.0 in stage 10.0 (TID 176) in 8053 ms on 10.0.0.133 (executor driver) (160/208) -26/04/01 08:42:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:25 INFO Executor: Finished task 160.0 in stage 10.0 (TID 177). 6563 bytes result sent to driver -26/04/01 08:42:25 INFO TaskSetManager: Starting task 168.0 in stage 10.0 (TID 185) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:42:25 INFO TaskSetManager: Finished task 160.0 in stage 10.0 (TID 177) in 8041 ms on 10.0.0.133 (executor driver) (161/208) -26/04/01 08:42:25 INFO Executor: Running task 168.0 in stage 10.0 (TID 185) -26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:25 INFO Executor: Finished task 161.0 in stage 10.0 (TID 178). 6563 bytes result sent to driver -26/04/01 08:42:25 INFO TaskSetManager: Starting task 169.0 in stage 10.0 (TID 186) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:25 INFO TaskSetManager: Finished task 161.0 in stage 10.0 (TID 178) in 8091 ms on 10.0.0.133 (executor driver) (162/208) -26/04/01 08:42:25 INFO Executor: Running task 169.0 in stage 10.0 (TID 186) -26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:25 INFO Executor: Finished task 162.0 in stage 10.0 (TID 179). 6563 bytes result sent to driver -26/04/01 08:42:25 INFO TaskSetManager: Starting task 170.0 in stage 10.0 (TID 187) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:25 INFO Executor: Running task 170.0 in stage 10.0 (TID 187) -26/04/01 08:42:25 INFO TaskSetManager: Finished task 162.0 in stage 10.0 (TID 179) in 8004 ms on 10.0.0.133 (executor driver) (163/208) -26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:25 INFO Executor: Finished task 164.0 in stage 10.0 (TID 181). 6563 bytes result sent to driver -26/04/01 08:42:25 INFO TaskSetManager: Starting task 171.0 in stage 10.0 (TID 188) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:25 INFO Executor: Finished task 163.0 in stage 10.0 (TID 180). 6563 bytes result sent to driver -26/04/01 08:42:25 INFO Executor: Running task 171.0 in stage 10.0 (TID 188) -26/04/01 08:42:25 INFO TaskSetManager: Finished task 164.0 in stage 10.0 (TID 181) in 8062 ms on 10.0.0.133 (executor driver) (164/208) -26/04/01 08:42:25 INFO TaskSetManager: Starting task 172.0 in stage 10.0 (TID 189) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:25 INFO TaskSetManager: Finished task 163.0 in stage 10.0 (TID 180) in 8082 ms on 10.0.0.133 (executor driver) (165/208) -26/04/01 08:42:25 INFO Executor: Running task 172.0 in stage 10.0 (TID 189) -26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:26 INFO Executor: Finished task 165.0 in stage 10.0 (TID 182). 6563 bytes result sent to driver -26/04/01 08:42:26 INFO TaskSetManager: Starting task 173.0 in stage 10.0 (TID 190) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:26 INFO Executor: Running task 173.0 in stage 10.0 (TID 190) -26/04/01 08:42:26 INFO TaskSetManager: Finished task 165.0 in stage 10.0 (TID 182) in 8053 ms on 10.0.0.133 (executor driver) (166/208) -26/04/01 08:42:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:26 INFO Executor: Finished task 166.0 in stage 10.0 (TID 183). 6563 bytes result sent to driver -26/04/01 08:42:26 INFO TaskSetManager: Starting task 174.0 in stage 10.0 (TID 191) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:42:26 INFO TaskSetManager: Finished task 166.0 in stage 10.0 (TID 183) in 8066 ms on 10.0.0.133 (executor driver) (167/208) -26/04/01 08:42:26 INFO Executor: Running task 174.0 in stage 10.0 (TID 191) -26/04/01 08:42:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:26 INFO Executor: Finished task 167.0 in stage 10.0 (TID 184). 6563 bytes result sent to driver -26/04/01 08:42:26 INFO TaskSetManager: Starting task 175.0 in stage 10.0 (TID 192) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:42:26 INFO Executor: Running task 175.0 in stage 10.0 (TID 192) -26/04/01 08:42:26 INFO TaskSetManager: Finished task 167.0 in stage 10.0 (TID 184) in 8074 ms on 10.0.0.133 (executor driver) (168/208) -26/04/01 08:42:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:33 INFO Executor: Finished task 168.0 in stage 10.0 (TID 185). 6563 bytes result sent to driver -26/04/01 08:42:33 INFO TaskSetManager: Starting task 176.0 in stage 10.0 (TID 193) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:33 INFO TaskSetManager: Finished task 168.0 in stage 10.0 (TID 185) in 8042 ms on 10.0.0.133 (executor driver) (169/208) -26/04/01 08:42:33 INFO Executor: Running task 176.0 in stage 10.0 (TID 193) -26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:33 INFO Executor: Finished task 169.0 in stage 10.0 (TID 186). 6563 bytes result sent to driver -26/04/01 08:42:33 INFO TaskSetManager: Starting task 177.0 in stage 10.0 (TID 194) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:33 INFO TaskSetManager: Finished task 169.0 in stage 10.0 (TID 186) in 8050 ms on 10.0.0.133 (executor driver) (170/208) -26/04/01 08:42:33 INFO Executor: Running task 177.0 in stage 10.0 (TID 194) -26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:33 INFO Executor: Finished task 170.0 in stage 10.0 (TID 187). 6563 bytes result sent to driver -26/04/01 08:42:33 INFO TaskSetManager: Starting task 178.0 in stage 10.0 (TID 195) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:33 INFO TaskSetManager: Finished task 170.0 in stage 10.0 (TID 187) in 8022 ms on 10.0.0.133 (executor driver) (171/208) -26/04/01 08:42:33 INFO Executor: Running task 178.0 in stage 10.0 (TID 195) -26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:33 INFO Executor: Finished task 172.0 in stage 10.0 (TID 189). 6563 bytes result sent to driver -26/04/01 08:42:33 INFO TaskSetManager: Starting task 179.0 in stage 10.0 (TID 196) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:33 INFO TaskSetManager: Finished task 172.0 in stage 10.0 (TID 189) in 8050 ms on 10.0.0.133 (executor driver) (172/208) -26/04/01 08:42:33 INFO Executor: Running task 179.0 in stage 10.0 (TID 196) -26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:33 INFO Executor: Finished task 171.0 in stage 10.0 (TID 188). 6563 bytes result sent to driver -26/04/01 08:42:33 INFO TaskSetManager: Starting task 180.0 in stage 10.0 (TID 197) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:42:33 INFO TaskSetManager: Finished task 171.0 in stage 10.0 (TID 188) in 8109 ms on 10.0.0.133 (executor driver) (173/208) -26/04/01 08:42:33 INFO Executor: Running task 180.0 in stage 10.0 (TID 197) -26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:34 INFO Executor: Finished task 173.0 in stage 10.0 (TID 190). 6563 bytes result sent to driver -26/04/01 08:42:34 INFO TaskSetManager: Starting task 181.0 in stage 10.0 (TID 198) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:34 INFO TaskSetManager: Finished task 173.0 in stage 10.0 (TID 190) in 8109 ms on 10.0.0.133 (executor driver) (174/208) -26/04/01 08:42:34 INFO Executor: Running task 181.0 in stage 10.0 (TID 198) -26/04/01 08:42:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:34 INFO Executor: Finished task 174.0 in stage 10.0 (TID 191). 6563 bytes result sent to driver -26/04/01 08:42:34 INFO TaskSetManager: Starting task 182.0 in stage 10.0 (TID 199) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:34 INFO TaskSetManager: Finished task 174.0 in stage 10.0 (TID 191) in 8140 ms on 10.0.0.133 (executor driver) (175/208) -26/04/01 08:42:34 INFO Executor: Running task 182.0 in stage 10.0 (TID 199) -26/04/01 08:42:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:34 INFO Executor: Finished task 175.0 in stage 10.0 (TID 192). 6563 bytes result sent to driver -26/04/01 08:42:34 INFO TaskSetManager: Starting task 183.0 in stage 10.0 (TID 200) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:34 INFO Executor: Running task 183.0 in stage 10.0 (TID 200) -26/04/01 08:42:34 INFO TaskSetManager: Finished task 175.0 in stage 10.0 (TID 192) in 8171 ms on 10.0.0.133 (executor driver) (176/208) -26/04/01 08:42:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:41 INFO Executor: Finished task 176.0 in stage 10.0 (TID 193). 6563 bytes result sent to driver -26/04/01 08:42:41 INFO TaskSetManager: Starting task 184.0 in stage 10.0 (TID 201) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:41 INFO TaskSetManager: Finished task 176.0 in stage 10.0 (TID 193) in 7973 ms on 10.0.0.133 (executor driver) (177/208) -26/04/01 08:42:41 INFO Executor: Running task 184.0 in stage 10.0 (TID 201) -26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:41 INFO Executor: Finished task 177.0 in stage 10.0 (TID 194). 6563 bytes result sent to driver -26/04/01 08:42:41 INFO TaskSetManager: Starting task 185.0 in stage 10.0 (TID 202) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:41 INFO Executor: Running task 185.0 in stage 10.0 (TID 202) -26/04/01 08:42:41 INFO TaskSetManager: Finished task 177.0 in stage 10.0 (TID 194) in 7972 ms on 10.0.0.133 (executor driver) (178/208) -26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:41 INFO Executor: Finished task 178.0 in stage 10.0 (TID 195). 6563 bytes result sent to driver -26/04/01 08:42:41 INFO TaskSetManager: Starting task 186.0 in stage 10.0 (TID 203) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:42:41 INFO TaskSetManager: Finished task 178.0 in stage 10.0 (TID 195) in 7978 ms on 10.0.0.133 (executor driver) (179/208) -26/04/01 08:42:41 INFO Executor: Running task 186.0 in stage 10.0 (TID 203) -26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:41 INFO Executor: Finished task 179.0 in stage 10.0 (TID 196). 6563 bytes result sent to driver -26/04/01 08:42:41 INFO TaskSetManager: Starting task 187.0 in stage 10.0 (TID 204) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:42:41 INFO TaskSetManager: Finished task 179.0 in stage 10.0 (TID 196) in 7969 ms on 10.0.0.133 (executor driver) (180/208) -26/04/01 08:42:41 INFO Executor: Running task 187.0 in stage 10.0 (TID 204) -26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:41 INFO Executor: Finished task 180.0 in stage 10.0 (TID 197). 6563 bytes result sent to driver -26/04/01 08:42:41 INFO TaskSetManager: Starting task 188.0 in stage 10.0 (TID 205) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:41 INFO Executor: Running task 188.0 in stage 10.0 (TID 205) -26/04/01 08:42:41 INFO TaskSetManager: Finished task 180.0 in stage 10.0 (TID 197) in 7961 ms on 10.0.0.133 (executor driver) (181/208) -26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:42 INFO Executor: Finished task 181.0 in stage 10.0 (TID 198). 6563 bytes result sent to driver -26/04/01 08:42:42 INFO TaskSetManager: Starting task 189.0 in stage 10.0 (TID 206) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:42 INFO TaskSetManager: Finished task 181.0 in stage 10.0 (TID 198) in 7980 ms on 10.0.0.133 (executor driver) (182/208) -26/04/01 08:42:42 INFO Executor: Running task 189.0 in stage 10.0 (TID 206) -26/04/01 08:42:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:42 INFO Executor: Finished task 182.0 in stage 10.0 (TID 199). 6563 bytes result sent to driver -26/04/01 08:42:42 INFO TaskSetManager: Starting task 190.0 in stage 10.0 (TID 207) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:42 INFO Executor: Running task 190.0 in stage 10.0 (TID 207) -26/04/01 08:42:42 INFO TaskSetManager: Finished task 182.0 in stage 10.0 (TID 199) in 7992 ms on 10.0.0.133 (executor driver) (183/208) -26/04/01 08:42:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:42 INFO Executor: Finished task 183.0 in stage 10.0 (TID 200). 6563 bytes result sent to driver -26/04/01 08:42:42 INFO TaskSetManager: Starting task 191.0 in stage 10.0 (TID 208) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:42:42 INFO TaskSetManager: Finished task 183.0 in stage 10.0 (TID 200) in 7964 ms on 10.0.0.133 (executor driver) (184/208) -26/04/01 08:42:42 INFO Executor: Running task 191.0 in stage 10.0 (TID 208) -26/04/01 08:42:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:49 INFO Executor: Finished task 184.0 in stage 10.0 (TID 201). 6606 bytes result sent to driver -26/04/01 08:42:49 INFO TaskSetManager: Starting task 192.0 in stage 10.0 (TID 209) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:42:49 INFO TaskSetManager: Finished task 184.0 in stage 10.0 (TID 201) in 7976 ms on 10.0.0.133 (executor driver) (185/208) -26/04/01 08:42:49 INFO Executor: Running task 192.0 in stage 10.0 (TID 209) -26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:49 INFO Executor: Finished task 185.0 in stage 10.0 (TID 202). 6606 bytes result sent to driver -26/04/01 08:42:49 INFO TaskSetManager: Starting task 193.0 in stage 10.0 (TID 210) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9346 bytes) -26/04/01 08:42:49 INFO TaskSetManager: Finished task 185.0 in stage 10.0 (TID 202) in 7974 ms on 10.0.0.133 (executor driver) (186/208) -26/04/01 08:42:49 INFO Executor: Running task 193.0 in stage 10.0 (TID 210) -26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:49 INFO Executor: Finished task 186.0 in stage 10.0 (TID 203). 6606 bytes result sent to driver -26/04/01 08:42:49 INFO TaskSetManager: Starting task 194.0 in stage 10.0 (TID 211) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:42:49 INFO Executor: Running task 194.0 in stage 10.0 (TID 211) -26/04/01 08:42:49 INFO TaskSetManager: Finished task 186.0 in stage 10.0 (TID 203) in 7969 ms on 10.0.0.133 (executor driver) (187/208) -26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:49 INFO Executor: Finished task 187.0 in stage 10.0 (TID 204). 6606 bytes result sent to driver -26/04/01 08:42:49 INFO TaskSetManager: Starting task 195.0 in stage 10.0 (TID 212) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:42:49 INFO TaskSetManager: Finished task 187.0 in stage 10.0 (TID 204) in 7994 ms on 10.0.0.133 (executor driver) (188/208) -26/04/01 08:42:49 INFO Executor: Running task 195.0 in stage 10.0 (TID 212) -26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:49 INFO Executor: Finished task 188.0 in stage 10.0 (TID 205). 6606 bytes result sent to driver -26/04/01 08:42:49 INFO TaskSetManager: Starting task 196.0 in stage 10.0 (TID 213) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9346 bytes) -26/04/01 08:42:49 INFO TaskSetManager: Finished task 188.0 in stage 10.0 (TID 205) in 7969 ms on 10.0.0.133 (executor driver) (189/208) -26/04/01 08:42:49 INFO Executor: Running task 196.0 in stage 10.0 (TID 213) -26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:50 INFO Executor: Finished task 189.0 in stage 10.0 (TID 206). 6606 bytes result sent to driver -26/04/01 08:42:50 INFO TaskSetManager: Starting task 197.0 in stage 10.0 (TID 214) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9346 bytes) -26/04/01 08:42:50 INFO TaskSetManager: Finished task 189.0 in stage 10.0 (TID 206) in 7975 ms on 10.0.0.133 (executor driver) (190/208) -26/04/01 08:42:50 INFO Executor: Running task 197.0 in stage 10.0 (TID 214) -26/04/01 08:42:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:50 INFO Executor: Finished task 190.0 in stage 10.0 (TID 207). 6606 bytes result sent to driver -26/04/01 08:42:50 INFO TaskSetManager: Starting task 198.0 in stage 10.0 (TID 215) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:42:50 INFO Executor: Running task 198.0 in stage 10.0 (TID 215) -26/04/01 08:42:50 INFO TaskSetManager: Finished task 190.0 in stage 10.0 (TID 207) in 7999 ms on 10.0.0.133 (executor driver) (191/208) -26/04/01 08:42:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:50 INFO Executor: Finished task 191.0 in stage 10.0 (TID 208). 6606 bytes result sent to driver -26/04/01 08:42:50 INFO TaskSetManager: Starting task 199.0 in stage 10.0 (TID 216) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:42:50 INFO TaskSetManager: Finished task 191.0 in stage 10.0 (TID 208) in 7965 ms on 10.0.0.133 (executor driver) (192/208) -26/04/01 08:42:50 INFO Executor: Running task 199.0 in stage 10.0 (TID 216) -26/04/01 08:42:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:54 INFO Executor: Finished task 192.0 in stage 10.0 (TID 209). 6563 bytes result sent to driver -26/04/01 08:42:54 INFO TaskSetManager: Starting task 200.0 in stage 10.0 (TID 217) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9345 bytes) -26/04/01 08:42:54 INFO TaskSetManager: Finished task 192.0 in stage 10.0 (TID 209) in 4979 ms on 10.0.0.133 (executor driver) (193/208) -26/04/01 08:42:54 INFO Executor: Running task 200.0 in stage 10.0 (TID 217) -26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:54 INFO Executor: Finished task 193.0 in stage 10.0 (TID 210). 6563 bytes result sent to driver -26/04/01 08:42:54 INFO TaskSetManager: Starting task 201.0 in stage 10.0 (TID 218) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9346 bytes) -26/04/01 08:42:54 INFO TaskSetManager: Finished task 193.0 in stage 10.0 (TID 210) in 4958 ms on 10.0.0.133 (executor driver) (194/208) -26/04/01 08:42:54 INFO Executor: Running task 201.0 in stage 10.0 (TID 218) -26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:54 INFO Executor: Finished task 194.0 in stage 10.0 (TID 211). 6563 bytes result sent to driver -26/04/01 08:42:54 INFO TaskSetManager: Starting task 202.0 in stage 10.0 (TID 219) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9346 bytes) -26/04/01 08:42:54 INFO TaskSetManager: Finished task 194.0 in stage 10.0 (TID 211) in 4952 ms on 10.0.0.133 (executor driver) (195/208) -26/04/01 08:42:54 INFO Executor: Running task 202.0 in stage 10.0 (TID 219) -26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:54 INFO Executor: Finished task 195.0 in stage 10.0 (TID 212). 6563 bytes result sent to driver -26/04/01 08:42:54 INFO TaskSetManager: Starting task 203.0 in stage 10.0 (TID 220) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:42:54 INFO TaskSetManager: Finished task 195.0 in stage 10.0 (TID 212) in 4956 ms on 10.0.0.133 (executor driver) (196/208) -26/04/01 08:42:54 INFO Executor: Running task 203.0 in stage 10.0 (TID 220) -26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:54 INFO Executor: Finished task 196.0 in stage 10.0 (TID 213). 6563 bytes result sent to driver -26/04/01 08:42:54 INFO TaskSetManager: Starting task 204.0 in stage 10.0 (TID 221) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:42:54 INFO TaskSetManager: Finished task 196.0 in stage 10.0 (TID 213) in 4950 ms on 10.0.0.133 (executor driver) (197/208) -26/04/01 08:42:54 INFO Executor: Running task 204.0 in stage 10.0 (TID 221) -26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:55 INFO Executor: Finished task 197.0 in stage 10.0 (TID 214). 6563 bytes result sent to driver -26/04/01 08:42:55 INFO TaskSetManager: Starting task 205.0 in stage 10.0 (TID 222) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:42:55 INFO TaskSetManager: Finished task 197.0 in stage 10.0 (TID 214) in 4923 ms on 10.0.0.133 (executor driver) (198/208) -26/04/01 08:42:55 INFO Executor: Running task 205.0 in stage 10.0 (TID 222) -26/04/01 08:42:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:55 INFO Executor: Finished task 198.0 in stage 10.0 (TID 215). 6563 bytes result sent to driver -26/04/01 08:42:55 INFO TaskSetManager: Starting task 206.0 in stage 10.0 (TID 223) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9346 bytes) -26/04/01 08:42:55 INFO TaskSetManager: Finished task 198.0 in stage 10.0 (TID 215) in 4924 ms on 10.0.0.133 (executor driver) (199/208) -26/04/01 08:42:55 INFO Executor: Running task 206.0 in stage 10.0 (TID 223) -26/04/01 08:42:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:55 INFO Executor: Finished task 199.0 in stage 10.0 (TID 216). 6563 bytes result sent to driver -26/04/01 08:42:55 INFO TaskSetManager: Starting task 207.0 in stage 10.0 (TID 224) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9346 bytes) -26/04/01 08:42:55 INFO Executor: Running task 207.0 in stage 10.0 (TID 224) -26/04/01 08:42:55 INFO TaskSetManager: Finished task 199.0 in stage 10.0 (TID 216) in 4924 ms on 10.0.0.133 (executor driver) (200/208) -26/04/01 08:42:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO Executor: Finished task 200.0 in stage 10.0 (TID 217). 6563 bytes result sent to driver -26/04/01 08:42:59 INFO TaskSetManager: Starting task 0.0 in stage 11.0 (TID 225) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9463 bytes) -26/04/01 08:42:59 INFO TaskSetManager: Finished task 200.0 in stage 10.0 (TID 217) in 4938 ms on 10.0.0.133 (executor driver) (201/208) -26/04/01 08:42:59 INFO Executor: Running task 0.0 in stage 11.0 (TID 225) -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO Executor: Finished task 201.0 in stage 10.0 (TID 218). 6563 bytes result sent to driver -26/04/01 08:42:59 INFO TaskSetManager: Starting task 1.0 in stage 11.0 (TID 226) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9464 bytes) -26/04/01 08:42:59 INFO TaskSetManager: Finished task 201.0 in stage 10.0 (TID 218) in 4941 ms on 10.0.0.133 (executor driver) (202/208) -26/04/01 08:42:59 INFO Executor: Running task 1.0 in stage 11.0 (TID 226) -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO Executor: Finished task 0.0 in stage 11.0 (TID 225). 6563 bytes result sent to driver -26/04/01 08:42:59 INFO TaskSetManager: Starting task 2.0 in stage 11.0 (TID 227) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9464 bytes) -26/04/01 08:42:59 INFO TaskSetManager: Finished task 0.0 in stage 11.0 (TID 225) in 106 ms on 10.0.0.133 (executor driver) (1/8) -26/04/01 08:42:59 INFO Executor: Running task 2.0 in stage 11.0 (TID 227) -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO Executor: Finished task 202.0 in stage 10.0 (TID 219). 6563 bytes result sent to driver -26/04/01 08:42:59 INFO TaskSetManager: Starting task 3.0 in stage 11.0 (TID 228) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9465 bytes) -26/04/01 08:42:59 INFO TaskSetManager: Finished task 202.0 in stage 10.0 (TID 219) in 4932 ms on 10.0.0.133 (executor driver) (203/208) -26/04/01 08:42:59 INFO Executor: Running task 3.0 in stage 11.0 (TID 228) -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO Executor: Finished task 1.0 in stage 11.0 (TID 226). 6563 bytes result sent to driver -26/04/01 08:42:59 INFO TaskSetManager: Starting task 4.0 in stage 11.0 (TID 229) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9465 bytes) -26/04/01 08:42:59 INFO TaskSetManager: Finished task 1.0 in stage 11.0 (TID 226) in 109 ms on 10.0.0.133 (executor driver) (2/8) -26/04/01 08:42:59 INFO Executor: Running task 4.0 in stage 11.0 (TID 229) -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO Executor: Finished task 2.0 in stage 11.0 (TID 227). 6563 bytes result sent to driver -26/04/01 08:42:59 INFO TaskSetManager: Starting task 5.0 in stage 11.0 (TID 230) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9464 bytes) -26/04/01 08:42:59 INFO Executor: Running task 5.0 in stage 11.0 (TID 230) -26/04/01 08:42:59 INFO TaskSetManager: Finished task 2.0 in stage 11.0 (TID 227) in 107 ms on 10.0.0.133 (executor driver) (3/8) -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO Executor: Finished task 3.0 in stage 11.0 (TID 228). 6563 bytes result sent to driver -26/04/01 08:42:59 INFO TaskSetManager: Starting task 6.0 in stage 11.0 (TID 231) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9463 bytes) -26/04/01 08:42:59 INFO TaskSetManager: Finished task 3.0 in stage 11.0 (TID 228) in 109 ms on 10.0.0.133 (executor driver) (4/8) -26/04/01 08:42:59 INFO Executor: Running task 6.0 in stage 11.0 (TID 231) -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO Executor: Finished task 203.0 in stage 10.0 (TID 220). 6563 bytes result sent to driver -26/04/01 08:42:59 INFO TaskSetManager: Starting task 7.0 in stage 11.0 (TID 232) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9463 bytes) -26/04/01 08:42:59 INFO TaskSetManager: Finished task 203.0 in stage 10.0 (TID 220) in 4931 ms on 10.0.0.133 (executor driver) (204/208) -26/04/01 08:42:59 INFO Executor: Running task 7.0 in stage 11.0 (TID 232) -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO Executor: Finished task 204.0 in stage 10.0 (TID 221). 6563 bytes result sent to driver -26/04/01 08:42:59 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 233) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:42:59 INFO Executor: Running task 0.0 in stage 12.0 (TID 233) -26/04/01 08:42:59 INFO TaskSetManager: Finished task 204.0 in stage 10.0 (TID 221) in 4933 ms on 10.0.0.133 (executor driver) (205/208) -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO Executor: Finished task 4.0 in stage 11.0 (TID 229). 6563 bytes result sent to driver -26/04/01 08:42:59 INFO TaskSetManager: Starting task 1.0 in stage 12.0 (TID 234) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:42:59 INFO Executor: Running task 1.0 in stage 12.0 (TID 234) -26/04/01 08:42:59 INFO TaskSetManager: Finished task 4.0 in stage 11.0 (TID 229) in 111 ms on 10.0.0.133 (executor driver) (5/8) -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO Executor: Finished task 5.0 in stage 11.0 (TID 230). 6563 bytes result sent to driver -26/04/01 08:42:59 INFO TaskSetManager: Starting task 2.0 in stage 12.0 (TID 235) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:42:59 INFO TaskSetManager: Finished task 5.0 in stage 11.0 (TID 230) in 111 ms on 10.0.0.133 (executor driver) (6/8) -26/04/01 08:42:59 INFO Executor: Running task 2.0 in stage 12.0 (TID 235) -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO Executor: Finished task 6.0 in stage 11.0 (TID 231). 6563 bytes result sent to driver -26/04/01 08:42:59 INFO TaskSetManager: Starting task 3.0 in stage 12.0 (TID 236) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:42:59 INFO TaskSetManager: Finished task 6.0 in stage 11.0 (TID 231) in 111 ms on 10.0.0.133 (executor driver) (7/8) -26/04/01 08:42:59 INFO Executor: Running task 3.0 in stage 12.0 (TID 236) -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO Executor: Finished task 7.0 in stage 11.0 (TID 232). 6563 bytes result sent to driver -26/04/01 08:42:59 INFO TaskSetManager: Starting task 4.0 in stage 12.0 (TID 237) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:42:59 INFO TaskSetManager: Finished task 7.0 in stage 11.0 (TID 232) in 114 ms on 10.0.0.133 (executor driver) (8/8) -26/04/01 08:42:59 INFO Executor: Running task 4.0 in stage 12.0 (TID 237) -26/04/01 08:42:59 INFO TaskSchedulerImpl: Removed TaskSet 11.0, whose tasks have all completed, from pool -26/04/01 08:42:59 INFO DAGScheduler: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 203.765 s -26/04/01 08:42:59 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:42:59 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 12, ShuffleMapStage 13, ShuffleMapStage 10) -26/04/01 08:42:59 INFO DAGScheduler: waiting: Set() -26/04/01 08:42:59 INFO DAGScheduler: failed: Set() -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:42:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:00 INFO Executor: Finished task 205.0 in stage 10.0 (TID 222). 6563 bytes result sent to driver -26/04/01 08:43:00 INFO TaskSetManager: Starting task 5.0 in stage 12.0 (TID 238) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:00 INFO TaskSetManager: Finished task 205.0 in stage 10.0 (TID 222) in 4909 ms on 10.0.0.133 (executor driver) (206/208) -26/04/01 08:43:00 INFO Executor: Running task 5.0 in stage 12.0 (TID 238) -26/04/01 08:43:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:00 INFO Executor: Finished task 206.0 in stage 10.0 (TID 223). 6563 bytes result sent to driver -26/04/01 08:43:00 INFO TaskSetManager: Starting task 6.0 in stage 12.0 (TID 239) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:00 INFO TaskSetManager: Finished task 206.0 in stage 10.0 (TID 223) in 4920 ms on 10.0.0.133 (executor driver) (207/208) -26/04/01 08:43:00 INFO Executor: Running task 6.0 in stage 12.0 (TID 239) -26/04/01 08:43:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:00 INFO Executor: Finished task 207.0 in stage 10.0 (TID 224). 6563 bytes result sent to driver -26/04/01 08:43:00 INFO TaskSetManager: Starting task 7.0 in stage 12.0 (TID 240) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:00 INFO Executor: Running task 7.0 in stage 12.0 (TID 240) -26/04/01 08:43:00 INFO TaskSetManager: Finished task 207.0 in stage 10.0 (TID 224) in 4944 ms on 10.0.0.133 (executor driver) (208/208) -26/04/01 08:43:00 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool -26/04/01 08:43:00 INFO DAGScheduler: ShuffleMapStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 204.203 s -26/04/01 08:43:00 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:43:00 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 12, ShuffleMapStage 13) -26/04/01 08:43:00 INFO DAGScheduler: waiting: Set() -26/04/01 08:43:00 INFO DAGScheduler: failed: Set() -26/04/01 08:43:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO Executor: Finished task 0.0 in stage 12.0 (TID 233). 6563 bytes result sent to driver -26/04/01 08:43:03 INFO TaskSetManager: Starting task 8.0 in stage 12.0 (TID 241) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:03 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 233) in 3395 ms on 10.0.0.133 (executor driver) (1/37) -26/04/01 08:43:03 INFO Executor: Running task 8.0 in stage 12.0 (TID 241) -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO Executor: Finished task 1.0 in stage 12.0 (TID 234). 6563 bytes result sent to driver -26/04/01 08:43:03 INFO TaskSetManager: Starting task 9.0 in stage 12.0 (TID 242) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:03 INFO Executor: Running task 9.0 in stage 12.0 (TID 242) -26/04/01 08:43:03 INFO TaskSetManager: Finished task 1.0 in stage 12.0 (TID 234) in 3379 ms on 10.0.0.133 (executor driver) (2/37) -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO Executor: Finished task 2.0 in stage 12.0 (TID 235). 6563 bytes result sent to driver -26/04/01 08:43:03 INFO TaskSetManager: Starting task 10.0 in stage 12.0 (TID 243) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:03 INFO TaskSetManager: Finished task 2.0 in stage 12.0 (TID 235) in 3393 ms on 10.0.0.133 (executor driver) (3/37) -26/04/01 08:43:03 INFO Executor: Running task 10.0 in stage 12.0 (TID 243) -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO Executor: Finished task 3.0 in stage 12.0 (TID 236). 6563 bytes result sent to driver -26/04/01 08:43:03 INFO TaskSetManager: Starting task 11.0 in stage 12.0 (TID 244) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:03 INFO TaskSetManager: Finished task 3.0 in stage 12.0 (TID 236) in 3388 ms on 10.0.0.133 (executor driver) (4/37) -26/04/01 08:43:03 INFO Executor: Running task 11.0 in stage 12.0 (TID 244) -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO Executor: Finished task 4.0 in stage 12.0 (TID 237). 6563 bytes result sent to driver -26/04/01 08:43:03 INFO TaskSetManager: Starting task 12.0 in stage 12.0 (TID 245) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:03 INFO TaskSetManager: Finished task 4.0 in stage 12.0 (TID 237) in 3388 ms on 10.0.0.133 (executor driver) (5/37) -26/04/01 08:43:03 INFO Executor: Running task 12.0 in stage 12.0 (TID 245) -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO Executor: Finished task 5.0 in stage 12.0 (TID 238). 6563 bytes result sent to driver -26/04/01 08:43:03 INFO TaskSetManager: Starting task 13.0 in stage 12.0 (TID 246) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:03 INFO TaskSetManager: Finished task 5.0 in stage 12.0 (TID 238) in 3380 ms on 10.0.0.133 (executor driver) (6/37) -26/04/01 08:43:03 INFO Executor: Running task 13.0 in stage 12.0 (TID 246) -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO Executor: Finished task 6.0 in stage 12.0 (TID 239). 6563 bytes result sent to driver -26/04/01 08:43:03 INFO TaskSetManager: Starting task 14.0 in stage 12.0 (TID 247) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:03 INFO TaskSetManager: Finished task 6.0 in stage 12.0 (TID 239) in 3364 ms on 10.0.0.133 (executor driver) (7/37) -26/04/01 08:43:03 INFO Executor: Running task 14.0 in stage 12.0 (TID 247) -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO Executor: Finished task 7.0 in stage 12.0 (TID 240). 6606 bytes result sent to driver -26/04/01 08:43:03 INFO TaskSetManager: Starting task 15.0 in stage 12.0 (TID 248) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:03 INFO TaskSetManager: Finished task 7.0 in stage 12.0 (TID 240) in 3386 ms on 10.0.0.133 (executor driver) (8/37) -26/04/01 08:43:03 INFO Executor: Running task 15.0 in stage 12.0 (TID 248) -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:06 INFO Executor: Finished task 8.0 in stage 12.0 (TID 241). 6563 bytes result sent to driver -26/04/01 08:43:06 INFO TaskSetManager: Starting task 16.0 in stage 12.0 (TID 249) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:06 INFO TaskSetManager: Finished task 8.0 in stage 12.0 (TID 241) in 3383 ms on 10.0.0.133 (executor driver) (9/37) -26/04/01 08:43:06 INFO Executor: Running task 16.0 in stage 12.0 (TID 249) -26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:06 INFO Executor: Finished task 9.0 in stage 12.0 (TID 242). 6563 bytes result sent to driver -26/04/01 08:43:06 INFO TaskSetManager: Starting task 17.0 in stage 12.0 (TID 250) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:06 INFO TaskSetManager: Finished task 9.0 in stage 12.0 (TID 242) in 3393 ms on 10.0.0.133 (executor driver) (10/37) -26/04/01 08:43:06 INFO Executor: Running task 17.0 in stage 12.0 (TID 250) -26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:06 INFO Executor: Finished task 10.0 in stage 12.0 (TID 243). 6563 bytes result sent to driver -26/04/01 08:43:06 INFO TaskSetManager: Starting task 18.0 in stage 12.0 (TID 251) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:06 INFO TaskSetManager: Finished task 10.0 in stage 12.0 (TID 243) in 3410 ms on 10.0.0.133 (executor driver) (11/37) -26/04/01 08:43:06 INFO Executor: Running task 18.0 in stage 12.0 (TID 251) -26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:06 INFO Executor: Finished task 11.0 in stage 12.0 (TID 244). 6563 bytes result sent to driver -26/04/01 08:43:06 INFO TaskSetManager: Starting task 19.0 in stage 12.0 (TID 252) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:06 INFO TaskSetManager: Finished task 11.0 in stage 12.0 (TID 244) in 3418 ms on 10.0.0.133 (executor driver) (12/37) -26/04/01 08:43:06 INFO Executor: Running task 19.0 in stage 12.0 (TID 252) -26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:06 INFO Executor: Finished task 12.0 in stage 12.0 (TID 245). 6563 bytes result sent to driver -26/04/01 08:43:06 INFO TaskSetManager: Starting task 20.0 in stage 12.0 (TID 253) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:06 INFO TaskSetManager: Finished task 12.0 in stage 12.0 (TID 245) in 3442 ms on 10.0.0.133 (executor driver) (13/37) -26/04/01 08:43:06 INFO Executor: Running task 20.0 in stage 12.0 (TID 253) -26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:06 INFO Executor: Finished task 13.0 in stage 12.0 (TID 246). 6563 bytes result sent to driver -26/04/01 08:43:06 INFO TaskSetManager: Starting task 21.0 in stage 12.0 (TID 254) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:06 INFO TaskSetManager: Finished task 13.0 in stage 12.0 (TID 246) in 3420 ms on 10.0.0.133 (executor driver) (14/37) -26/04/01 08:43:06 INFO Executor: Running task 21.0 in stage 12.0 (TID 254) -26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:06 INFO Executor: Finished task 14.0 in stage 12.0 (TID 247). 6563 bytes result sent to driver -26/04/01 08:43:06 INFO TaskSetManager: Starting task 22.0 in stage 12.0 (TID 255) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:06 INFO TaskSetManager: Finished task 14.0 in stage 12.0 (TID 247) in 3407 ms on 10.0.0.133 (executor driver) (15/37) -26/04/01 08:43:06 INFO Executor: Running task 22.0 in stage 12.0 (TID 255) -26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:07 INFO Executor: Finished task 15.0 in stage 12.0 (TID 248). 6563 bytes result sent to driver -26/04/01 08:43:07 INFO TaskSetManager: Starting task 23.0 in stage 12.0 (TID 256) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:07 INFO TaskSetManager: Finished task 15.0 in stage 12.0 (TID 248) in 3408 ms on 10.0.0.133 (executor driver) (16/37) -26/04/01 08:43:07 INFO Executor: Running task 23.0 in stage 12.0 (TID 256) -26/04/01 08:43:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:09 INFO Executor: Finished task 16.0 in stage 12.0 (TID 249). 6563 bytes result sent to driver -26/04/01 08:43:09 INFO TaskSetManager: Starting task 24.0 in stage 12.0 (TID 257) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:09 INFO TaskSetManager: Finished task 16.0 in stage 12.0 (TID 249) in 3324 ms on 10.0.0.133 (executor driver) (17/37) -26/04/01 08:43:09 INFO Executor: Running task 24.0 in stage 12.0 (TID 257) -26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:09 INFO Executor: Finished task 17.0 in stage 12.0 (TID 250). 6563 bytes result sent to driver -26/04/01 08:43:09 INFO TaskSetManager: Starting task 25.0 in stage 12.0 (TID 258) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:09 INFO TaskSetManager: Finished task 17.0 in stage 12.0 (TID 250) in 3314 ms on 10.0.0.133 (executor driver) (18/37) -26/04/01 08:43:09 INFO Executor: Running task 25.0 in stage 12.0 (TID 258) -26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:09 INFO Executor: Finished task 18.0 in stage 12.0 (TID 251). 6563 bytes result sent to driver -26/04/01 08:43:09 INFO TaskSetManager: Starting task 26.0 in stage 12.0 (TID 259) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:09 INFO TaskSetManager: Finished task 18.0 in stage 12.0 (TID 251) in 3334 ms on 10.0.0.133 (executor driver) (19/37) -26/04/01 08:43:09 INFO Executor: Running task 26.0 in stage 12.0 (TID 259) -26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:09 INFO Executor: Finished task 19.0 in stage 12.0 (TID 252). 6563 bytes result sent to driver -26/04/01 08:43:09 INFO TaskSetManager: Starting task 27.0 in stage 12.0 (TID 260) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:09 INFO TaskSetManager: Finished task 19.0 in stage 12.0 (TID 252) in 3335 ms on 10.0.0.133 (executor driver) (20/37) -26/04/01 08:43:09 INFO Executor: Running task 27.0 in stage 12.0 (TID 260) -26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:10 INFO Executor: Finished task 20.0 in stage 12.0 (TID 253). 6563 bytes result sent to driver -26/04/01 08:43:10 INFO TaskSetManager: Starting task 28.0 in stage 12.0 (TID 261) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:10 INFO TaskSetManager: Finished task 20.0 in stage 12.0 (TID 253) in 3328 ms on 10.0.0.133 (executor driver) (21/37) -26/04/01 08:43:10 INFO Executor: Running task 28.0 in stage 12.0 (TID 261) -26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:10 INFO Executor: Finished task 21.0 in stage 12.0 (TID 254). 6563 bytes result sent to driver -26/04/01 08:43:10 INFO TaskSetManager: Starting task 29.0 in stage 12.0 (TID 262) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:10 INFO TaskSetManager: Finished task 21.0 in stage 12.0 (TID 254) in 3329 ms on 10.0.0.133 (executor driver) (22/37) -26/04/01 08:43:10 INFO Executor: Running task 29.0 in stage 12.0 (TID 262) -26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:10 INFO Executor: Finished task 22.0 in stage 12.0 (TID 255). 6563 bytes result sent to driver -26/04/01 08:43:10 INFO TaskSetManager: Starting task 30.0 in stage 12.0 (TID 263) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:10 INFO TaskSetManager: Finished task 22.0 in stage 12.0 (TID 255) in 3349 ms on 10.0.0.133 (executor driver) (23/37) -26/04/01 08:43:10 INFO Executor: Running task 30.0 in stage 12.0 (TID 263) -26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:10 INFO Executor: Finished task 23.0 in stage 12.0 (TID 256). 6563 bytes result sent to driver -26/04/01 08:43:10 INFO TaskSetManager: Starting task 31.0 in stage 12.0 (TID 264) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:10 INFO TaskSetManager: Finished task 23.0 in stage 12.0 (TID 256) in 3324 ms on 10.0.0.133 (executor driver) (24/37) -26/04/01 08:43:10 INFO Executor: Running task 31.0 in stage 12.0 (TID 264) -26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO Executor: Finished task 24.0 in stage 12.0 (TID 257). 6563 bytes result sent to driver -26/04/01 08:43:13 INFO TaskSetManager: Starting task 32.0 in stage 12.0 (TID 265) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9694 bytes) -26/04/01 08:43:13 INFO Executor: Running task 32.0 in stage 12.0 (TID 265) -26/04/01 08:43:13 INFO TaskSetManager: Finished task 24.0 in stage 12.0 (TID 257) in 3350 ms on 10.0.0.133 (executor driver) (25/37) -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO Executor: Finished task 25.0 in stage 12.0 (TID 258). 6563 bytes result sent to driver -26/04/01 08:43:13 INFO TaskSetManager: Starting task 33.0 in stage 12.0 (TID 266) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9695 bytes) -26/04/01 08:43:13 INFO TaskSetManager: Finished task 25.0 in stage 12.0 (TID 258) in 3344 ms on 10.0.0.133 (executor driver) (26/37) -26/04/01 08:43:13 INFO Executor: Running task 33.0 in stage 12.0 (TID 266) -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO Executor: Finished task 26.0 in stage 12.0 (TID 259). 6563 bytes result sent to driver -26/04/01 08:43:13 INFO TaskSetManager: Starting task 34.0 in stage 12.0 (TID 267) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9691 bytes) -26/04/01 08:43:13 INFO TaskSetManager: Finished task 26.0 in stage 12.0 (TID 259) in 3339 ms on 10.0.0.133 (executor driver) (27/37) -26/04/01 08:43:13 INFO Executor: Running task 34.0 in stage 12.0 (TID 267) -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO Executor: Finished task 27.0 in stage 12.0 (TID 260). 6563 bytes result sent to driver -26/04/01 08:43:13 INFO TaskSetManager: Starting task 35.0 in stage 12.0 (TID 268) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9694 bytes) -26/04/01 08:43:13 INFO TaskSetManager: Finished task 27.0 in stage 12.0 (TID 260) in 3352 ms on 10.0.0.133 (executor driver) (28/37) -26/04/01 08:43:13 INFO Executor: Running task 35.0 in stage 12.0 (TID 268) -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO Executor: Finished task 28.0 in stage 12.0 (TID 261). 6563 bytes result sent to driver -26/04/01 08:43:13 INFO TaskSetManager: Starting task 36.0 in stage 12.0 (TID 269) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9482 bytes) -26/04/01 08:43:13 INFO TaskSetManager: Finished task 28.0 in stage 12.0 (TID 261) in 3357 ms on 10.0.0.133 (executor driver) (29/37) -26/04/01 08:43:13 INFO Executor: Running task 36.0 in stage 12.0 (TID 269) -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO Executor: Finished task 29.0 in stage 12.0 (TID 262). 6563 bytes result sent to driver -26/04/01 08:43:13 INFO TaskSetManager: Starting task 0.0 in stage 13.0 (TID 270) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:13 INFO TaskSetManager: Finished task 29.0 in stage 12.0 (TID 262) in 3351 ms on 10.0.0.133 (executor driver) (30/37) -26/04/01 08:43:13 INFO Executor: Running task 0.0 in stage 13.0 (TID 270) -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO BlockManagerInfo: Removed broadcast_15_piece0 on 10.0.0.133:59303 in memory (size: 7.9 KiB, free: 8.6 GiB) -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO Executor: Finished task 30.0 in stage 12.0 (TID 263). 6606 bytes result sent to driver -26/04/01 08:43:13 INFO TaskSetManager: Starting task 1.0 in stage 13.0 (TID 271) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:43:13 INFO Executor: Running task 1.0 in stage 13.0 (TID 271) -26/04/01 08:43:13 INFO TaskSetManager: Finished task 30.0 in stage 12.0 (TID 263) in 3328 ms on 10.0.0.133 (executor driver) (31/37) -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO Executor: Finished task 31.0 in stage 12.0 (TID 264). 6606 bytes result sent to driver -26/04/01 08:43:13 INFO TaskSetManager: Starting task 2.0 in stage 13.0 (TID 272) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:13 INFO Executor: Running task 2.0 in stage 13.0 (TID 272) -26/04/01 08:43:13 INFO TaskSetManager: Finished task 31.0 in stage 12.0 (TID 264) in 3321 ms on 10.0.0.133 (executor driver) (32/37) -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:14 INFO Executor: Finished task 36.0 in stage 12.0 (TID 269). 6606 bytes result sent to driver -26/04/01 08:43:14 INFO TaskSetManager: Starting task 3.0 in stage 13.0 (TID 273) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:14 INFO Executor: Running task 3.0 in stage 13.0 (TID 273) -26/04/01 08:43:14 INFO TaskSetManager: Finished task 36.0 in stage 12.0 (TID 269) in 1146 ms on 10.0.0.133 (executor driver) (33/37) -26/04/01 08:43:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:15 INFO Executor: Finished task 33.0 in stage 12.0 (TID 266). 6606 bytes result sent to driver -26/04/01 08:43:15 INFO TaskSetManager: Starting task 4.0 in stage 13.0 (TID 274) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:15 INFO TaskSetManager: Finished task 33.0 in stage 12.0 (TID 266) in 2037 ms on 10.0.0.133 (executor driver) (34/37) -26/04/01 08:43:15 INFO Executor: Running task 4.0 in stage 13.0 (TID 274) -26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:15 INFO Executor: Finished task 32.0 in stage 12.0 (TID 265). 6606 bytes result sent to driver -26/04/01 08:43:15 INFO TaskSetManager: Starting task 5.0 in stage 13.0 (TID 275) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:43:15 INFO TaskSetManager: Finished task 32.0 in stage 12.0 (TID 265) in 2050 ms on 10.0.0.133 (executor driver) (35/37) -26/04/01 08:43:15 INFO Executor: Running task 5.0 in stage 13.0 (TID 275) -26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:15 INFO Executor: Finished task 34.0 in stage 12.0 (TID 267). 6606 bytes result sent to driver -26/04/01 08:43:15 INFO TaskSetManager: Starting task 6.0 in stage 13.0 (TID 276) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:15 INFO TaskSetManager: Finished task 34.0 in stage 12.0 (TID 267) in 2038 ms on 10.0.0.133 (executor driver) (36/37) -26/04/01 08:43:15 INFO Executor: Running task 6.0 in stage 13.0 (TID 276) -26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:15 INFO Executor: Finished task 35.0 in stage 12.0 (TID 268). 6606 bytes result sent to driver -26/04/01 08:43:15 INFO TaskSetManager: Starting task 7.0 in stage 13.0 (TID 277) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:15 INFO TaskSetManager: Finished task 35.0 in stage 12.0 (TID 268) in 2043 ms on 10.0.0.133 (executor driver) (37/37) -26/04/01 08:43:15 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool -26/04/01 08:43:15 INFO Executor: Running task 7.0 in stage 13.0 (TID 277) -26/04/01 08:43:15 INFO DAGScheduler: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 219.261 s -26/04/01 08:43:15 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:43:15 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 13) -26/04/01 08:43:15 INFO DAGScheduler: waiting: Set() -26/04/01 08:43:15 INFO DAGScheduler: failed: Set() -26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:15 INFO Executor: Finished task 0.0 in stage 13.0 (TID 270). 6649 bytes result sent to driver -26/04/01 08:43:15 INFO TaskSetManager: Starting task 8.0 in stage 13.0 (TID 278) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:43:15 INFO TaskSetManager: Finished task 0.0 in stage 13.0 (TID 270) in 2270 ms on 10.0.0.133 (executor driver) (1/64) -26/04/01 08:43:15 INFO Executor: Running task 8.0 in stage 13.0 (TID 278) -26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:15 INFO Executor: Finished task 1.0 in stage 13.0 (TID 271). 6563 bytes result sent to driver -26/04/01 08:43:15 INFO TaskSetManager: Starting task 9.0 in stage 13.0 (TID 279) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:15 INFO Executor: Running task 9.0 in stage 13.0 (TID 279) -26/04/01 08:43:15 INFO TaskSetManager: Finished task 1.0 in stage 13.0 (TID 271) in 2277 ms on 10.0.0.133 (executor driver) (2/64) -26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:15 INFO Executor: Finished task 2.0 in stage 13.0 (TID 272). 6563 bytes result sent to driver -26/04/01 08:43:15 INFO TaskSetManager: Starting task 10.0 in stage 13.0 (TID 280) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:43:15 INFO TaskSetManager: Finished task 2.0 in stage 13.0 (TID 272) in 2263 ms on 10.0.0.133 (executor driver) (3/64) -26/04/01 08:43:15 INFO Executor: Running task 10.0 in stage 13.0 (TID 280) -26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:16 INFO Executor: Finished task 3.0 in stage 13.0 (TID 273). 6563 bytes result sent to driver -26/04/01 08:43:16 INFO TaskSetManager: Starting task 11.0 in stage 13.0 (TID 281) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:16 INFO TaskSetManager: Finished task 3.0 in stage 13.0 (TID 273) in 2248 ms on 10.0.0.133 (executor driver) (4/64) -26/04/01 08:43:16 INFO Executor: Running task 11.0 in stage 13.0 (TID 281) -26/04/01 08:43:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:17 INFO Executor: Finished task 4.0 in stage 13.0 (TID 274). 6563 bytes result sent to driver -26/04/01 08:43:17 INFO TaskSetManager: Starting task 12.0 in stage 13.0 (TID 282) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:17 INFO TaskSetManager: Finished task 4.0 in stage 13.0 (TID 274) in 2280 ms on 10.0.0.133 (executor driver) (5/64) -26/04/01 08:43:17 INFO Executor: Running task 12.0 in stage 13.0 (TID 282) -26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:17 INFO Executor: Finished task 5.0 in stage 13.0 (TID 275). 6563 bytes result sent to driver -26/04/01 08:43:17 INFO TaskSetManager: Starting task 13.0 in stage 13.0 (TID 283) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:17 INFO Executor: Running task 13.0 in stage 13.0 (TID 283) -26/04/01 08:43:17 INFO TaskSetManager: Finished task 5.0 in stage 13.0 (TID 275) in 2295 ms on 10.0.0.133 (executor driver) (6/64) -26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:17 INFO Executor: Finished task 6.0 in stage 13.0 (TID 276). 6563 bytes result sent to driver -26/04/01 08:43:17 INFO TaskSetManager: Starting task 14.0 in stage 13.0 (TID 284) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:17 INFO TaskSetManager: Finished task 6.0 in stage 13.0 (TID 276) in 2291 ms on 10.0.0.133 (executor driver) (7/64) -26/04/01 08:43:17 INFO Executor: Running task 14.0 in stage 13.0 (TID 284) -26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:17 INFO Executor: Finished task 7.0 in stage 13.0 (TID 277). 6563 bytes result sent to driver -26/04/01 08:43:17 INFO TaskSetManager: Starting task 15.0 in stage 13.0 (TID 285) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:17 INFO TaskSetManager: Finished task 7.0 in stage 13.0 (TID 277) in 2306 ms on 10.0.0.133 (executor driver) (8/64) -26/04/01 08:43:17 INFO Executor: Running task 15.0 in stage 13.0 (TID 285) -26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:18 INFO Executor: Finished task 8.0 in stage 13.0 (TID 278). 6563 bytes result sent to driver -26/04/01 08:43:18 INFO TaskSetManager: Starting task 16.0 in stage 13.0 (TID 286) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:43:18 INFO TaskSetManager: Finished task 8.0 in stage 13.0 (TID 278) in 2307 ms on 10.0.0.133 (executor driver) (9/64) -26/04/01 08:43:18 INFO Executor: Running task 16.0 in stage 13.0 (TID 286) -26/04/01 08:43:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:18 INFO Executor: Finished task 9.0 in stage 13.0 (TID 279). 6563 bytes result sent to driver -26/04/01 08:43:18 INFO TaskSetManager: Starting task 17.0 in stage 13.0 (TID 287) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:18 INFO TaskSetManager: Finished task 9.0 in stage 13.0 (TID 279) in 2314 ms on 10.0.0.133 (executor driver) (10/64) -26/04/01 08:43:18 INFO Executor: Running task 17.0 in stage 13.0 (TID 287) -26/04/01 08:43:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:18 INFO Executor: Finished task 10.0 in stage 13.0 (TID 280). 6563 bytes result sent to driver -26/04/01 08:43:18 INFO TaskSetManager: Starting task 18.0 in stage 13.0 (TID 288) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:43:18 INFO TaskSetManager: Finished task 10.0 in stage 13.0 (TID 280) in 2286 ms on 10.0.0.133 (executor driver) (11/64) -26/04/01 08:43:18 INFO Executor: Running task 18.0 in stage 13.0 (TID 288) -26/04/01 08:43:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:19 INFO Executor: Finished task 11.0 in stage 13.0 (TID 281). 6563 bytes result sent to driver -26/04/01 08:43:19 INFO TaskSetManager: Starting task 19.0 in stage 13.0 (TID 289) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:19 INFO TaskSetManager: Finished task 11.0 in stage 13.0 (TID 281) in 2280 ms on 10.0.0.133 (executor driver) (12/64) -26/04/01 08:43:19 INFO Executor: Running task 19.0 in stage 13.0 (TID 289) -26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:19 INFO Executor: Finished task 12.0 in stage 13.0 (TID 282). 6563 bytes result sent to driver -26/04/01 08:43:19 INFO TaskSetManager: Starting task 20.0 in stage 13.0 (TID 290) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:19 INFO Executor: Running task 20.0 in stage 13.0 (TID 290) -26/04/01 08:43:19 INFO TaskSetManager: Finished task 12.0 in stage 13.0 (TID 282) in 2301 ms on 10.0.0.133 (executor driver) (13/64) -26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:19 INFO Executor: Finished task 13.0 in stage 13.0 (TID 283). 6563 bytes result sent to driver -26/04/01 08:43:19 INFO TaskSetManager: Starting task 21.0 in stage 13.0 (TID 291) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:43:19 INFO Executor: Running task 21.0 in stage 13.0 (TID 291) -26/04/01 08:43:19 INFO TaskSetManager: Finished task 13.0 in stage 13.0 (TID 283) in 2295 ms on 10.0.0.133 (executor driver) (14/64) -26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:19 INFO Executor: Finished task 14.0 in stage 13.0 (TID 284). 6563 bytes result sent to driver -26/04/01 08:43:19 INFO TaskSetManager: Starting task 22.0 in stage 13.0 (TID 292) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:19 INFO TaskSetManager: Finished task 14.0 in stage 13.0 (TID 284) in 2283 ms on 10.0.0.133 (executor driver) (15/64) -26/04/01 08:43:19 INFO Executor: Running task 22.0 in stage 13.0 (TID 292) -26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:19 INFO Executor: Finished task 15.0 in stage 13.0 (TID 285). 6563 bytes result sent to driver -26/04/01 08:43:19 INFO TaskSetManager: Starting task 23.0 in stage 13.0 (TID 293) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:19 INFO TaskSetManager: Finished task 15.0 in stage 13.0 (TID 285) in 2288 ms on 10.0.0.133 (executor driver) (16/64) -26/04/01 08:43:19 INFO Executor: Running task 23.0 in stage 13.0 (TID 293) -26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:20 INFO Executor: Finished task 16.0 in stage 13.0 (TID 286). 6563 bytes result sent to driver -26/04/01 08:43:20 INFO TaskSetManager: Starting task 24.0 in stage 13.0 (TID 294) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:43:20 INFO Executor: Running task 24.0 in stage 13.0 (TID 294) -26/04/01 08:43:20 INFO TaskSetManager: Finished task 16.0 in stage 13.0 (TID 286) in 2261 ms on 10.0.0.133 (executor driver) (17/64) -26/04/01 08:43:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:20 INFO Executor: Finished task 17.0 in stage 13.0 (TID 287). 6563 bytes result sent to driver -26/04/01 08:43:20 INFO TaskSetManager: Starting task 25.0 in stage 13.0 (TID 295) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:20 INFO TaskSetManager: Finished task 17.0 in stage 13.0 (TID 287) in 2283 ms on 10.0.0.133 (executor driver) (18/64) -26/04/01 08:43:20 INFO Executor: Running task 25.0 in stage 13.0 (TID 295) -26/04/01 08:43:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:20 INFO Executor: Finished task 18.0 in stage 13.0 (TID 288). 6563 bytes result sent to driver -26/04/01 08:43:20 INFO TaskSetManager: Starting task 26.0 in stage 13.0 (TID 296) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:20 INFO TaskSetManager: Finished task 18.0 in stage 13.0 (TID 288) in 2265 ms on 10.0.0.133 (executor driver) (19/64) -26/04/01 08:43:20 INFO Executor: Running task 26.0 in stage 13.0 (TID 296) -26/04/01 08:43:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:21 INFO Executor: Finished task 19.0 in stage 13.0 (TID 289). 6563 bytes result sent to driver -26/04/01 08:43:21 INFO TaskSetManager: Starting task 27.0 in stage 13.0 (TID 297) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:21 INFO TaskSetManager: Finished task 19.0 in stage 13.0 (TID 289) in 2253 ms on 10.0.0.133 (executor driver) (20/64) -26/04/01 08:43:21 INFO Executor: Running task 27.0 in stage 13.0 (TID 297) -26/04/01 08:43:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:22 INFO Executor: Finished task 21.0 in stage 13.0 (TID 291). 6563 bytes result sent to driver -26/04/01 08:43:22 INFO TaskSetManager: Starting task 28.0 in stage 13.0 (TID 298) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:22 INFO Executor: Running task 28.0 in stage 13.0 (TID 298) -26/04/01 08:43:22 INFO TaskSetManager: Finished task 21.0 in stage 13.0 (TID 291) in 2272 ms on 10.0.0.133 (executor driver) (21/64) -26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:22 INFO Executor: Finished task 20.0 in stage 13.0 (TID 290). 6563 bytes result sent to driver -26/04/01 08:43:22 INFO TaskSetManager: Starting task 29.0 in stage 13.0 (TID 299) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:22 INFO Executor: Running task 29.0 in stage 13.0 (TID 299) -26/04/01 08:43:22 INFO TaskSetManager: Finished task 20.0 in stage 13.0 (TID 290) in 2296 ms on 10.0.0.133 (executor driver) (22/64) -26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:22 INFO Executor: Finished task 22.0 in stage 13.0 (TID 292). 6563 bytes result sent to driver -26/04/01 08:43:22 INFO TaskSetManager: Starting task 30.0 in stage 13.0 (TID 300) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:43:22 INFO TaskSetManager: Finished task 22.0 in stage 13.0 (TID 292) in 2279 ms on 10.0.0.133 (executor driver) (23/64) -26/04/01 08:43:22 INFO Executor: Running task 30.0 in stage 13.0 (TID 300) -26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:22 INFO Executor: Finished task 23.0 in stage 13.0 (TID 293). 6563 bytes result sent to driver -26/04/01 08:43:22 INFO TaskSetManager: Starting task 31.0 in stage 13.0 (TID 301) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:43:22 INFO TaskSetManager: Finished task 23.0 in stage 13.0 (TID 293) in 2289 ms on 10.0.0.133 (executor driver) (24/64) -26/04/01 08:43:22 INFO Executor: Running task 31.0 in stage 13.0 (TID 301) -26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:22 INFO Executor: Finished task 24.0 in stage 13.0 (TID 294). 6563 bytes result sent to driver -26/04/01 08:43:22 INFO TaskSetManager: Starting task 32.0 in stage 13.0 (TID 302) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:22 INFO TaskSetManager: Finished task 24.0 in stage 13.0 (TID 294) in 2267 ms on 10.0.0.133 (executor driver) (25/64) -26/04/01 08:43:22 INFO Executor: Running task 32.0 in stage 13.0 (TID 302) -26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:22 INFO Executor: Finished task 25.0 in stage 13.0 (TID 295). 6563 bytes result sent to driver -26/04/01 08:43:22 INFO TaskSetManager: Starting task 33.0 in stage 13.0 (TID 303) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:22 INFO TaskSetManager: Finished task 25.0 in stage 13.0 (TID 295) in 2261 ms on 10.0.0.133 (executor driver) (26/64) -26/04/01 08:43:22 INFO Executor: Running task 33.0 in stage 13.0 (TID 303) -26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:22 INFO Executor: Finished task 26.0 in stage 13.0 (TID 296). 6563 bytes result sent to driver -26/04/01 08:43:22 INFO TaskSetManager: Starting task 34.0 in stage 13.0 (TID 304) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:22 INFO TaskSetManager: Finished task 26.0 in stage 13.0 (TID 296) in 2269 ms on 10.0.0.133 (executor driver) (27/64) -26/04/01 08:43:22 INFO Executor: Running task 34.0 in stage 13.0 (TID 304) -26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:23 INFO Executor: Finished task 27.0 in stage 13.0 (TID 297). 6563 bytes result sent to driver -26/04/01 08:43:23 INFO TaskSetManager: Starting task 35.0 in stage 13.0 (TID 305) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:23 INFO TaskSetManager: Finished task 27.0 in stage 13.0 (TID 297) in 2245 ms on 10.0.0.133 (executor driver) (28/64) -26/04/01 08:43:23 INFO Executor: Running task 35.0 in stage 13.0 (TID 305) -26/04/01 08:43:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:24 INFO Executor: Finished task 32.0 in stage 13.0 (TID 302). 6563 bytes result sent to driver -26/04/01 08:43:24 INFO TaskSetManager: Starting task 36.0 in stage 13.0 (TID 306) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:24 INFO Executor: Running task 36.0 in stage 13.0 (TID 306) -26/04/01 08:43:24 INFO TaskSetManager: Finished task 32.0 in stage 13.0 (TID 302) in 1596 ms on 10.0.0.133 (executor driver) (29/64) -26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:24 INFO Executor: Finished task 33.0 in stage 13.0 (TID 303). 6563 bytes result sent to driver -26/04/01 08:43:24 INFO TaskSetManager: Starting task 37.0 in stage 13.0 (TID 307) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:24 INFO TaskSetManager: Finished task 33.0 in stage 13.0 (TID 303) in 1595 ms on 10.0.0.133 (executor driver) (30/64) -26/04/01 08:43:24 INFO Executor: Running task 37.0 in stage 13.0 (TID 307) -26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:24 INFO Executor: Finished task 34.0 in stage 13.0 (TID 304). 6563 bytes result sent to driver -26/04/01 08:43:24 INFO TaskSetManager: Starting task 38.0 in stage 13.0 (TID 308) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:24 INFO TaskSetManager: Finished task 34.0 in stage 13.0 (TID 304) in 1585 ms on 10.0.0.133 (executor driver) (31/64) -26/04/01 08:43:24 INFO Executor: Running task 38.0 in stage 13.0 (TID 308) -26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:24 INFO Executor: Finished task 29.0 in stage 13.0 (TID 299). 6563 bytes result sent to driver -26/04/01 08:43:24 INFO TaskSetManager: Starting task 39.0 in stage 13.0 (TID 309) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:24 INFO TaskSetManager: Finished task 29.0 in stage 13.0 (TID 299) in 2290 ms on 10.0.0.133 (executor driver) (32/64) -26/04/01 08:43:24 INFO Executor: Running task 39.0 in stage 13.0 (TID 309) -26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:24 INFO Executor: Finished task 28.0 in stage 13.0 (TID 298). 6563 bytes result sent to driver -26/04/01 08:43:24 INFO TaskSetManager: Starting task 40.0 in stage 13.0 (TID 310) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:24 INFO TaskSetManager: Finished task 28.0 in stage 13.0 (TID 298) in 2301 ms on 10.0.0.133 (executor driver) (33/64) -26/04/01 08:43:24 INFO Executor: Running task 40.0 in stage 13.0 (TID 310) -26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:24 INFO Executor: Finished task 30.0 in stage 13.0 (TID 300). 6563 bytes result sent to driver -26/04/01 08:43:24 INFO TaskSetManager: Starting task 41.0 in stage 13.0 (TID 311) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:24 INFO TaskSetManager: Finished task 30.0 in stage 13.0 (TID 300) in 2294 ms on 10.0.0.133 (executor driver) (34/64) -26/04/01 08:43:24 INFO Executor: Running task 41.0 in stage 13.0 (TID 311) -26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:24 INFO Executor: Finished task 31.0 in stage 13.0 (TID 301). 6563 bytes result sent to driver -26/04/01 08:43:24 INFO TaskSetManager: Starting task 42.0 in stage 13.0 (TID 312) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:24 INFO TaskSetManager: Finished task 31.0 in stage 13.0 (TID 301) in 2285 ms on 10.0.0.133 (executor driver) (35/64) -26/04/01 08:43:24 INFO Executor: Running task 42.0 in stage 13.0 (TID 312) -26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:25 INFO Executor: Finished task 35.0 in stage 13.0 (TID 305). 6563 bytes result sent to driver -26/04/01 08:43:25 INFO TaskSetManager: Starting task 43.0 in stage 13.0 (TID 313) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:25 INFO TaskSetManager: Finished task 35.0 in stage 13.0 (TID 305) in 1570 ms on 10.0.0.133 (executor driver) (36/64) -26/04/01 08:43:25 INFO Executor: Running task 43.0 in stage 13.0 (TID 313) -26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:25 INFO Executor: Finished task 36.0 in stage 13.0 (TID 306). 6563 bytes result sent to driver -26/04/01 08:43:25 INFO TaskSetManager: Starting task 44.0 in stage 13.0 (TID 314) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:25 INFO Executor: Running task 44.0 in stage 13.0 (TID 314) -26/04/01 08:43:25 INFO TaskSetManager: Finished task 36.0 in stage 13.0 (TID 306) in 1597 ms on 10.0.0.133 (executor driver) (37/64) -26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:25 INFO Executor: Finished task 37.0 in stage 13.0 (TID 307). 6563 bytes result sent to driver -26/04/01 08:43:25 INFO TaskSetManager: Starting task 45.0 in stage 13.0 (TID 315) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:25 INFO Executor: Running task 45.0 in stage 13.0 (TID 315) -26/04/01 08:43:25 INFO TaskSetManager: Finished task 37.0 in stage 13.0 (TID 307) in 1612 ms on 10.0.0.133 (executor driver) (38/64) -26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:25 INFO Executor: Finished task 38.0 in stage 13.0 (TID 308). 6563 bytes result sent to driver -26/04/01 08:43:25 INFO TaskSetManager: Starting task 46.0 in stage 13.0 (TID 316) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:25 INFO TaskSetManager: Finished task 38.0 in stage 13.0 (TID 308) in 1605 ms on 10.0.0.133 (executor driver) (39/64) -26/04/01 08:43:25 INFO Executor: Running task 46.0 in stage 13.0 (TID 316) -26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:26 INFO Executor: Finished task 40.0 in stage 13.0 (TID 310). 6563 bytes result sent to driver -26/04/01 08:43:26 INFO TaskSetManager: Starting task 47.0 in stage 13.0 (TID 317) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:26 INFO Executor: Running task 47.0 in stage 13.0 (TID 317) -26/04/01 08:43:26 INFO TaskSetManager: Finished task 40.0 in stage 13.0 (TID 310) in 1595 ms on 10.0.0.133 (executor driver) (40/64) -26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:26 INFO Executor: Finished task 39.0 in stage 13.0 (TID 309). 6563 bytes result sent to driver -26/04/01 08:43:26 INFO TaskSetManager: Starting task 48.0 in stage 13.0 (TID 318) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:26 INFO TaskSetManager: Finished task 39.0 in stage 13.0 (TID 309) in 1599 ms on 10.0.0.133 (executor driver) (41/64) -26/04/01 08:43:26 INFO Executor: Running task 48.0 in stage 13.0 (TID 318) -26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:26 INFO Executor: Finished task 41.0 in stage 13.0 (TID 311). 6563 bytes result sent to driver -26/04/01 08:43:26 INFO TaskSetManager: Starting task 49.0 in stage 13.0 (TID 319) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:26 INFO TaskSetManager: Finished task 41.0 in stage 13.0 (TID 311) in 1596 ms on 10.0.0.133 (executor driver) (42/64) -26/04/01 08:43:26 INFO Executor: Running task 49.0 in stage 13.0 (TID 319) -26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:26 INFO Executor: Finished task 42.0 in stage 13.0 (TID 312). 6563 bytes result sent to driver -26/04/01 08:43:26 INFO TaskSetManager: Starting task 50.0 in stage 13.0 (TID 320) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:26 INFO TaskSetManager: Finished task 42.0 in stage 13.0 (TID 312) in 1592 ms on 10.0.0.133 (executor driver) (43/64) -26/04/01 08:43:26 INFO Executor: Running task 50.0 in stage 13.0 (TID 320) -26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:26 INFO Executor: Finished task 43.0 in stage 13.0 (TID 313). 6563 bytes result sent to driver -26/04/01 08:43:26 INFO TaskSetManager: Starting task 51.0 in stage 13.0 (TID 321) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:26 INFO TaskSetManager: Finished task 43.0 in stage 13.0 (TID 313) in 1579 ms on 10.0.0.133 (executor driver) (44/64) -26/04/01 08:43:26 INFO Executor: Running task 51.0 in stage 13.0 (TID 321) -26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:27 INFO Executor: Finished task 44.0 in stage 13.0 (TID 314). 6563 bytes result sent to driver -26/04/01 08:43:27 INFO TaskSetManager: Starting task 52.0 in stage 13.0 (TID 322) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:27 INFO TaskSetManager: Finished task 44.0 in stage 13.0 (TID 314) in 1611 ms on 10.0.0.133 (executor driver) (45/64) -26/04/01 08:43:27 INFO Executor: Running task 52.0 in stage 13.0 (TID 322) -26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:27 INFO Executor: Finished task 45.0 in stage 13.0 (TID 315). 6563 bytes result sent to driver -26/04/01 08:43:27 INFO TaskSetManager: Starting task 53.0 in stage 13.0 (TID 323) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:27 INFO TaskSetManager: Finished task 45.0 in stage 13.0 (TID 315) in 1602 ms on 10.0.0.133 (executor driver) (46/64) -26/04/01 08:43:27 INFO Executor: Running task 53.0 in stage 13.0 (TID 323) -26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:27 INFO Executor: Finished task 46.0 in stage 13.0 (TID 316). 6563 bytes result sent to driver -26/04/01 08:43:27 INFO TaskSetManager: Starting task 54.0 in stage 13.0 (TID 324) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:27 INFO Executor: Running task 54.0 in stage 13.0 (TID 324) -26/04/01 08:43:27 INFO TaskSetManager: Finished task 46.0 in stage 13.0 (TID 316) in 1615 ms on 10.0.0.133 (executor driver) (47/64) -26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:27 INFO Executor: Finished task 48.0 in stage 13.0 (TID 318). 6563 bytes result sent to driver -26/04/01 08:43:27 INFO TaskSetManager: Starting task 55.0 in stage 13.0 (TID 325) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:27 INFO TaskSetManager: Finished task 48.0 in stage 13.0 (TID 318) in 1615 ms on 10.0.0.133 (executor driver) (48/64) -26/04/01 08:43:27 INFO Executor: Running task 55.0 in stage 13.0 (TID 325) -26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:27 INFO Executor: Finished task 47.0 in stage 13.0 (TID 317). 6563 bytes result sent to driver -26/04/01 08:43:27 INFO TaskSetManager: Starting task 56.0 in stage 13.0 (TID 326) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:27 INFO TaskSetManager: Finished task 47.0 in stage 13.0 (TID 317) in 1626 ms on 10.0.0.133 (executor driver) (49/64) -26/04/01 08:43:27 INFO Executor: Running task 56.0 in stage 13.0 (TID 326) -26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:27 INFO Executor: Finished task 49.0 in stage 13.0 (TID 319). 6563 bytes result sent to driver -26/04/01 08:43:27 INFO TaskSetManager: Starting task 57.0 in stage 13.0 (TID 327) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:27 INFO TaskSetManager: Finished task 49.0 in stage 13.0 (TID 319) in 1616 ms on 10.0.0.133 (executor driver) (50/64) -26/04/01 08:43:27 INFO Executor: Running task 57.0 in stage 13.0 (TID 327) -26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:27 INFO Executor: Finished task 50.0 in stage 13.0 (TID 320). 6563 bytes result sent to driver -26/04/01 08:43:27 INFO TaskSetManager: Starting task 58.0 in stage 13.0 (TID 328) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:27 INFO TaskSetManager: Finished task 50.0 in stage 13.0 (TID 320) in 1607 ms on 10.0.0.133 (executor driver) (51/64) -26/04/01 08:43:27 INFO Executor: Running task 58.0 in stage 13.0 (TID 328) -26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:28 INFO Executor: Finished task 51.0 in stage 13.0 (TID 321). 6563 bytes result sent to driver -26/04/01 08:43:28 INFO TaskSetManager: Starting task 59.0 in stage 13.0 (TID 329) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:28 INFO TaskSetManager: Finished task 51.0 in stage 13.0 (TID 321) in 1570 ms on 10.0.0.133 (executor driver) (52/64) -26/04/01 08:43:28 INFO Executor: Running task 59.0 in stage 13.0 (TID 329) -26/04/01 08:43:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO Executor: Finished task 52.0 in stage 13.0 (TID 322). 6563 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 60.0 in stage 13.0 (TID 330) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 52.0 in stage 13.0 (TID 322) in 1597 ms on 10.0.0.133 (executor driver) (53/64) -26/04/01 08:43:29 INFO Executor: Running task 60.0 in stage 13.0 (TID 330) -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO Executor: Finished task 53.0 in stage 13.0 (TID 323). 6563 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 61.0 in stage 13.0 (TID 331) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 53.0 in stage 13.0 (TID 323) in 1608 ms on 10.0.0.133 (executor driver) (54/64) -26/04/01 08:43:29 INFO Executor: Running task 61.0 in stage 13.0 (TID 331) -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO Executor: Finished task 54.0 in stage 13.0 (TID 324). 6563 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 62.0 in stage 13.0 (TID 332) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 54.0 in stage 13.0 (TID 324) in 1606 ms on 10.0.0.133 (executor driver) (55/64) -26/04/01 08:43:29 INFO Executor: Running task 62.0 in stage 13.0 (TID 332) -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO Executor: Finished task 56.0 in stage 13.0 (TID 326). 6563 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 55.0 in stage 13.0 (TID 325). 6563 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 63.0 in stage 13.0 (TID 333) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:43:29 INFO Executor: Running task 63.0 in stage 13.0 (TID 333) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 334) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO TaskSetManager: Finished task 56.0 in stage 13.0 (TID 326) in 1602 ms on 10.0.0.133 (executor driver) (56/64) -26/04/01 08:43:29 INFO Executor: Running task 0.0 in stage 15.0 (TID 334) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 55.0 in stage 13.0 (TID 325) in 1610 ms on 10.0.0.133 (executor driver) (57/64) -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:43:29 INFO Executor: Finished task 0.0 in stage 15.0 (TID 334). 33895 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 1.0 in stage 15.0 (TID 335) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 1.0 in stage 15.0 (TID 335) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 334) in 26 ms on 10.0.0.133 (executor driver) (1/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 1.0 in stage 15.0 (TID 335). 34027 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 2.0 in stage 15.0 (TID 336) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 2.0 in stage 15.0 (TID 336) -26/04/01 08:43:29 INFO Executor: Finished task 57.0 in stage 13.0 (TID 327). 6563 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 3.0 in stage 15.0 (TID 337) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 1.0 in stage 15.0 (TID 335) in 6 ms on 10.0.0.133 (executor driver) (2/200) -26/04/01 08:43:29 INFO Executor: Running task 3.0 in stage 15.0 (TID 337) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 57.0 in stage 13.0 (TID 327) in 1594 ms on 10.0.0.133 (executor driver) (58/64) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO BlockManagerInfo: Removed broadcast_17_piece0 on 10.0.0.133:59303 in memory (size: 8.0 KiB, free: 8.6 GiB) -26/04/01 08:43:29 INFO Executor: Finished task 2.0 in stage 15.0 (TID 336). 33844 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 3.0 in stage 15.0 (TID 337). 34371 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 4.0 in stage 15.0 (TID 338) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 4.0 in stage 15.0 (TID 338) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 5.0 in stage 15.0 (TID 339) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 2.0 in stage 15.0 (TID 336) in 7 ms on 10.0.0.133 (executor driver) (3/200) -26/04/01 08:43:29 INFO Executor: Running task 5.0 in stage 15.0 (TID 339) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 3.0 in stage 15.0 (TID 337) in 7 ms on 10.0.0.133 (executor driver) (4/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 4.0 in stage 15.0 (TID 338). 34473 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 5.0 in stage 15.0 (TID 339). 34156 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 6.0 in stage 15.0 (TID 340) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 7.0 in stage 15.0 (TID 341) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 6.0 in stage 15.0 (TID 340) -26/04/01 08:43:29 INFO Executor: Running task 7.0 in stage 15.0 (TID 341) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 4.0 in stage 15.0 (TID 338) in 6 ms on 10.0.0.133 (executor driver) (5/200) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 5.0 in stage 15.0 (TID 339) in 6 ms on 10.0.0.133 (executor driver) (6/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 6.0 in stage 15.0 (TID 340). 33938 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 8.0 in stage 15.0 (TID 342) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Finished task 7.0 in stage 15.0 (TID 341). 34062 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Finished task 6.0 in stage 15.0 (TID 340) in 5 ms on 10.0.0.133 (executor driver) (7/200) -26/04/01 08:43:29 INFO Executor: Running task 8.0 in stage 15.0 (TID 342) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 9.0 in stage 15.0 (TID 343) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 9.0 in stage 15.0 (TID 343) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 7.0 in stage 15.0 (TID 341) in 5 ms on 10.0.0.133 (executor driver) (8/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 9.0 in stage 15.0 (TID 343). 34979 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 8.0 in stage 15.0 (TID 342). 33972 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 10.0 in stage 15.0 (TID 344) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 10.0 in stage 15.0 (TID 344) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 11.0 in stage 15.0 (TID 345) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 9.0 in stage 15.0 (TID 343) in 6 ms on 10.0.0.133 (executor driver) (9/200) -26/04/01 08:43:29 INFO Executor: Running task 11.0 in stage 15.0 (TID 345) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 8.0 in stage 15.0 (TID 342) in 6 ms on 10.0.0.133 (executor driver) (10/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 10.0 in stage 15.0 (TID 344). 34352 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 11.0 in stage 15.0 (TID 345). 33620 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 12.0 in stage 15.0 (TID 346) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 12.0 in stage 15.0 (TID 346) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 13.0 in stage 15.0 (TID 347) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 10.0 in stage 15.0 (TID 344) in 5 ms on 10.0.0.133 (executor driver) (11/200) -26/04/01 08:43:29 INFO Executor: Running task 13.0 in stage 15.0 (TID 347) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 11.0 in stage 15.0 (TID 345) in 5 ms on 10.0.0.133 (executor driver) (12/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 12.0 in stage 15.0 (TID 346). 34084 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 14.0 in stage 15.0 (TID 348) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 14.0 in stage 15.0 (TID 348) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 12.0 in stage 15.0 (TID 346) in 5 ms on 10.0.0.133 (executor driver) (13/200) -26/04/01 08:43:29 INFO Executor: Finished task 13.0 in stage 15.0 (TID 347). 34520 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 15.0 in stage 15.0 (TID 349) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 15.0 in stage 15.0 (TID 349) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 13.0 in stage 15.0 (TID 347) in 5 ms on 10.0.0.133 (executor driver) (14/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 14.0 in stage 15.0 (TID 348). 34658 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 16.0 in stage 15.0 (TID 350) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 16.0 in stage 15.0 (TID 350) -26/04/01 08:43:29 INFO Executor: Finished task 15.0 in stage 15.0 (TID 349). 34244 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Finished task 14.0 in stage 15.0 (TID 348) in 5 ms on 10.0.0.133 (executor driver) (15/200) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 17.0 in stage 15.0 (TID 351) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 17.0 in stage 15.0 (TID 351) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 15.0 in stage 15.0 (TID 349) in 4 ms on 10.0.0.133 (executor driver) (16/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 16.0 in stage 15.0 (TID 350). 33628 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 18.0 in stage 15.0 (TID 352) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 18.0 in stage 15.0 (TID 352) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 16.0 in stage 15.0 (TID 350) in 5 ms on 10.0.0.133 (executor driver) (17/200) -26/04/01 08:43:29 INFO Executor: Finished task 17.0 in stage 15.0 (TID 351). 34240 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 19.0 in stage 15.0 (TID 353) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO Executor: Running task 19.0 in stage 15.0 (TID 353) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Finished task 17.0 in stage 15.0 (TID 351) in 6 ms on 10.0.0.133 (executor driver) (18/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 18.0 in stage 15.0 (TID 352). 34217 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 20.0 in stage 15.0 (TID 354) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 20.0 in stage 15.0 (TID 354) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 18.0 in stage 15.0 (TID 352) in 4 ms on 10.0.0.133 (executor driver) (19/200) -26/04/01 08:43:29 INFO Executor: Finished task 19.0 in stage 15.0 (TID 353). 35085 bytes result sent to driver -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Starting task 21.0 in stage 15.0 (TID 355) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 21.0 in stage 15.0 (TID 355) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 19.0 in stage 15.0 (TID 353) in 5 ms on 10.0.0.133 (executor driver) (20/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 20.0 in stage 15.0 (TID 354). 34559 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 22.0 in stage 15.0 (TID 356) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 22.0 in stage 15.0 (TID 356) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 20.0 in stage 15.0 (TID 354) in 5 ms on 10.0.0.133 (executor driver) (21/200) -26/04/01 08:43:29 INFO Executor: Finished task 58.0 in stage 13.0 (TID 328). 6606 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 23.0 in stage 15.0 (TID 357) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 23.0 in stage 15.0 (TID 357) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 58.0 in stage 13.0 (TID 328) in 1597 ms on 10.0.0.133 (executor driver) (59/64) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 21.0 in stage 15.0 (TID 355). 34824 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 24.0 in stage 15.0 (TID 358) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Running task 24.0 in stage 15.0 (TID 358) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 21.0 in stage 15.0 (TID 355) in 5 ms on 10.0.0.133 (executor driver) (22/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 22.0 in stage 15.0 (TID 356). 34572 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 23.0 in stage 15.0 (TID 357). 34082 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 25.0 in stage 15.0 (TID 359) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 25.0 in stage 15.0 (TID 359) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 26.0 in stage 15.0 (TID 360) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 22.0 in stage 15.0 (TID 356) in 5 ms on 10.0.0.133 (executor driver) (23/200) -26/04/01 08:43:29 INFO Executor: Running task 26.0 in stage 15.0 (TID 360) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 23.0 in stage 15.0 (TID 357) in 5 ms on 10.0.0.133 (executor driver) (24/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO Executor: Finished task 24.0 in stage 15.0 (TID 358). 34683 bytes result sent to driver -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Starting task 27.0 in stage 15.0 (TID 361) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 27.0 in stage 15.0 (TID 361) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 24.0 in stage 15.0 (TID 358) in 5 ms on 10.0.0.133 (executor driver) (25/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 25.0 in stage 15.0 (TID 359). 34085 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 28.0 in stage 15.0 (TID 362) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 28.0 in stage 15.0 (TID 362) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 25.0 in stage 15.0 (TID 359) in 5 ms on 10.0.0.133 (executor driver) (26/200) -26/04/01 08:43:29 INFO Executor: Finished task 26.0 in stage 15.0 (TID 360). 33825 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 29.0 in stage 15.0 (TID 363) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 29.0 in stage 15.0 (TID 363) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO TaskSetManager: Finished task 26.0 in stage 15.0 (TID 360) in 4 ms on 10.0.0.133 (executor driver) (27/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 27.0 in stage 15.0 (TID 361). 34141 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 30.0 in stage 15.0 (TID 364) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 30.0 in stage 15.0 (TID 364) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 27.0 in stage 15.0 (TID 361) in 4 ms on 10.0.0.133 (executor driver) (28/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 28.0 in stage 15.0 (TID 362). 34850 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 31.0 in stage 15.0 (TID 365) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 31.0 in stage 15.0 (TID 365) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 28.0 in stage 15.0 (TID 362) in 4 ms on 10.0.0.133 (executor driver) (29/200) -26/04/01 08:43:29 INFO Executor: Finished task 29.0 in stage 15.0 (TID 363). 33924 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 32.0 in stage 15.0 (TID 366) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 32.0 in stage 15.0 (TID 366) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 29.0 in stage 15.0 (TID 363) in 5 ms on 10.0.0.133 (executor driver) (30/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 30.0 in stage 15.0 (TID 364). 33936 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 33.0 in stage 15.0 (TID 367) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 30.0 in stage 15.0 (TID 364) in 5 ms on 10.0.0.133 (executor driver) (31/200) -26/04/01 08:43:29 INFO Executor: Running task 33.0 in stage 15.0 (TID 367) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 31.0 in stage 15.0 (TID 365). 34338 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 34.0 in stage 15.0 (TID 368) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 34.0 in stage 15.0 (TID 368) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 31.0 in stage 15.0 (TID 365) in 5 ms on 10.0.0.133 (executor driver) (32/200) -26/04/01 08:43:29 INFO Executor: Finished task 32.0 in stage 15.0 (TID 366). 34649 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 35.0 in stage 15.0 (TID 369) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 35.0 in stage 15.0 (TID 369) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 32.0 in stage 15.0 (TID 366) in 4 ms on 10.0.0.133 (executor driver) (33/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 33.0 in stage 15.0 (TID 367). 34079 bytes result sent to driver -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Starting task 36.0 in stage 15.0 (TID 370) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 33.0 in stage 15.0 (TID 367) in 4 ms on 10.0.0.133 (executor driver) (34/200) -26/04/01 08:43:29 INFO Executor: Running task 36.0 in stage 15.0 (TID 370) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 34.0 in stage 15.0 (TID 368). 34290 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 37.0 in stage 15.0 (TID 371) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 37.0 in stage 15.0 (TID 371) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 34.0 in stage 15.0 (TID 368) in 5 ms on 10.0.0.133 (executor driver) (35/200) -26/04/01 08:43:29 INFO Executor: Finished task 35.0 in stage 15.0 (TID 369). 34960 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 38.0 in stage 15.0 (TID 372) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 38.0 in stage 15.0 (TID 372) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 35.0 in stage 15.0 (TID 369) in 5 ms on 10.0.0.133 (executor driver) (36/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 36.0 in stage 15.0 (TID 370). 34927 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 39.0 in stage 15.0 (TID 373) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 39.0 in stage 15.0 (TID 373) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 36.0 in stage 15.0 (TID 370) in 5 ms on 10.0.0.133 (executor driver) (37/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 38.0 in stage 15.0 (TID 372). 33680 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 37.0 in stage 15.0 (TID 371). 34242 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 40.0 in stage 15.0 (TID 374) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 40.0 in stage 15.0 (TID 374) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 41.0 in stage 15.0 (TID 375) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 38.0 in stage 15.0 (TID 372) in 5 ms on 10.0.0.133 (executor driver) (38/200) -26/04/01 08:43:29 INFO Executor: Running task 41.0 in stage 15.0 (TID 375) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 37.0 in stage 15.0 (TID 371) in 5 ms on 10.0.0.133 (executor driver) (39/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 39.0 in stage 15.0 (TID 373). 33691 bytes result sent to driver -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Starting task 42.0 in stage 15.0 (TID 376) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 42.0 in stage 15.0 (TID 376) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 39.0 in stage 15.0 (TID 373) in 5 ms on 10.0.0.133 (executor driver) (40/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 40.0 in stage 15.0 (TID 374). 34584 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 41.0 in stage 15.0 (TID 375). 34443 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 43.0 in stage 15.0 (TID 377) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 43.0 in stage 15.0 (TID 377) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 44.0 in stage 15.0 (TID 378) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Finished task 42.0 in stage 15.0 (TID 376). 34213 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Finished task 40.0 in stage 15.0 (TID 374) in 5 ms on 10.0.0.133 (executor driver) (41/200) -26/04/01 08:43:29 INFO Executor: Running task 44.0 in stage 15.0 (TID 378) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 45.0 in stage 15.0 (TID 379) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 41.0 in stage 15.0 (TID 375) in 6 ms on 10.0.0.133 (executor driver) (42/200) -26/04/01 08:43:29 INFO Executor: Running task 45.0 in stage 15.0 (TID 379) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 42.0 in stage 15.0 (TID 376) in 5 ms on 10.0.0.133 (executor driver) (43/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 43.0 in stage 15.0 (TID 377). 33552 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 46.0 in stage 15.0 (TID 380) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 46.0 in stage 15.0 (TID 380) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 43.0 in stage 15.0 (TID 377) in 5 ms on 10.0.0.133 (executor driver) (44/200) -26/04/01 08:43:29 INFO Executor: Finished task 44.0 in stage 15.0 (TID 378). 33901 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 45.0 in stage 15.0 (TID 379). 34122 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 47.0 in stage 15.0 (TID 381) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 47.0 in stage 15.0 (TID 381) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 48.0 in stage 15.0 (TID 382) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO TaskSetManager: Finished task 44.0 in stage 15.0 (TID 378) in 7 ms on 10.0.0.133 (executor driver) (45/200) -26/04/01 08:43:29 INFO Executor: Running task 48.0 in stage 15.0 (TID 382) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 45.0 in stage 15.0 (TID 379) in 7 ms on 10.0.0.133 (executor driver) (46/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 46.0 in stage 15.0 (TID 380). 34158 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 49.0 in stage 15.0 (TID 383) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 49.0 in stage 15.0 (TID 383) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 46.0 in stage 15.0 (TID 380) in 7 ms on 10.0.0.133 (executor driver) (47/200) -26/04/01 08:43:29 INFO Executor: Finished task 47.0 in stage 15.0 (TID 381). 34470 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 50.0 in stage 15.0 (TID 384) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 50.0 in stage 15.0 (TID 384) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 47.0 in stage 15.0 (TID 381) in 5 ms on 10.0.0.133 (executor driver) (48/200) -26/04/01 08:43:29 INFO Executor: Finished task 48.0 in stage 15.0 (TID 382). 33889 bytes result sent to driver -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Starting task 51.0 in stage 15.0 (TID 385) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 51.0 in stage 15.0 (TID 385) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 48.0 in stage 15.0 (TID 382) in 5 ms on 10.0.0.133 (executor driver) (49/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 49.0 in stage 15.0 (TID 383). 33761 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 52.0 in stage 15.0 (TID 386) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 52.0 in stage 15.0 (TID 386) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 49.0 in stage 15.0 (TID 383) in 5 ms on 10.0.0.133 (executor driver) (50/200) -26/04/01 08:43:29 INFO Executor: Finished task 50.0 in stage 15.0 (TID 384). 34418 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 53.0 in stage 15.0 (TID 387) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 53.0 in stage 15.0 (TID 387) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 50.0 in stage 15.0 (TID 384) in 4 ms on 10.0.0.133 (executor driver) (51/200) -26/04/01 08:43:29 INFO Executor: Finished task 51.0 in stage 15.0 (TID 385). 34243 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 54.0 in stage 15.0 (TID 388) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 54.0 in stage 15.0 (TID 388) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 51.0 in stage 15.0 (TID 385) in 4 ms on 10.0.0.133 (executor driver) (52/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 54.0 in stage 15.0 (TID 388). 34702 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 52.0 in stage 15.0 (TID 386). 34119 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 53.0 in stage 15.0 (TID 387). 34449 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 55.0 in stage 15.0 (TID 389) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 55.0 in stage 15.0 (TID 389) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 56.0 in stage 15.0 (TID 390) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 54.0 in stage 15.0 (TID 388) in 5 ms on 10.0.0.133 (executor driver) (53/200) -26/04/01 08:43:29 INFO Executor: Running task 56.0 in stage 15.0 (TID 390) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 57.0 in stage 15.0 (TID 391) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 57.0 in stage 15.0 (TID 391) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 52.0 in stage 15.0 (TID 386) in 5 ms on 10.0.0.133 (executor driver) (54/200) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 53.0 in stage 15.0 (TID 387) in 5 ms on 10.0.0.133 (executor driver) (55/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 56.0 in stage 15.0 (TID 390). 34901 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 55.0 in stage 15.0 (TID 389). 34244 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 58.0 in stage 15.0 (TID 392) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 58.0 in stage 15.0 (TID 392) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 59.0 in stage 15.0 (TID 393) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 56.0 in stage 15.0 (TID 390) in 4 ms on 10.0.0.133 (executor driver) (56/200) -26/04/01 08:43:29 INFO Executor: Running task 59.0 in stage 15.0 (TID 393) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 55.0 in stage 15.0 (TID 389) in 4 ms on 10.0.0.133 (executor driver) (57/200) -26/04/01 08:43:29 INFO Executor: Finished task 57.0 in stage 15.0 (TID 391). 34181 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 60.0 in stage 15.0 (TID 394) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 57.0 in stage 15.0 (TID 391) in 5 ms on 10.0.0.133 (executor driver) (58/200) -26/04/01 08:43:29 INFO Executor: Running task 60.0 in stage 15.0 (TID 394) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 58.0 in stage 15.0 (TID 392). 34726 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 61.0 in stage 15.0 (TID 395) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 61.0 in stage 15.0 (TID 395) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 58.0 in stage 15.0 (TID 392) in 4 ms on 10.0.0.133 (executor driver) (59/200) -26/04/01 08:43:29 INFO Executor: Finished task 59.0 in stage 15.0 (TID 393). 34502 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 62.0 in stage 15.0 (TID 396) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 62.0 in stage 15.0 (TID 396) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 59.0 in stage 15.0 (TID 393) in 5 ms on 10.0.0.133 (executor driver) (60/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO Executor: Finished task 60.0 in stage 15.0 (TID 394). 34580 bytes result sent to driver -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Starting task 63.0 in stage 15.0 (TID 397) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 63.0 in stage 15.0 (TID 397) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 60.0 in stage 15.0 (TID 394) in 5 ms on 10.0.0.133 (executor driver) (61/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 61.0 in stage 15.0 (TID 395). 33977 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 62.0 in stage 15.0 (TID 396). 34073 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 64.0 in stage 15.0 (TID 398) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 64.0 in stage 15.0 (TID 398) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 65.0 in stage 15.0 (TID 399) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 61.0 in stage 15.0 (TID 395) in 4 ms on 10.0.0.133 (executor driver) (62/200) -26/04/01 08:43:29 INFO Executor: Running task 65.0 in stage 15.0 (TID 399) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 62.0 in stage 15.0 (TID 396) in 4 ms on 10.0.0.133 (executor driver) (63/200) -26/04/01 08:43:29 INFO Executor: Finished task 63.0 in stage 15.0 (TID 397). 33522 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 66.0 in stage 15.0 (TID 400) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 66.0 in stage 15.0 (TID 400) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 63.0 in stage 15.0 (TID 397) in 4 ms on 10.0.0.133 (executor driver) (64/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 64.0 in stage 15.0 (TID 398). 33929 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 67.0 in stage 15.0 (TID 401) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Finished task 65.0 in stage 15.0 (TID 399). 34066 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Running task 67.0 in stage 15.0 (TID 401) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 68.0 in stage 15.0 (TID 402) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 64.0 in stage 15.0 (TID 398) in 5 ms on 10.0.0.133 (executor driver) (65/200) -26/04/01 08:43:29 INFO Executor: Running task 68.0 in stage 15.0 (TID 402) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 65.0 in stage 15.0 (TID 399) in 5 ms on 10.0.0.133 (executor driver) (66/200) -26/04/01 08:43:29 INFO Executor: Finished task 66.0 in stage 15.0 (TID 400). 34407 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 69.0 in stage 15.0 (TID 403) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 69.0 in stage 15.0 (TID 403) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 66.0 in stage 15.0 (TID 400) in 4 ms on 10.0.0.133 (executor driver) (67/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 67.0 in stage 15.0 (TID 401). 34324 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 70.0 in stage 15.0 (TID 404) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 70.0 in stage 15.0 (TID 404) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 67.0 in stage 15.0 (TID 401) in 3 ms on 10.0.0.133 (executor driver) (68/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 68.0 in stage 15.0 (TID 402). 33789 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 69.0 in stage 15.0 (TID 403). 33810 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 71.0 in stage 15.0 (TID 405) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 71.0 in stage 15.0 (TID 405) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 72.0 in stage 15.0 (TID 406) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 68.0 in stage 15.0 (TID 402) in 5 ms on 10.0.0.133 (executor driver) (69/200) -26/04/01 08:43:29 INFO Executor: Running task 72.0 in stage 15.0 (TID 406) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 69.0 in stage 15.0 (TID 403) in 5 ms on 10.0.0.133 (executor driver) (70/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 70.0 in stage 15.0 (TID 404). 34912 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 73.0 in stage 15.0 (TID 407) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 73.0 in stage 15.0 (TID 407) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 70.0 in stage 15.0 (TID 404) in 4 ms on 10.0.0.133 (executor driver) (71/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 71.0 in stage 15.0 (TID 405). 34265 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 74.0 in stage 15.0 (TID 408) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 74.0 in stage 15.0 (TID 408) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 71.0 in stage 15.0 (TID 405) in 5 ms on 10.0.0.133 (executor driver) (72/200) -26/04/01 08:43:29 INFO Executor: Finished task 72.0 in stage 15.0 (TID 406). 34272 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 75.0 in stage 15.0 (TID 409) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 75.0 in stage 15.0 (TID 409) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 72.0 in stage 15.0 (TID 406) in 4 ms on 10.0.0.133 (executor driver) (73/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 73.0 in stage 15.0 (TID 407). 34476 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 76.0 in stage 15.0 (TID 410) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 76.0 in stage 15.0 (TID 410) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 73.0 in stage 15.0 (TID 407) in 4 ms on 10.0.0.133 (executor driver) (74/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 74.0 in stage 15.0 (TID 408). 33908 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 75.0 in stage 15.0 (TID 409). 34266 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 77.0 in stage 15.0 (TID 411) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 77.0 in stage 15.0 (TID 411) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 78.0 in stage 15.0 (TID 412) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 75.0 in stage 15.0 (TID 409) in 4 ms on 10.0.0.133 (executor driver) (75/200) -26/04/01 08:43:29 INFO Executor: Running task 78.0 in stage 15.0 (TID 412) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 74.0 in stage 15.0 (TID 408) in 5 ms on 10.0.0.133 (executor driver) (76/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 76.0 in stage 15.0 (TID 410). 34196 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 79.0 in stage 15.0 (TID 413) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 79.0 in stage 15.0 (TID 413) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 76.0 in stage 15.0 (TID 410) in 4 ms on 10.0.0.133 (executor driver) (77/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 78.0 in stage 15.0 (TID 412). 34141 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 77.0 in stage 15.0 (TID 411). 34812 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 80.0 in stage 15.0 (TID 414) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 80.0 in stage 15.0 (TID 414) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 81.0 in stage 15.0 (TID 415) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 77.0 in stage 15.0 (TID 411) in 4 ms on 10.0.0.133 (executor driver) (78/200) -26/04/01 08:43:29 INFO Executor: Running task 81.0 in stage 15.0 (TID 415) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 78.0 in stage 15.0 (TID 412) in 4 ms on 10.0.0.133 (executor driver) (79/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.6 KiB) non-empty blocks including 8 (31.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 79.0 in stage 15.0 (TID 413). 34085 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 82.0 in stage 15.0 (TID 416) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 82.0 in stage 15.0 (TID 416) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 79.0 in stage 15.0 (TID 413) in 4 ms on 10.0.0.133 (executor driver) (80/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 80.0 in stage 15.0 (TID 414). 34593 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 83.0 in stage 15.0 (TID 417) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 83.0 in stage 15.0 (TID 417) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 80.0 in stage 15.0 (TID 414) in 4 ms on 10.0.0.133 (executor driver) (81/200) -26/04/01 08:43:29 INFO Executor: Finished task 81.0 in stage 15.0 (TID 415). 33935 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 84.0 in stage 15.0 (TID 418) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 84.0 in stage 15.0 (TID 418) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 81.0 in stage 15.0 (TID 415) in 4 ms on 10.0.0.133 (executor driver) (82/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 82.0 in stage 15.0 (TID 416). 34466 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 85.0 in stage 15.0 (TID 419) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 85.0 in stage 15.0 (TID 419) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 82.0 in stage 15.0 (TID 416) in 4 ms on 10.0.0.133 (executor driver) (83/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 83.0 in stage 15.0 (TID 417). 34273 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 84.0 in stage 15.0 (TID 418). 33511 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 86.0 in stage 15.0 (TID 420) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 86.0 in stage 15.0 (TID 420) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 87.0 in stage 15.0 (TID 421) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 83.0 in stage 15.0 (TID 417) in 5 ms on 10.0.0.133 (executor driver) (84/200) -26/04/01 08:43:29 INFO Executor: Running task 87.0 in stage 15.0 (TID 421) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 84.0 in stage 15.0 (TID 418) in 4 ms on 10.0.0.133 (executor driver) (85/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 85.0 in stage 15.0 (TID 419). 34506 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 88.0 in stage 15.0 (TID 422) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 88.0 in stage 15.0 (TID 422) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 85.0 in stage 15.0 (TID 419) in 3 ms on 10.0.0.133 (executor driver) (86/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 86.0 in stage 15.0 (TID 420). 34439 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 89.0 in stage 15.0 (TID 423) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 89.0 in stage 15.0 (TID 423) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 86.0 in stage 15.0 (TID 420) in 6 ms on 10.0.0.133 (executor driver) (87/200) -26/04/01 08:43:29 INFO Executor: Finished task 87.0 in stage 15.0 (TID 421). 33817 bytes result sent to driver -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Starting task 90.0 in stage 15.0 (TID 424) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 90.0 in stage 15.0 (TID 424) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 87.0 in stage 15.0 (TID 421) in 7 ms on 10.0.0.133 (executor driver) (88/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 88.0 in stage 15.0 (TID 422). 34831 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 91.0 in stage 15.0 (TID 425) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 91.0 in stage 15.0 (TID 425) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 88.0 in stage 15.0 (TID 422) in 7 ms on 10.0.0.133 (executor driver) (89/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 89.0 in stage 15.0 (TID 423). 33927 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 92.0 in stage 15.0 (TID 426) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 92.0 in stage 15.0 (TID 426) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 89.0 in stage 15.0 (TID 423) in 7 ms on 10.0.0.133 (executor driver) (90/200) -26/04/01 08:43:29 INFO Executor: Finished task 90.0 in stage 15.0 (TID 424). 33756 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 93.0 in stage 15.0 (TID 427) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Running task 93.0 in stage 15.0 (TID 427) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 90.0 in stage 15.0 (TID 424) in 4 ms on 10.0.0.133 (executor driver) (91/200) -26/04/01 08:43:29 INFO Executor: Finished task 91.0 in stage 15.0 (TID 425). 33835 bytes result sent to driver -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO TaskSetManager: Starting task 94.0 in stage 15.0 (TID 428) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Finished task 91.0 in stage 15.0 (TID 425) in 4 ms on 10.0.0.133 (executor driver) (92/200) -26/04/01 08:43:29 INFO Executor: Running task 94.0 in stage 15.0 (TID 428) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 92.0 in stage 15.0 (TID 426). 33953 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 95.0 in stage 15.0 (TID 429) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 95.0 in stage 15.0 (TID 429) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 92.0 in stage 15.0 (TID 426) in 4 ms on 10.0.0.133 (executor driver) (93/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 93.0 in stage 15.0 (TID 427). 33995 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 96.0 in stage 15.0 (TID 430) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 96.0 in stage 15.0 (TID 430) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 93.0 in stage 15.0 (TID 427) in 4 ms on 10.0.0.133 (executor driver) (94/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO Executor: Finished task 94.0 in stage 15.0 (TID 428). 34551 bytes result sent to driver -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Starting task 97.0 in stage 15.0 (TID 431) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 97.0 in stage 15.0 (TID 431) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 94.0 in stage 15.0 (TID 428) in 5 ms on 10.0.0.133 (executor driver) (95/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 95.0 in stage 15.0 (TID 429). 33891 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 98.0 in stage 15.0 (TID 432) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 98.0 in stage 15.0 (TID 432) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 95.0 in stage 15.0 (TID 429) in 4 ms on 10.0.0.133 (executor driver) (96/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 96.0 in stage 15.0 (TID 430). 34205 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 99.0 in stage 15.0 (TID 433) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 99.0 in stage 15.0 (TID 433) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 96.0 in stage 15.0 (TID 430) in 4 ms on 10.0.0.133 (executor driver) (97/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO Executor: Finished task 97.0 in stage 15.0 (TID 431). 33883 bytes result sent to driver -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Starting task 100.0 in stage 15.0 (TID 434) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 100.0 in stage 15.0 (TID 434) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 97.0 in stage 15.0 (TID 431) in 5 ms on 10.0.0.133 (executor driver) (98/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 98.0 in stage 15.0 (TID 432). 34607 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 101.0 in stage 15.0 (TID 435) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 101.0 in stage 15.0 (TID 435) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 98.0 in stage 15.0 (TID 432) in 3 ms on 10.0.0.133 (executor driver) (99/200) -26/04/01 08:43:29 INFO Executor: Finished task 99.0 in stage 15.0 (TID 433). 34229 bytes result sent to driver -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Starting task 102.0 in stage 15.0 (TID 436) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 102.0 in stage 15.0 (TID 436) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 99.0 in stage 15.0 (TID 433) in 3 ms on 10.0.0.133 (executor driver) (100/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 100.0 in stage 15.0 (TID 434). 34078 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 103.0 in stage 15.0 (TID 437) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 103.0 in stage 15.0 (TID 437) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 100.0 in stage 15.0 (TID 434) in 4 ms on 10.0.0.133 (executor driver) (101/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 101.0 in stage 15.0 (TID 435). 33625 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 104.0 in stage 15.0 (TID 438) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 104.0 in stage 15.0 (TID 438) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 101.0 in stage 15.0 (TID 435) in 4 ms on 10.0.0.133 (executor driver) (102/200) -26/04/01 08:43:29 INFO Executor: Finished task 102.0 in stage 15.0 (TID 436). 34103 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 105.0 in stage 15.0 (TID 439) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 105.0 in stage 15.0 (TID 439) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 102.0 in stage 15.0 (TID 436) in 3 ms on 10.0.0.133 (executor driver) (103/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 103.0 in stage 15.0 (TID 437). 34407 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 106.0 in stage 15.0 (TID 440) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 106.0 in stage 15.0 (TID 440) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 103.0 in stage 15.0 (TID 437) in 3 ms on 10.0.0.133 (executor driver) (104/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 104.0 in stage 15.0 (TID 438). 34155 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 107.0 in stage 15.0 (TID 441) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 107.0 in stage 15.0 (TID 441) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 104.0 in stage 15.0 (TID 438) in 4 ms on 10.0.0.133 (executor driver) (105/200) -26/04/01 08:43:29 INFO Executor: Finished task 105.0 in stage 15.0 (TID 439). 33868 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 108.0 in stage 15.0 (TID 442) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO Executor: Running task 108.0 in stage 15.0 (TID 442) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 106.0 in stage 15.0 (TID 440). 33851 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Finished task 105.0 in stage 15.0 (TID 439) in 5 ms on 10.0.0.133 (executor driver) (106/200) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 109.0 in stage 15.0 (TID 443) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 109.0 in stage 15.0 (TID 443) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 106.0 in stage 15.0 (TID 440) in 4 ms on 10.0.0.133 (executor driver) (107/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 107.0 in stage 15.0 (TID 441). 33898 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 110.0 in stage 15.0 (TID 444) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Finished task 108.0 in stage 15.0 (TID 442). 34303 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Running task 110.0 in stage 15.0 (TID 444) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 107.0 in stage 15.0 (TID 441) in 4 ms on 10.0.0.133 (executor driver) (108/200) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 111.0 in stage 15.0 (TID 445) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 111.0 in stage 15.0 (TID 445) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 108.0 in stage 15.0 (TID 442) in 3 ms on 10.0.0.133 (executor driver) (109/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO Executor: Finished task 109.0 in stage 15.0 (TID 443). 34424 bytes result sent to driver -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Starting task 112.0 in stage 15.0 (TID 446) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 112.0 in stage 15.0 (TID 446) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 109.0 in stage 15.0 (TID 443) in 4 ms on 10.0.0.133 (executor driver) (110/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 110.0 in stage 15.0 (TID 444). 34141 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 113.0 in stage 15.0 (TID 447) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 113.0 in stage 15.0 (TID 447) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 110.0 in stage 15.0 (TID 444) in 4 ms on 10.0.0.133 (executor driver) (111/200) -26/04/01 08:43:29 INFO Executor: Finished task 111.0 in stage 15.0 (TID 445). 34432 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 114.0 in stage 15.0 (TID 448) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 114.0 in stage 15.0 (TID 448) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 111.0 in stage 15.0 (TID 445) in 4 ms on 10.0.0.133 (executor driver) (112/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 112.0 in stage 15.0 (TID 446). 34551 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 115.0 in stage 15.0 (TID 449) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 115.0 in stage 15.0 (TID 449) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 112.0 in stage 15.0 (TID 446) in 4 ms on 10.0.0.133 (executor driver) (113/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 113.0 in stage 15.0 (TID 447). 34235 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 116.0 in stage 15.0 (TID 450) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 116.0 in stage 15.0 (TID 450) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 113.0 in stage 15.0 (TID 447) in 4 ms on 10.0.0.133 (executor driver) (114/200) -26/04/01 08:43:29 INFO Executor: Finished task 114.0 in stage 15.0 (TID 448). 34058 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 117.0 in stage 15.0 (TID 451) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Finished task 115.0 in stage 15.0 (TID 449). 33531 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Running task 117.0 in stage 15.0 (TID 451) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 118.0 in stage 15.0 (TID 452) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Finished task 114.0 in stage 15.0 (TID 448) in 5 ms on 10.0.0.133 (executor driver) (115/200) -26/04/01 08:43:29 INFO Executor: Running task 118.0 in stage 15.0 (TID 452) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 115.0 in stage 15.0 (TID 449) in 4 ms on 10.0.0.133 (executor driver) (116/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 116.0 in stage 15.0 (TID 450). 34105 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 119.0 in stage 15.0 (TID 453) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 119.0 in stage 15.0 (TID 453) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 116.0 in stage 15.0 (TID 450) in 4 ms on 10.0.0.133 (executor driver) (117/200) -26/04/01 08:43:29 INFO Executor: Finished task 117.0 in stage 15.0 (TID 451). 34192 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 120.0 in stage 15.0 (TID 454) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 120.0 in stage 15.0 (TID 454) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO TaskSetManager: Finished task 117.0 in stage 15.0 (TID 451) in 5 ms on 10.0.0.133 (executor driver) (118/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 118.0 in stage 15.0 (TID 452). 34647 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 121.0 in stage 15.0 (TID 455) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 121.0 in stage 15.0 (TID 455) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 118.0 in stage 15.0 (TID 452) in 5 ms on 10.0.0.133 (executor driver) (119/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 119.0 in stage 15.0 (TID 453). 33642 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 122.0 in stage 15.0 (TID 456) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 122.0 in stage 15.0 (TID 456) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 119.0 in stage 15.0 (TID 453) in 4 ms on 10.0.0.133 (executor driver) (120/200) -26/04/01 08:43:29 INFO Executor: Finished task 120.0 in stage 15.0 (TID 454). 33991 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 121.0 in stage 15.0 (TID 455). 34107 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 123.0 in stage 15.0 (TID 457) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 123.0 in stage 15.0 (TID 457) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 124.0 in stage 15.0 (TID 458) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 120.0 in stage 15.0 (TID 454) in 5 ms on 10.0.0.133 (executor driver) (121/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO Executor: Running task 124.0 in stage 15.0 (TID 458) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Finished task 121.0 in stage 15.0 (TID 455) in 4 ms on 10.0.0.133 (executor driver) (122/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 122.0 in stage 15.0 (TID 456). 34345 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 125.0 in stage 15.0 (TID 459) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 125.0 in stage 15.0 (TID 459) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 122.0 in stage 15.0 (TID 456) in 4 ms on 10.0.0.133 (executor driver) (123/200) -26/04/01 08:43:29 INFO Executor: Finished task 124.0 in stage 15.0 (TID 458). 34152 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 123.0 in stage 15.0 (TID 457). 34123 bytes result sent to driver -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Starting task 126.0 in stage 15.0 (TID 460) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 126.0 in stage 15.0 (TID 460) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 127.0 in stage 15.0 (TID 461) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 124.0 in stage 15.0 (TID 458) in 4 ms on 10.0.0.133 (executor driver) (124/200) -26/04/01 08:43:29 INFO Executor: Running task 127.0 in stage 15.0 (TID 461) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 123.0 in stage 15.0 (TID 457) in 5 ms on 10.0.0.133 (executor driver) (125/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 125.0 in stage 15.0 (TID 459). 34554 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 128.0 in stage 15.0 (TID 462) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 128.0 in stage 15.0 (TID 462) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 125.0 in stage 15.0 (TID 459) in 4 ms on 10.0.0.133 (executor driver) (126/200) -26/04/01 08:43:29 INFO Executor: Finished task 126.0 in stage 15.0 (TID 460). 33922 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 129.0 in stage 15.0 (TID 463) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 129.0 in stage 15.0 (TID 463) -26/04/01 08:43:29 INFO Executor: Finished task 127.0 in stage 15.0 (TID 461). 34650 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Finished task 126.0 in stage 15.0 (TID 460) in 4 ms on 10.0.0.133 (executor driver) (127/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Starting task 130.0 in stage 15.0 (TID 464) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 130.0 in stage 15.0 (TID 464) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 127.0 in stage 15.0 (TID 461) in 4 ms on 10.0.0.133 (executor driver) (128/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.6 KiB) non-empty blocks including 8 (31.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 128.0 in stage 15.0 (TID 462). 34392 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 131.0 in stage 15.0 (TID 465) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 128.0 in stage 15.0 (TID 462) in 7 ms on 10.0.0.133 (executor driver) (129/200) -26/04/01 08:43:29 INFO Executor: Running task 131.0 in stage 15.0 (TID 465) -26/04/01 08:43:29 INFO Executor: Finished task 130.0 in stage 15.0 (TID 464). 35133 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 132.0 in stage 15.0 (TID 466) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Finished task 129.0 in stage 15.0 (TID 463). 34055 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Finished task 130.0 in stage 15.0 (TID 464) in 6 ms on 10.0.0.133 (executor driver) (130/200) -26/04/01 08:43:29 INFO Executor: Running task 132.0 in stage 15.0 (TID 466) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 133.0 in stage 15.0 (TID 467) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 133.0 in stage 15.0 (TID 467) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 129.0 in stage 15.0 (TID 463) in 6 ms on 10.0.0.133 (executor driver) (131/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 131.0 in stage 15.0 (TID 465). 33973 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 134.0 in stage 15.0 (TID 468) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 134.0 in stage 15.0 (TID 468) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 131.0 in stage 15.0 (TID 465) in 4 ms on 10.0.0.133 (executor driver) (132/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 132.0 in stage 15.0 (TID 466). 34084 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 133.0 in stage 15.0 (TID 467). 33764 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 135.0 in stage 15.0 (TID 469) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 135.0 in stage 15.0 (TID 469) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 136.0 in stage 15.0 (TID 470) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 132.0 in stage 15.0 (TID 466) in 5 ms on 10.0.0.133 (executor driver) (133/200) -26/04/01 08:43:29 INFO Executor: Running task 136.0 in stage 15.0 (TID 470) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 133.0 in stage 15.0 (TID 467) in 5 ms on 10.0.0.133 (executor driver) (134/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 134.0 in stage 15.0 (TID 468). 33712 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 137.0 in stage 15.0 (TID 471) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 137.0 in stage 15.0 (TID 471) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 134.0 in stage 15.0 (TID 468) in 4 ms on 10.0.0.133 (executor driver) (135/200) -26/04/01 08:43:29 INFO Executor: Finished task 135.0 in stage 15.0 (TID 469). 34404 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 138.0 in stage 15.0 (TID 472) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 138.0 in stage 15.0 (TID 472) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 135.0 in stage 15.0 (TID 469) in 3 ms on 10.0.0.133 (executor driver) (136/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 136.0 in stage 15.0 (TID 470). 34430 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 139.0 in stage 15.0 (TID 473) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 139.0 in stage 15.0 (TID 473) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 136.0 in stage 15.0 (TID 470) in 4 ms on 10.0.0.133 (executor driver) (137/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 137.0 in stage 15.0 (TID 471). 34756 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 140.0 in stage 15.0 (TID 474) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 140.0 in stage 15.0 (TID 474) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 137.0 in stage 15.0 (TID 471) in 5 ms on 10.0.0.133 (executor driver) (138/200) -26/04/01 08:43:29 INFO Executor: Finished task 138.0 in stage 15.0 (TID 472). 33693 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 139.0 in stage 15.0 (TID 473). 34378 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 141.0 in stage 15.0 (TID 475) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 141.0 in stage 15.0 (TID 475) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 142.0 in stage 15.0 (TID 476) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 138.0 in stage 15.0 (TID 472) in 4 ms on 10.0.0.133 (executor driver) (139/200) -26/04/01 08:43:29 INFO Executor: Running task 142.0 in stage 15.0 (TID 476) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 139.0 in stage 15.0 (TID 473) in 4 ms on 10.0.0.133 (executor driver) (140/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 140.0 in stage 15.0 (TID 474). 33890 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 143.0 in stage 15.0 (TID 477) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 143.0 in stage 15.0 (TID 477) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 140.0 in stage 15.0 (TID 474) in 5 ms on 10.0.0.133 (executor driver) (141/200) -26/04/01 08:43:29 INFO Executor: Finished task 141.0 in stage 15.0 (TID 475). 34328 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 142.0 in stage 15.0 (TID 476). 34951 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 144.0 in stage 15.0 (TID 478) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO Executor: Running task 144.0 in stage 15.0 (TID 478) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Starting task 145.0 in stage 15.0 (TID 479) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 141.0 in stage 15.0 (TID 475) in 4 ms on 10.0.0.133 (executor driver) (142/200) -26/04/01 08:43:29 INFO Executor: Running task 145.0 in stage 15.0 (TID 479) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 142.0 in stage 15.0 (TID 476) in 4 ms on 10.0.0.133 (executor driver) (143/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 144.0 in stage 15.0 (TID 478). 33655 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 143.0 in stage 15.0 (TID 477). 34375 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 146.0 in stage 15.0 (TID 480) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 146.0 in stage 15.0 (TID 480) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 147.0 in stage 15.0 (TID 481) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 144.0 in stage 15.0 (TID 478) in 4 ms on 10.0.0.133 (executor driver) (144/200) -26/04/01 08:43:29 INFO Executor: Running task 147.0 in stage 15.0 (TID 481) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 143.0 in stage 15.0 (TID 477) in 5 ms on 10.0.0.133 (executor driver) (145/200) -26/04/01 08:43:29 INFO Executor: Finished task 145.0 in stage 15.0 (TID 479). 34189 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 148.0 in stage 15.0 (TID 482) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO Executor: Running task 148.0 in stage 15.0 (TID 482) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 145.0 in stage 15.0 (TID 479) in 4 ms on 10.0.0.133 (executor driver) (146/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 148.0 in stage 15.0 (TID 482). 34244 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 149.0 in stage 15.0 (TID 483) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Finished task 146.0 in stage 15.0 (TID 480). 33726 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 147.0 in stage 15.0 (TID 481). 33597 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Running task 149.0 in stage 15.0 (TID 483) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 148.0 in stage 15.0 (TID 482) in 4 ms on 10.0.0.133 (executor driver) (147/200) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 150.0 in stage 15.0 (TID 484) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 150.0 in stage 15.0 (TID 484) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 151.0 in stage 15.0 (TID 485) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 146.0 in stage 15.0 (TID 480) in 4 ms on 10.0.0.133 (executor driver) (148/200) -26/04/01 08:43:29 INFO Executor: Running task 151.0 in stage 15.0 (TID 485) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 147.0 in stage 15.0 (TID 481) in 4 ms on 10.0.0.133 (executor driver) (149/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 151.0 in stage 15.0 (TID 485). 34392 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 150.0 in stage 15.0 (TID 484). 34597 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 149.0 in stage 15.0 (TID 483). 34799 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 152.0 in stage 15.0 (TID 486) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 152.0 in stage 15.0 (TID 486) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 153.0 in stage 15.0 (TID 487) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 151.0 in stage 15.0 (TID 485) in 4 ms on 10.0.0.133 (executor driver) (150/200) -26/04/01 08:43:29 INFO Executor: Running task 153.0 in stage 15.0 (TID 487) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 154.0 in stage 15.0 (TID 488) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 150.0 in stage 15.0 (TID 484) in 5 ms on 10.0.0.133 (executor driver) (151/200) -26/04/01 08:43:29 INFO Executor: Running task 154.0 in stage 15.0 (TID 488) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 149.0 in stage 15.0 (TID 483) in 5 ms on 10.0.0.133 (executor driver) (152/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 154.0 in stage 15.0 (TID 488). 33726 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 152.0 in stage 15.0 (TID 486). 34441 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 155.0 in stage 15.0 (TID 489) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Finished task 153.0 in stage 15.0 (TID 487). 34127 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Running task 155.0 in stage 15.0 (TID 489) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 156.0 in stage 15.0 (TID 490) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 154.0 in stage 15.0 (TID 488) in 4 ms on 10.0.0.133 (executor driver) (153/200) -26/04/01 08:43:29 INFO Executor: Running task 156.0 in stage 15.0 (TID 490) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 152.0 in stage 15.0 (TID 486) in 5 ms on 10.0.0.133 (executor driver) (154/200) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 157.0 in stage 15.0 (TID 491) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 157.0 in stage 15.0 (TID 491) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 153.0 in stage 15.0 (TID 487) in 5 ms on 10.0.0.133 (executor driver) (155/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 155.0 in stage 15.0 (TID 489). 33738 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 158.0 in stage 15.0 (TID 492) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 158.0 in stage 15.0 (TID 492) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 155.0 in stage 15.0 (TID 489) in 4 ms on 10.0.0.133 (executor driver) (156/200) -26/04/01 08:43:29 INFO Executor: Finished task 157.0 in stage 15.0 (TID 491). 33697 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 156.0 in stage 15.0 (TID 490). 34636 bytes result sent to driver -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO TaskSetManager: Starting task 159.0 in stage 15.0 (TID 493) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 159.0 in stage 15.0 (TID 493) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 160.0 in stage 15.0 (TID 494) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 157.0 in stage 15.0 (TID 491) in 4 ms on 10.0.0.133 (executor driver) (157/200) -26/04/01 08:43:29 INFO Executor: Running task 160.0 in stage 15.0 (TID 494) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 156.0 in stage 15.0 (TID 490) in 4 ms on 10.0.0.133 (executor driver) (158/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 158.0 in stage 15.0 (TID 492). 34759 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 161.0 in stage 15.0 (TID 495) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 161.0 in stage 15.0 (TID 495) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 158.0 in stage 15.0 (TID 492) in 4 ms on 10.0.0.133 (executor driver) (159/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 159.0 in stage 15.0 (TID 493). 34519 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 160.0 in stage 15.0 (TID 494). 34867 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 162.0 in stage 15.0 (TID 496) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 162.0 in stage 15.0 (TID 496) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 163.0 in stage 15.0 (TID 497) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 159.0 in stage 15.0 (TID 493) in 5 ms on 10.0.0.133 (executor driver) (160/200) -26/04/01 08:43:29 INFO Executor: Running task 163.0 in stage 15.0 (TID 497) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 160.0 in stage 15.0 (TID 494) in 5 ms on 10.0.0.133 (executor driver) (161/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 161.0 in stage 15.0 (TID 495). 33835 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 164.0 in stage 15.0 (TID 498) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 164.0 in stage 15.0 (TID 498) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 161.0 in stage 15.0 (TID 495) in 5 ms on 10.0.0.133 (executor driver) (162/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 163.0 in stage 15.0 (TID 497). 34082 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 162.0 in stage 15.0 (TID 496). 34295 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 165.0 in stage 15.0 (TID 499) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 165.0 in stage 15.0 (TID 499) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 166.0 in stage 15.0 (TID 500) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 163.0 in stage 15.0 (TID 497) in 4 ms on 10.0.0.133 (executor driver) (163/200) -26/04/01 08:43:29 INFO Executor: Running task 166.0 in stage 15.0 (TID 500) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 162.0 in stage 15.0 (TID 496) in 4 ms on 10.0.0.133 (executor driver) (164/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 164.0 in stage 15.0 (TID 498). 33851 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 167.0 in stage 15.0 (TID 501) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 167.0 in stage 15.0 (TID 501) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 164.0 in stage 15.0 (TID 498) in 4 ms on 10.0.0.133 (executor driver) (165/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 165.0 in stage 15.0 (TID 499). 34396 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 166.0 in stage 15.0 (TID 500). 34848 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 168.0 in stage 15.0 (TID 502) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 168.0 in stage 15.0 (TID 502) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 169.0 in stage 15.0 (TID 503) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 165.0 in stage 15.0 (TID 499) in 4 ms on 10.0.0.133 (executor driver) (166/200) -26/04/01 08:43:29 INFO Executor: Running task 169.0 in stage 15.0 (TID 503) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 166.0 in stage 15.0 (TID 500) in 4 ms on 10.0.0.133 (executor driver) (167/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 167.0 in stage 15.0 (TID 501). 33887 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 170.0 in stage 15.0 (TID 504) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 170.0 in stage 15.0 (TID 504) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 167.0 in stage 15.0 (TID 501) in 4 ms on 10.0.0.133 (executor driver) (168/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 169.0 in stage 15.0 (TID 503). 34471 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 171.0 in stage 15.0 (TID 505) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 171.0 in stage 15.0 (TID 505) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 169.0 in stage 15.0 (TID 503) in 4 ms on 10.0.0.133 (executor driver) (169/200) -26/04/01 08:43:29 INFO Executor: Finished task 168.0 in stage 15.0 (TID 502). 33588 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 172.0 in stage 15.0 (TID 506) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 172.0 in stage 15.0 (TID 506) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 168.0 in stage 15.0 (TID 502) in 4 ms on 10.0.0.133 (executor driver) (170/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 170.0 in stage 15.0 (TID 504). 34026 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 173.0 in stage 15.0 (TID 507) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 170.0 in stage 15.0 (TID 504) in 4 ms on 10.0.0.133 (executor driver) (171/200) -26/04/01 08:43:29 INFO Executor: Running task 173.0 in stage 15.0 (TID 507) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 171.0 in stage 15.0 (TID 505). 34142 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 174.0 in stage 15.0 (TID 508) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Finished task 172.0 in stage 15.0 (TID 506). 33962 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Running task 174.0 in stage 15.0 (TID 508) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 171.0 in stage 15.0 (TID 505) in 6 ms on 10.0.0.133 (executor driver) (172/200) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 175.0 in stage 15.0 (TID 509) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 172.0 in stage 15.0 (TID 506) in 7 ms on 10.0.0.133 (executor driver) (173/200) -26/04/01 08:43:29 INFO Executor: Running task 175.0 in stage 15.0 (TID 509) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 173.0 in stage 15.0 (TID 507). 34609 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 176.0 in stage 15.0 (TID 510) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 176.0 in stage 15.0 (TID 510) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 173.0 in stage 15.0 (TID 507) in 6 ms on 10.0.0.133 (executor driver) (174/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 174.0 in stage 15.0 (TID 508). 33575 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 177.0 in stage 15.0 (TID 511) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 174.0 in stage 15.0 (TID 508) in 4 ms on 10.0.0.133 (executor driver) (175/200) -26/04/01 08:43:29 INFO Executor: Running task 177.0 in stage 15.0 (TID 511) -26/04/01 08:43:29 INFO Executor: Finished task 175.0 in stage 15.0 (TID 509). 34342 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 178.0 in stage 15.0 (TID 512) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 178.0 in stage 15.0 (TID 512) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 175.0 in stage 15.0 (TID 509) in 3 ms on 10.0.0.133 (executor driver) (176/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 176.0 in stage 15.0 (TID 510). 33675 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 179.0 in stage 15.0 (TID 513) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 179.0 in stage 15.0 (TID 513) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 176.0 in stage 15.0 (TID 510) in 4 ms on 10.0.0.133 (executor driver) (177/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 178.0 in stage 15.0 (TID 512). 34255 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 177.0 in stage 15.0 (TID 511). 34399 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 180.0 in stage 15.0 (TID 514) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 180.0 in stage 15.0 (TID 514) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 181.0 in stage 15.0 (TID 515) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 178.0 in stage 15.0 (TID 512) in 4 ms on 10.0.0.133 (executor driver) (178/200) -26/04/01 08:43:29 INFO Executor: Running task 181.0 in stage 15.0 (TID 515) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 177.0 in stage 15.0 (TID 511) in 4 ms on 10.0.0.133 (executor driver) (179/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 179.0 in stage 15.0 (TID 513). 34030 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 182.0 in stage 15.0 (TID 516) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 182.0 in stage 15.0 (TID 516) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 179.0 in stage 15.0 (TID 513) in 4 ms on 10.0.0.133 (executor driver) (180/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 180.0 in stage 15.0 (TID 514). 34531 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 181.0 in stage 15.0 (TID 515). 34153 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 183.0 in stage 15.0 (TID 517) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 183.0 in stage 15.0 (TID 517) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 184.0 in stage 15.0 (TID 518) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 180.0 in stage 15.0 (TID 514) in 4 ms on 10.0.0.133 (executor driver) (181/200) -26/04/01 08:43:29 INFO Executor: Running task 184.0 in stage 15.0 (TID 518) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 181.0 in stage 15.0 (TID 515) in 4 ms on 10.0.0.133 (executor driver) (182/200) -26/04/01 08:43:29 INFO Executor: Finished task 182.0 in stage 15.0 (TID 516). 34252 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 185.0 in stage 15.0 (TID 519) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 185.0 in stage 15.0 (TID 519) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 182.0 in stage 15.0 (TID 516) in 4 ms on 10.0.0.133 (executor driver) (183/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (31.6 KiB) non-empty blocks including 8 (31.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 183.0 in stage 15.0 (TID 517). 34150 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 186.0 in stage 15.0 (TID 520) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Finished task 184.0 in stage 15.0 (TID 518). 34956 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Running task 186.0 in stage 15.0 (TID 520) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 187.0 in stage 15.0 (TID 521) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Finished task 185.0 in stage 15.0 (TID 519). 34506 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Finished task 183.0 in stage 15.0 (TID 517) in 4 ms on 10.0.0.133 (executor driver) (184/200) -26/04/01 08:43:29 INFO Executor: Running task 187.0 in stage 15.0 (TID 521) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 184.0 in stage 15.0 (TID 518) in 4 ms on 10.0.0.133 (executor driver) (185/200) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 188.0 in stage 15.0 (TID 522) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 188.0 in stage 15.0 (TID 522) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 185.0 in stage 15.0 (TID 519) in 4 ms on 10.0.0.133 (executor driver) (186/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 186.0 in stage 15.0 (TID 520). 33801 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 189.0 in stage 15.0 (TID 523) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 189.0 in stage 15.0 (TID 523) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 186.0 in stage 15.0 (TID 520) in 4 ms on 10.0.0.133 (executor driver) (187/200) -26/04/01 08:43:29 INFO Executor: Finished task 188.0 in stage 15.0 (TID 522). 34848 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 190.0 in stage 15.0 (TID 524) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 190.0 in stage 15.0 (TID 524) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 188.0 in stage 15.0 (TID 522) in 4 ms on 10.0.0.133 (executor driver) (188/200) -26/04/01 08:43:29 INFO Executor: Finished task 187.0 in stage 15.0 (TID 521). 34650 bytes result sent to driver -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO TaskSetManager: Starting task 191.0 in stage 15.0 (TID 525) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Running task 191.0 in stage 15.0 (TID 525) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 187.0 in stage 15.0 (TID 521) in 5 ms on 10.0.0.133 (executor driver) (189/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 189.0 in stage 15.0 (TID 523). 34333 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 192.0 in stage 15.0 (TID 526) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Finished task 190.0 in stage 15.0 (TID 524). 34397 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 193.0 in stage 15.0 (TID 527) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 192.0 in stage 15.0 (TID 526) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 189.0 in stage 15.0 (TID 523) in 4 ms on 10.0.0.133 (executor driver) (190/200) -26/04/01 08:43:29 INFO Executor: Running task 193.0 in stage 15.0 (TID 527) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 190.0 in stage 15.0 (TID 524) in 4 ms on 10.0.0.133 (executor driver) (191/200) -26/04/01 08:43:29 INFO Executor: Finished task 191.0 in stage 15.0 (TID 525). 34559 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 194.0 in stage 15.0 (TID 528) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 194.0 in stage 15.0 (TID 528) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 191.0 in stage 15.0 (TID 525) in 4 ms on 10.0.0.133 (executor driver) (192/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 194.0 in stage 15.0 (TID 528). 34348 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 193.0 in stage 15.0 (TID 527). 34499 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 192.0 in stage 15.0 (TID 526). 33410 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 195.0 in stage 15.0 (TID 529) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 195.0 in stage 15.0 (TID 529) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 196.0 in stage 15.0 (TID 530) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 194.0 in stage 15.0 (TID 528) in 5 ms on 10.0.0.133 (executor driver) (193/200) -26/04/01 08:43:29 INFO Executor: Running task 196.0 in stage 15.0 (TID 530) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 193.0 in stage 15.0 (TID 527) in 5 ms on 10.0.0.133 (executor driver) (194/200) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 197.0 in stage 15.0 (TID 531) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 197.0 in stage 15.0 (TID 531) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 192.0 in stage 15.0 (TID 526) in 5 ms on 10.0.0.133 (executor driver) (195/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 195.0 in stage 15.0 (TID 529). 34725 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 198.0 in stage 15.0 (TID 532) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 198.0 in stage 15.0 (TID 532) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 195.0 in stage 15.0 (TID 529) in 4 ms on 10.0.0.133 (executor driver) (196/200) -26/04/01 08:43:29 INFO Executor: Finished task 196.0 in stage 15.0 (TID 530). 34709 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 197.0 in stage 15.0 (TID 531). 34348 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 199.0 in stage 15.0 (TID 533) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9215 bytes) -26/04/01 08:43:29 INFO Executor: Running task 199.0 in stage 15.0 (TID 533) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 196.0 in stage 15.0 (TID 530) in 5 ms on 10.0.0.133 (executor driver) (197/200) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 197.0 in stage 15.0 (TID 531) in 4 ms on 10.0.0.133 (executor driver) (198/200) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO Executor: Finished task 198.0 in stage 15.0 (TID 532). 34063 bytes result sent to driver -26/04/01 08:43:29 INFO Executor: Finished task 199.0 in stage 15.0 (TID 533). 34766 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Finished task 198.0 in stage 15.0 (TID 532) in 4 ms on 10.0.0.133 (executor driver) (199/200) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 199.0 in stage 15.0 (TID 533) in 3 ms on 10.0.0.133 (executor driver) (200/200) -26/04/01 08:43:29 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool -26/04/01 08:43:29 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 231.012 s -26/04/01 08:43:29 INFO DAGScheduler: Job 14 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:43:29 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished -26/04/01 08:43:29 INFO DAGScheduler: Job 14 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 231.027845 s -26/04/01 08:43:29 INFO Utils: Coalesced 1600 broadcast batches into 1 (1088125 rows) -26/04/01 08:43:29 INFO MemoryStore: Block broadcast_22 stored as values in memory (estimated size 5.2 MiB, free 8.4 GiB) -26/04/01 08:43:29 INFO MemoryStore: Block broadcast_22_piece0 stored as bytes in memory (estimated size 4.0 MiB, free 8.4 GiB) -26/04/01 08:43:29 INFO BlockManagerInfo: Added broadcast_22_piece0 in memory on 10.0.0.133:59303 (size: 4.0 MiB, free: 8.6 GiB) -26/04/01 08:43:29 INFO MemoryStore: Block broadcast_22_piece1 stored as bytes in memory (estimated size 357.8 KiB, free 8.4 GiB) -26/04/01 08:43:29 INFO BlockManagerInfo: Added broadcast_22_piece1 in memory on 10.0.0.133:59303 (size: 357.8 KiB, free: 8.6 GiB) -26/04/01 08:43:29 INFO SparkContext: Created broadcast 22 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:43:29 INFO ShufflePartitionsUtil: For shuffle(1), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 08:43:29 INFO DAGScheduler: Registering RDD 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 5 -26/04/01 08:43:29 INFO DAGScheduler: Got map stage job 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions -26/04/01 08:43:29 INFO DAGScheduler: Final stage: ShuffleMapStage 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:43:29 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 16) -26/04/01 08:43:29 INFO DAGScheduler: Missing parents: List() -26/04/01 08:43:29 INFO DAGScheduler: Submitting ShuffleMapStage 17 (MapPartitionsRDD[39] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:43:29 INFO MemoryStore: Block broadcast_23 stored as values in memory (estimated size 21.7 KiB, free 8.4 GiB) -26/04/01 08:43:29 INFO MemoryStore: Block broadcast_23_piece0 stored as bytes in memory (estimated size 11.0 KiB, free 8.4 GiB) -26/04/01 08:43:29 INFO BlockManagerInfo: Added broadcast_23_piece0 in memory on 10.0.0.133:59303 (size: 11.0 KiB, free: 8.6 GiB) -26/04/01 08:43:29 INFO SparkContext: Created broadcast 23 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:43:29 INFO DAGScheduler: Submitting 200 missing tasks from ShuffleMapStage 17 (MapPartitionsRDD[39] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:43:29 INFO TaskSchedulerImpl: Adding task set 17.0 with 200 tasks resource profile 0 -26/04/01 08:43:29 INFO TaskSetManager: Starting task 0.0 in stage 17.0 (TID 534) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 1.0 in stage 17.0 (TID 535) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:29 INFO TaskSetManager: Starting task 2.0 in stage 17.0 (TID 536) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:29 INFO Executor: Running task 1.0 in stage 17.0 (TID 535) -26/04/01 08:43:29 INFO Executor: Running task 0.0 in stage 17.0 (TID 534) -26/04/01 08:43:29 INFO Executor: Running task 2.0 in stage 17.0 (TID 536) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO Executor: Finished task 59.0 in stage 13.0 (TID 329). 6606 bytes result sent to driver -26/04/01 08:43:29 INFO TaskSetManager: Starting task 3.0 in stage 17.0 (TID 537) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:29 INFO Executor: Running task 3.0 in stage 17.0 (TID 537) -26/04/01 08:43:29 INFO TaskSetManager: Finished task 59.0 in stage 13.0 (TID 329) in 1573 ms on 10.0.0.133 (executor driver) (60/64) -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:30 INFO Executor: Finished task 60.0 in stage 13.0 (TID 330). 6606 bytes result sent to driver -26/04/01 08:43:30 INFO TaskSetManager: Starting task 4.0 in stage 17.0 (TID 538) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:30 INFO Executor: Running task 4.0 in stage 17.0 (TID 538) -26/04/01 08:43:30 INFO TaskSetManager: Finished task 60.0 in stage 13.0 (TID 330) in 1595 ms on 10.0.0.133 (executor driver) (61/64) -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:30 INFO Executor: Finished task 61.0 in stage 13.0 (TID 331). 6606 bytes result sent to driver -26/04/01 08:43:30 INFO TaskSetManager: Starting task 5.0 in stage 17.0 (TID 539) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:30 INFO Executor: Running task 5.0 in stage 17.0 (TID 539) -26/04/01 08:43:30 INFO TaskSetManager: Finished task 61.0 in stage 13.0 (TID 331) in 1612 ms on 10.0.0.133 (executor driver) (62/64) -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:30 INFO Executor: Finished task 62.0 in stage 13.0 (TID 332). 6606 bytes result sent to driver -26/04/01 08:43:30 INFO TaskSetManager: Starting task 6.0 in stage 17.0 (TID 540) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:30 INFO TaskSetManager: Finished task 62.0 in stage 13.0 (TID 332) in 1592 ms on 10.0.0.133 (executor driver) (63/64) -26/04/01 08:43:30 INFO Executor: Running task 6.0 in stage 17.0 (TID 540) -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:30 INFO Executor: Finished task 63.0 in stage 13.0 (TID 333). 6606 bytes result sent to driver -26/04/01 08:43:30 INFO TaskSetManager: Starting task 7.0 in stage 17.0 (TID 541) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:30 INFO Executor: Running task 7.0 in stage 17.0 (TID 541) -26/04/01 08:43:30 INFO TaskSetManager: Finished task 63.0 in stage 13.0 (TID 333) in 1595 ms on 10.0.0.133 (executor driver) (64/64) -26/04/01 08:43:30 INFO TaskSchedulerImpl: Removed TaskSet 13.0, whose tasks have all completed, from pool -26/04/01 08:43:30 INFO DAGScheduler: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 234.724 s -26/04/01 08:43:30 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:43:30 INFO DAGScheduler: running: Set(ShuffleMapStage 17) -26/04/01 08:43:30 INFO DAGScheduler: waiting: Set() -26/04/01 08:43:30 INFO DAGScheduler: failed: Set() -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:31 INFO Executor: Finished task 0.0 in stage 17.0 (TID 534). 6401 bytes result sent to driver -26/04/01 08:43:31 INFO Executor: Finished task 1.0 in stage 17.0 (TID 535). 6401 bytes result sent to driver -26/04/01 08:43:31 INFO TaskSetManager: Starting task 8.0 in stage 17.0 (TID 542) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:31 INFO Executor: Running task 8.0 in stage 17.0 (TID 542) -26/04/01 08:43:31 INFO TaskSetManager: Starting task 9.0 in stage 17.0 (TID 543) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:31 INFO TaskSetManager: Finished task 0.0 in stage 17.0 (TID 534) in 2028 ms on 10.0.0.133 (executor driver) (1/200) -26/04/01 08:43:31 INFO Executor: Running task 9.0 in stage 17.0 (TID 543) -26/04/01 08:43:31 INFO TaskSetManager: Finished task 1.0 in stage 17.0 (TID 535) in 2027 ms on 10.0.0.133 (executor driver) (2/200) -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:31 INFO Executor: Finished task 2.0 in stage 17.0 (TID 536). 6401 bytes result sent to driver -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:31 INFO TaskSetManager: Starting task 10.0 in stage 17.0 (TID 544) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:31 INFO Executor: Running task 10.0 in stage 17.0 (TID 544) -26/04/01 08:43:31 INFO TaskSetManager: Finished task 2.0 in stage 17.0 (TID 536) in 2028 ms on 10.0.0.133 (executor driver) (3/200) -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:31 INFO Executor: Finished task 3.0 in stage 17.0 (TID 537). 6401 bytes result sent to driver -26/04/01 08:43:31 INFO TaskSetManager: Starting task 11.0 in stage 17.0 (TID 545) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:31 INFO TaskSetManager: Finished task 3.0 in stage 17.0 (TID 537) in 1947 ms on 10.0.0.133 (executor driver) (4/200) -26/04/01 08:43:31 INFO Executor: Running task 11.0 in stage 17.0 (TID 545) -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:32 INFO Executor: Finished task 4.0 in stage 17.0 (TID 538). 6401 bytes result sent to driver -26/04/01 08:43:32 INFO TaskSetManager: Starting task 12.0 in stage 17.0 (TID 546) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:32 INFO TaskSetManager: Finished task 4.0 in stage 17.0 (TID 538) in 1891 ms on 10.0.0.133 (executor driver) (5/200) -26/04/01 08:43:32 INFO Executor: Running task 12.0 in stage 17.0 (TID 546) -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:32 INFO Executor: Finished task 5.0 in stage 17.0 (TID 539). 6401 bytes result sent to driver -26/04/01 08:43:32 INFO TaskSetManager: Starting task 13.0 in stage 17.0 (TID 547) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:32 INFO Executor: Running task 13.0 in stage 17.0 (TID 547) -26/04/01 08:43:32 INFO TaskSetManager: Finished task 5.0 in stage 17.0 (TID 539) in 1879 ms on 10.0.0.133 (executor driver) (6/200) -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:32 INFO Executor: Finished task 6.0 in stage 17.0 (TID 540). 6401 bytes result sent to driver -26/04/01 08:43:32 INFO TaskSetManager: Starting task 14.0 in stage 17.0 (TID 548) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:32 INFO TaskSetManager: Finished task 6.0 in stage 17.0 (TID 540) in 1872 ms on 10.0.0.133 (executor driver) (7/200) -26/04/01 08:43:32 INFO Executor: Running task 14.0 in stage 17.0 (TID 548) -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:32 INFO Executor: Finished task 7.0 in stage 17.0 (TID 541). 6401 bytes result sent to driver -26/04/01 08:43:32 INFO TaskSetManager: Starting task 15.0 in stage 17.0 (TID 549) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:32 INFO Executor: Running task 15.0 in stage 17.0 (TID 549) -26/04/01 08:43:32 INFO TaskSetManager: Finished task 7.0 in stage 17.0 (TID 541) in 1906 ms on 10.0.0.133 (executor driver) (8/200) -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:33 INFO Executor: Finished task 8.0 in stage 17.0 (TID 542). 6401 bytes result sent to driver -26/04/01 08:43:33 INFO TaskSetManager: Starting task 16.0 in stage 17.0 (TID 550) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:33 INFO TaskSetManager: Finished task 8.0 in stage 17.0 (TID 542) in 1950 ms on 10.0.0.133 (executor driver) (9/200) -26/04/01 08:43:33 INFO Executor: Running task 16.0 in stage 17.0 (TID 550) -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:33 INFO Executor: Finished task 11.0 in stage 17.0 (TID 545). 6401 bytes result sent to driver -26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:33 INFO TaskSetManager: Starting task 17.0 in stage 17.0 (TID 551) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:33 INFO Executor: Running task 17.0 in stage 17.0 (TID 551) -26/04/01 08:43:33 INFO TaskSetManager: Finished task 11.0 in stage 17.0 (TID 545) in 1880 ms on 10.0.0.133 (executor driver) (10/200) -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:33 INFO Executor: Finished task 10.0 in stage 17.0 (TID 544). 6401 bytes result sent to driver -26/04/01 08:43:33 INFO TaskSetManager: Starting task 18.0 in stage 17.0 (TID 552) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:33 INFO Executor: Running task 18.0 in stage 17.0 (TID 552) -26/04/01 08:43:33 INFO TaskSetManager: Finished task 10.0 in stage 17.0 (TID 544) in 1962 ms on 10.0.0.133 (executor driver) (11/200) -26/04/01 08:43:33 INFO Executor: Finished task 9.0 in stage 17.0 (TID 543). 6401 bytes result sent to driver -26/04/01 08:43:33 INFO TaskSetManager: Starting task 19.0 in stage 17.0 (TID 553) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:33 INFO TaskSetManager: Finished task 9.0 in stage 17.0 (TID 543) in 1964 ms on 10.0.0.133 (executor driver) (12/200) -26/04/01 08:43:33 INFO Executor: Running task 19.0 in stage 17.0 (TID 553) -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:34 INFO Executor: Finished task 12.0 in stage 17.0 (TID 546). 6401 bytes result sent to driver -26/04/01 08:43:34 INFO TaskSetManager: Starting task 20.0 in stage 17.0 (TID 554) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:34 INFO Executor: Running task 20.0 in stage 17.0 (TID 554) -26/04/01 08:43:34 INFO TaskSetManager: Finished task 12.0 in stage 17.0 (TID 546) in 1862 ms on 10.0.0.133 (executor driver) (13/200) -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:34 INFO Executor: Finished task 13.0 in stage 17.0 (TID 547). 6401 bytes result sent to driver -26/04/01 08:43:34 INFO TaskSetManager: Starting task 21.0 in stage 17.0 (TID 555) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:34 INFO Executor: Running task 21.0 in stage 17.0 (TID 555) -26/04/01 08:43:34 INFO TaskSetManager: Finished task 13.0 in stage 17.0 (TID 547) in 1877 ms on 10.0.0.133 (executor driver) (14/200) -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:34 INFO Executor: Finished task 14.0 in stage 17.0 (TID 548). 6401 bytes result sent to driver -26/04/01 08:43:34 INFO TaskSetManager: Starting task 22.0 in stage 17.0 (TID 556) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:34 INFO TaskSetManager: Finished task 14.0 in stage 17.0 (TID 548) in 1880 ms on 10.0.0.133 (executor driver) (15/200) -26/04/01 08:43:34 INFO Executor: Running task 22.0 in stage 17.0 (TID 556) -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (76.4 MiB) non-empty blocks including 208 (76.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (76.4 MiB) non-empty blocks including 208 (76.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:34 INFO Executor: Finished task 15.0 in stage 17.0 (TID 549). 6401 bytes result sent to driver -26/04/01 08:43:34 INFO TaskSetManager: Starting task 23.0 in stage 17.0 (TID 557) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:34 INFO TaskSetManager: Finished task 15.0 in stage 17.0 (TID 549) in 1883 ms on 10.0.0.133 (executor driver) (16/200) -26/04/01 08:43:34 INFO Executor: Running task 23.0 in stage 17.0 (TID 557) -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:35 INFO Executor: Finished task 16.0 in stage 17.0 (TID 550). 6401 bytes result sent to driver -26/04/01 08:43:35 INFO TaskSetManager: Starting task 24.0 in stage 17.0 (TID 558) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:35 INFO TaskSetManager: Finished task 16.0 in stage 17.0 (TID 550) in 1908 ms on 10.0.0.133 (executor driver) (17/200) -26/04/01 08:43:35 INFO Executor: Running task 24.0 in stage 17.0 (TID 558) -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:35 INFO Executor: Finished task 17.0 in stage 17.0 (TID 551). 6401 bytes result sent to driver -26/04/01 08:43:35 INFO TaskSetManager: Starting task 25.0 in stage 17.0 (TID 559) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:35 INFO TaskSetManager: Finished task 17.0 in stage 17.0 (TID 551) in 1910 ms on 10.0.0.133 (executor driver) (18/200) -26/04/01 08:43:35 INFO Executor: Running task 25.0 in stage 17.0 (TID 559) -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:35 INFO Executor: Finished task 18.0 in stage 17.0 (TID 552). 6401 bytes result sent to driver -26/04/01 08:43:35 INFO TaskSetManager: Starting task 26.0 in stage 17.0 (TID 560) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:35 INFO TaskSetManager: Finished task 18.0 in stage 17.0 (TID 552) in 1913 ms on 10.0.0.133 (executor driver) (19/200) -26/04/01 08:43:35 INFO Executor: Running task 26.0 in stage 17.0 (TID 560) -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:35 INFO Executor: Finished task 19.0 in stage 17.0 (TID 553). 6401 bytes result sent to driver -26/04/01 08:43:35 INFO TaskSetManager: Starting task 27.0 in stage 17.0 (TID 561) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:35 INFO TaskSetManager: Finished task 19.0 in stage 17.0 (TID 553) in 1931 ms on 10.0.0.133 (executor driver) (20/200) -26/04/01 08:43:35 INFO Executor: Running task 27.0 in stage 17.0 (TID 561) -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:36 INFO Executor: Finished task 20.0 in stage 17.0 (TID 554). 6401 bytes result sent to driver -26/04/01 08:43:36 INFO TaskSetManager: Starting task 28.0 in stage 17.0 (TID 562) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:36 INFO Executor: Running task 28.0 in stage 17.0 (TID 562) -26/04/01 08:43:36 INFO TaskSetManager: Finished task 20.0 in stage 17.0 (TID 554) in 1895 ms on 10.0.0.133 (executor driver) (21/200) -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:36 INFO Executor: Finished task 21.0 in stage 17.0 (TID 555). 6401 bytes result sent to driver -26/04/01 08:43:36 INFO TaskSetManager: Starting task 29.0 in stage 17.0 (TID 563) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:36 INFO Executor: Running task 29.0 in stage 17.0 (TID 563) -26/04/01 08:43:36 INFO TaskSetManager: Finished task 21.0 in stage 17.0 (TID 555) in 1891 ms on 10.0.0.133 (executor driver) (22/200) -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:36 INFO Executor: Finished task 22.0 in stage 17.0 (TID 556). 6401 bytes result sent to driver -26/04/01 08:43:36 INFO TaskSetManager: Starting task 30.0 in stage 17.0 (TID 564) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:36 INFO Executor: Running task 30.0 in stage 17.0 (TID 564) -26/04/01 08:43:36 INFO TaskSetManager: Finished task 22.0 in stage 17.0 (TID 556) in 1888 ms on 10.0.0.133 (executor driver) (23/200) -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:36 INFO Executor: Finished task 23.0 in stage 17.0 (TID 557). 6401 bytes result sent to driver -26/04/01 08:43:36 INFO TaskSetManager: Starting task 31.0 in stage 17.0 (TID 565) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:36 INFO TaskSetManager: Finished task 23.0 in stage 17.0 (TID 557) in 1879 ms on 10.0.0.133 (executor driver) (24/200) -26/04/01 08:43:36 INFO Executor: Running task 31.0 in stage 17.0 (TID 565) -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:37 INFO Executor: Finished task 26.0 in stage 17.0 (TID 560). 6401 bytes result sent to driver -26/04/01 08:43:37 INFO TaskSetManager: Starting task 32.0 in stage 17.0 (TID 566) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:37 INFO TaskSetManager: Finished task 26.0 in stage 17.0 (TID 560) in 1869 ms on 10.0.0.133 (executor driver) (25/200) -26/04/01 08:43:37 INFO Executor: Running task 32.0 in stage 17.0 (TID 566) -26/04/01 08:43:37 INFO Executor: Finished task 25.0 in stage 17.0 (TID 559). 6401 bytes result sent to driver -26/04/01 08:43:37 INFO TaskSetManager: Starting task 33.0 in stage 17.0 (TID 567) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:37 INFO TaskSetManager: Finished task 25.0 in stage 17.0 (TID 559) in 1875 ms on 10.0.0.133 (executor driver) (26/200) -26/04/01 08:43:37 INFO Executor: Running task 33.0 in stage 17.0 (TID 567) -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:37 INFO Executor: Finished task 24.0 in stage 17.0 (TID 558). 6401 bytes result sent to driver -26/04/01 08:43:37 INFO TaskSetManager: Starting task 34.0 in stage 17.0 (TID 568) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:37 INFO TaskSetManager: Finished task 24.0 in stage 17.0 (TID 558) in 1890 ms on 10.0.0.133 (executor driver) (27/200) -26/04/01 08:43:37 INFO Executor: Running task 34.0 in stage 17.0 (TID 568) -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:37 INFO Executor: Finished task 27.0 in stage 17.0 (TID 561). 6401 bytes result sent to driver -26/04/01 08:43:37 INFO TaskSetManager: Starting task 35.0 in stage 17.0 (TID 569) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:37 INFO Executor: Running task 35.0 in stage 17.0 (TID 569) -26/04/01 08:43:37 INFO TaskSetManager: Finished task 27.0 in stage 17.0 (TID 561) in 1862 ms on 10.0.0.133 (executor driver) (28/200) -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:37 INFO BlockManagerInfo: Removed broadcast_19_piece0 on 10.0.0.133:59303 in memory (size: 7.9 KiB, free: 8.6 GiB) -26/04/01 08:43:37 INFO BlockManagerInfo: Removed broadcast_21_piece0 on 10.0.0.133:59303 in memory (size: 7.9 KiB, free: 8.6 GiB) -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:38 INFO Executor: Finished task 28.0 in stage 17.0 (TID 562). 6444 bytes result sent to driver -26/04/01 08:43:38 INFO TaskSetManager: Starting task 36.0 in stage 17.0 (TID 570) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:38 INFO Executor: Running task 36.0 in stage 17.0 (TID 570) -26/04/01 08:43:38 INFO TaskSetManager: Finished task 28.0 in stage 17.0 (TID 562) in 1888 ms on 10.0.0.133 (executor driver) (29/200) -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:38 INFO Executor: Finished task 29.0 in stage 17.0 (TID 563). 6444 bytes result sent to driver -26/04/01 08:43:38 INFO TaskSetManager: Starting task 37.0 in stage 17.0 (TID 571) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:38 INFO Executor: Running task 37.0 in stage 17.0 (TID 571) -26/04/01 08:43:38 INFO TaskSetManager: Finished task 29.0 in stage 17.0 (TID 563) in 1878 ms on 10.0.0.133 (executor driver) (30/200) -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:38 INFO Executor: Finished task 30.0 in stage 17.0 (TID 564). 6444 bytes result sent to driver -26/04/01 08:43:38 INFO TaskSetManager: Starting task 38.0 in stage 17.0 (TID 572) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:38 INFO TaskSetManager: Finished task 30.0 in stage 17.0 (TID 564) in 1881 ms on 10.0.0.133 (executor driver) (31/200) -26/04/01 08:43:38 INFO Executor: Running task 38.0 in stage 17.0 (TID 572) -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:38 INFO Executor: Finished task 31.0 in stage 17.0 (TID 565). 6444 bytes result sent to driver -26/04/01 08:43:38 INFO TaskSetManager: Starting task 39.0 in stage 17.0 (TID 573) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:38 INFO TaskSetManager: Finished task 31.0 in stage 17.0 (TID 565) in 1897 ms on 10.0.0.133 (executor driver) (32/200) -26/04/01 08:43:38 INFO Executor: Running task 39.0 in stage 17.0 (TID 573) -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:39 INFO Executor: Finished task 33.0 in stage 17.0 (TID 567). 7175 bytes result sent to driver -26/04/01 08:43:39 INFO Executor: Finished task 32.0 in stage 17.0 (TID 566). 7175 bytes result sent to driver -26/04/01 08:43:39 INFO TaskSetManager: Starting task 40.0 in stage 17.0 (TID 574) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:39 INFO Executor: Running task 40.0 in stage 17.0 (TID 574) -26/04/01 08:43:39 INFO TaskSetManager: Starting task 41.0 in stage 17.0 (TID 575) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:39 INFO TaskSetManager: Finished task 33.0 in stage 17.0 (TID 567) in 1890 ms on 10.0.0.133 (executor driver) (33/200) -26/04/01 08:43:39 INFO Executor: Running task 41.0 in stage 17.0 (TID 575) -26/04/01 08:43:39 INFO TaskSetManager: Finished task 32.0 in stage 17.0 (TID 566) in 1891 ms on 10.0.0.133 (executor driver) (34/200) -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:39 INFO Executor: Finished task 34.0 in stage 17.0 (TID 568). 7175 bytes result sent to driver -26/04/01 08:43:39 INFO TaskSetManager: Starting task 42.0 in stage 17.0 (TID 576) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:39 INFO TaskSetManager: Finished task 34.0 in stage 17.0 (TID 568) in 1893 ms on 10.0.0.133 (executor driver) (35/200) -26/04/01 08:43:39 INFO Executor: Running task 42.0 in stage 17.0 (TID 576) -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:39 INFO Executor: Finished task 35.0 in stage 17.0 (TID 569). 7132 bytes result sent to driver -26/04/01 08:43:39 INFO TaskSetManager: Starting task 43.0 in stage 17.0 (TID 577) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:39 INFO TaskSetManager: Finished task 35.0 in stage 17.0 (TID 569) in 1898 ms on 10.0.0.133 (executor driver) (36/200) -26/04/01 08:43:39 INFO Executor: Running task 43.0 in stage 17.0 (TID 577) -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:40 INFO Executor: Finished task 36.0 in stage 17.0 (TID 570). 7132 bytes result sent to driver -26/04/01 08:43:40 INFO TaskSetManager: Starting task 44.0 in stage 17.0 (TID 578) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:40 INFO TaskSetManager: Finished task 36.0 in stage 17.0 (TID 570) in 1884 ms on 10.0.0.133 (executor driver) (37/200) -26/04/01 08:43:40 INFO Executor: Running task 44.0 in stage 17.0 (TID 578) -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:40 INFO Executor: Finished task 37.0 in stage 17.0 (TID 571). 7132 bytes result sent to driver -26/04/01 08:43:40 INFO TaskSetManager: Starting task 45.0 in stage 17.0 (TID 579) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:40 INFO Executor: Running task 45.0 in stage 17.0 (TID 579) -26/04/01 08:43:40 INFO TaskSetManager: Finished task 37.0 in stage 17.0 (TID 571) in 1866 ms on 10.0.0.133 (executor driver) (38/200) -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:40 INFO Executor: Finished task 38.0 in stage 17.0 (TID 572). 7132 bytes result sent to driver -26/04/01 08:43:40 INFO TaskSetManager: Starting task 46.0 in stage 17.0 (TID 580) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:40 INFO TaskSetManager: Finished task 38.0 in stage 17.0 (TID 572) in 1862 ms on 10.0.0.133 (executor driver) (39/200) -26/04/01 08:43:40 INFO Executor: Running task 46.0 in stage 17.0 (TID 580) -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:40 INFO Executor: Finished task 39.0 in stage 17.0 (TID 573). 7132 bytes result sent to driver -26/04/01 08:43:40 INFO TaskSetManager: Starting task 47.0 in stage 17.0 (TID 581) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:40 INFO TaskSetManager: Finished task 39.0 in stage 17.0 (TID 573) in 1869 ms on 10.0.0.133 (executor driver) (40/200) -26/04/01 08:43:40 INFO Executor: Running task 47.0 in stage 17.0 (TID 581) -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:41 INFO Executor: Finished task 43.0 in stage 17.0 (TID 577). 6401 bytes result sent to driver -26/04/01 08:43:41 INFO TaskSetManager: Starting task 48.0 in stage 17.0 (TID 582) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:41 INFO Executor: Finished task 41.0 in stage 17.0 (TID 575). 6401 bytes result sent to driver -26/04/01 08:43:41 INFO Executor: Running task 48.0 in stage 17.0 (TID 582) -26/04/01 08:43:41 INFO TaskSetManager: Finished task 43.0 in stage 17.0 (TID 577) in 1871 ms on 10.0.0.133 (executor driver) (41/200) -26/04/01 08:43:41 INFO Executor: Finished task 40.0 in stage 17.0 (TID 574). 6401 bytes result sent to driver -26/04/01 08:43:41 INFO TaskSetManager: Starting task 49.0 in stage 17.0 (TID 583) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:41 INFO Executor: Running task 49.0 in stage 17.0 (TID 583) -26/04/01 08:43:41 INFO TaskSetManager: Starting task 50.0 in stage 17.0 (TID 584) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:41 INFO TaskSetManager: Finished task 41.0 in stage 17.0 (TID 575) in 1890 ms on 10.0.0.133 (executor driver) (42/200) -26/04/01 08:43:41 INFO Executor: Running task 50.0 in stage 17.0 (TID 584) -26/04/01 08:43:41 INFO TaskSetManager: Finished task 40.0 in stage 17.0 (TID 574) in 1890 ms on 10.0.0.133 (executor driver) (43/200) -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:41 INFO Executor: Finished task 42.0 in stage 17.0 (TID 576). 6401 bytes result sent to driver -26/04/01 08:43:41 INFO TaskSetManager: Starting task 51.0 in stage 17.0 (TID 585) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:41 INFO TaskSetManager: Finished task 42.0 in stage 17.0 (TID 576) in 1886 ms on 10.0.0.133 (executor driver) (44/200) -26/04/01 08:43:41 INFO Executor: Running task 51.0 in stage 17.0 (TID 585) -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:41 INFO Executor: Finished task 44.0 in stage 17.0 (TID 578). 6401 bytes result sent to driver -26/04/01 08:43:41 INFO TaskSetManager: Starting task 52.0 in stage 17.0 (TID 586) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:41 INFO Executor: Running task 52.0 in stage 17.0 (TID 586) -26/04/01 08:43:41 INFO TaskSetManager: Finished task 44.0 in stage 17.0 (TID 578) in 1868 ms on 10.0.0.133 (executor driver) (45/200) -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:42 INFO Executor: Finished task 45.0 in stage 17.0 (TID 579). 6401 bytes result sent to driver -26/04/01 08:43:42 INFO TaskSetManager: Starting task 53.0 in stage 17.0 (TID 587) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:42 INFO TaskSetManager: Finished task 45.0 in stage 17.0 (TID 579) in 1865 ms on 10.0.0.133 (executor driver) (46/200) -26/04/01 08:43:42 INFO Executor: Running task 53.0 in stage 17.0 (TID 587) -26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:42 INFO Executor: Finished task 46.0 in stage 17.0 (TID 580). 6401 bytes result sent to driver -26/04/01 08:43:42 INFO TaskSetManager: Starting task 54.0 in stage 17.0 (TID 588) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:42 INFO TaskSetManager: Finished task 46.0 in stage 17.0 (TID 580) in 1869 ms on 10.0.0.133 (executor driver) (47/200) -26/04/01 08:43:42 INFO Executor: Running task 54.0 in stage 17.0 (TID 588) -26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:42 INFO Executor: Finished task 47.0 in stage 17.0 (TID 581). 6401 bytes result sent to driver -26/04/01 08:43:42 INFO TaskSetManager: Starting task 55.0 in stage 17.0 (TID 589) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:42 INFO TaskSetManager: Finished task 47.0 in stage 17.0 (TID 581) in 1889 ms on 10.0.0.133 (executor driver) (48/200) -26/04/01 08:43:42 INFO Executor: Running task 55.0 in stage 17.0 (TID 589) -26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:42 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:43 INFO Executor: Finished task 48.0 in stage 17.0 (TID 582). 6401 bytes result sent to driver -26/04/01 08:43:43 INFO TaskSetManager: Starting task 56.0 in stage 17.0 (TID 590) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:43 INFO Executor: Finished task 49.0 in stage 17.0 (TID 583). 6401 bytes result sent to driver -26/04/01 08:43:43 INFO Executor: Running task 56.0 in stage 17.0 (TID 590) -26/04/01 08:43:43 INFO TaskSetManager: Starting task 57.0 in stage 17.0 (TID 591) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:43 INFO Executor: Running task 57.0 in stage 17.0 (TID 591) -26/04/01 08:43:43 INFO TaskSetManager: Finished task 48.0 in stage 17.0 (TID 582) in 1890 ms on 10.0.0.133 (executor driver) (49/200) -26/04/01 08:43:43 INFO TaskSetManager: Finished task 49.0 in stage 17.0 (TID 583) in 1891 ms on 10.0.0.133 (executor driver) (50/200) -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:43 INFO Executor: Finished task 50.0 in stage 17.0 (TID 584). 6401 bytes result sent to driver -26/04/01 08:43:43 INFO TaskSetManager: Starting task 58.0 in stage 17.0 (TID 592) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:43 INFO Executor: Running task 58.0 in stage 17.0 (TID 592) -26/04/01 08:43:43 INFO TaskSetManager: Finished task 50.0 in stage 17.0 (TID 584) in 1897 ms on 10.0.0.133 (executor driver) (51/200) -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:43 INFO Executor: Finished task 51.0 in stage 17.0 (TID 585). 6401 bytes result sent to driver -26/04/01 08:43:43 INFO TaskSetManager: Starting task 59.0 in stage 17.0 (TID 593) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:43 INFO TaskSetManager: Finished task 51.0 in stage 17.0 (TID 585) in 1917 ms on 10.0.0.133 (executor driver) (52/200) -26/04/01 08:43:43 INFO Executor: Running task 59.0 in stage 17.0 (TID 593) -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:43 INFO Executor: Finished task 52.0 in stage 17.0 (TID 586). 6401 bytes result sent to driver -26/04/01 08:43:43 INFO TaskSetManager: Starting task 60.0 in stage 17.0 (TID 594) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:43 INFO TaskSetManager: Finished task 52.0 in stage 17.0 (TID 586) in 1908 ms on 10.0.0.133 (executor driver) (53/200) -26/04/01 08:43:43 INFO Executor: Running task 60.0 in stage 17.0 (TID 594) -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:43 INFO Executor: Finished task 53.0 in stage 17.0 (TID 587). 6401 bytes result sent to driver -26/04/01 08:43:43 INFO TaskSetManager: Starting task 61.0 in stage 17.0 (TID 595) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:43 INFO Executor: Running task 61.0 in stage 17.0 (TID 595) -26/04/01 08:43:43 INFO TaskSetManager: Finished task 53.0 in stage 17.0 (TID 587) in 1899 ms on 10.0.0.133 (executor driver) (54/200) -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:43 INFO Executor: Finished task 54.0 in stage 17.0 (TID 588). 6401 bytes result sent to driver -26/04/01 08:43:43 INFO TaskSetManager: Starting task 62.0 in stage 17.0 (TID 596) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:43 INFO TaskSetManager: Finished task 54.0 in stage 17.0 (TID 588) in 1908 ms on 10.0.0.133 (executor driver) (55/200) -26/04/01 08:43:43 INFO Executor: Running task 62.0 in stage 17.0 (TID 596) -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:44 INFO Executor: Finished task 55.0 in stage 17.0 (TID 589). 6401 bytes result sent to driver -26/04/01 08:43:44 INFO TaskSetManager: Starting task 63.0 in stage 17.0 (TID 597) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:44 INFO Executor: Running task 63.0 in stage 17.0 (TID 597) -26/04/01 08:43:44 INFO TaskSetManager: Finished task 55.0 in stage 17.0 (TID 589) in 1861 ms on 10.0.0.133 (executor driver) (56/200) -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:44 INFO Executor: Finished task 57.0 in stage 17.0 (TID 591). 6401 bytes result sent to driver -26/04/01 08:43:44 INFO TaskSetManager: Starting task 64.0 in stage 17.0 (TID 598) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:44 INFO Executor: Running task 64.0 in stage 17.0 (TID 598) -26/04/01 08:43:44 INFO TaskSetManager: Finished task 57.0 in stage 17.0 (TID 591) in 1870 ms on 10.0.0.133 (executor driver) (57/200) -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:44 INFO Executor: Finished task 56.0 in stage 17.0 (TID 590). 6401 bytes result sent to driver -26/04/01 08:43:44 INFO TaskSetManager: Starting task 65.0 in stage 17.0 (TID 599) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:44 INFO Executor: Running task 65.0 in stage 17.0 (TID 599) -26/04/01 08:43:44 INFO TaskSetManager: Finished task 56.0 in stage 17.0 (TID 590) in 1875 ms on 10.0.0.133 (executor driver) (58/200) -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:44 INFO Executor: Finished task 58.0 in stage 17.0 (TID 592). 6401 bytes result sent to driver -26/04/01 08:43:44 INFO TaskSetManager: Starting task 66.0 in stage 17.0 (TID 600) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:44 INFO Executor: Running task 66.0 in stage 17.0 (TID 600) -26/04/01 08:43:44 INFO TaskSetManager: Finished task 58.0 in stage 17.0 (TID 592) in 1874 ms on 10.0.0.133 (executor driver) (59/200) -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:44 INFO Executor: Finished task 59.0 in stage 17.0 (TID 593). 6401 bytes result sent to driver -26/04/01 08:43:44 INFO TaskSetManager: Starting task 67.0 in stage 17.0 (TID 601) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:44 INFO Executor: Running task 67.0 in stage 17.0 (TID 601) -26/04/01 08:43:44 INFO TaskSetManager: Finished task 59.0 in stage 17.0 (TID 593) in 1866 ms on 10.0.0.133 (executor driver) (60/200) -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:45 INFO Executor: Finished task 60.0 in stage 17.0 (TID 594). 6401 bytes result sent to driver -26/04/01 08:43:45 INFO TaskSetManager: Starting task 68.0 in stage 17.0 (TID 602) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:45 INFO Executor: Running task 68.0 in stage 17.0 (TID 602) -26/04/01 08:43:45 INFO TaskSetManager: Finished task 60.0 in stage 17.0 (TID 594) in 1870 ms on 10.0.0.133 (executor driver) (61/200) -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:45 INFO Executor: Finished task 61.0 in stage 17.0 (TID 595). 6401 bytes result sent to driver -26/04/01 08:43:45 INFO TaskSetManager: Starting task 69.0 in stage 17.0 (TID 603) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:45 INFO TaskSetManager: Finished task 61.0 in stage 17.0 (TID 595) in 1867 ms on 10.0.0.133 (executor driver) (62/200) -26/04/01 08:43:45 INFO Executor: Running task 69.0 in stage 17.0 (TID 603) -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:45 INFO Executor: Finished task 62.0 in stage 17.0 (TID 596). 6401 bytes result sent to driver -26/04/01 08:43:45 INFO TaskSetManager: Starting task 70.0 in stage 17.0 (TID 604) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:45 INFO Executor: Running task 70.0 in stage 17.0 (TID 604) -26/04/01 08:43:45 INFO TaskSetManager: Finished task 62.0 in stage 17.0 (TID 596) in 1862 ms on 10.0.0.133 (executor driver) (63/200) -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:45 INFO Executor: Finished task 63.0 in stage 17.0 (TID 597). 6401 bytes result sent to driver -26/04/01 08:43:45 INFO TaskSetManager: Starting task 71.0 in stage 17.0 (TID 605) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:45 INFO Executor: Running task 71.0 in stage 17.0 (TID 605) -26/04/01 08:43:45 INFO TaskSetManager: Finished task 63.0 in stage 17.0 (TID 597) in 1862 ms on 10.0.0.133 (executor driver) (64/200) -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:46 INFO Executor: Finished task 64.0 in stage 17.0 (TID 598). 6401 bytes result sent to driver -26/04/01 08:43:46 INFO TaskSetManager: Starting task 72.0 in stage 17.0 (TID 606) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:46 INFO TaskSetManager: Finished task 64.0 in stage 17.0 (TID 598) in 1841 ms on 10.0.0.133 (executor driver) (65/200) -26/04/01 08:43:46 INFO Executor: Running task 72.0 in stage 17.0 (TID 606) -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:46 INFO Executor: Finished task 65.0 in stage 17.0 (TID 599). 6401 bytes result sent to driver -26/04/01 08:43:46 INFO TaskSetManager: Starting task 73.0 in stage 17.0 (TID 607) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:46 INFO TaskSetManager: Finished task 65.0 in stage 17.0 (TID 599) in 1846 ms on 10.0.0.133 (executor driver) (66/200) -26/04/01 08:43:46 INFO Executor: Running task 73.0 in stage 17.0 (TID 607) -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:46 INFO Executor: Finished task 66.0 in stage 17.0 (TID 600). 6401 bytes result sent to driver -26/04/01 08:43:46 INFO TaskSetManager: Starting task 74.0 in stage 17.0 (TID 608) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:46 INFO TaskSetManager: Finished task 66.0 in stage 17.0 (TID 600) in 1849 ms on 10.0.0.133 (executor driver) (67/200) -26/04/01 08:43:46 INFO Executor: Running task 74.0 in stage 17.0 (TID 608) -26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:46 INFO Executor: Finished task 67.0 in stage 17.0 (TID 601). 6401 bytes result sent to driver -26/04/01 08:43:46 INFO TaskSetManager: Starting task 75.0 in stage 17.0 (TID 609) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:46 INFO TaskSetManager: Finished task 67.0 in stage 17.0 (TID 601) in 1855 ms on 10.0.0.133 (executor driver) (68/200) -26/04/01 08:43:46 INFO Executor: Running task 75.0 in stage 17.0 (TID 609) -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:47 INFO Executor: Finished task 68.0 in stage 17.0 (TID 602). 6401 bytes result sent to driver -26/04/01 08:43:47 INFO TaskSetManager: Starting task 76.0 in stage 17.0 (TID 610) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:47 INFO TaskSetManager: Finished task 68.0 in stage 17.0 (TID 602) in 1884 ms on 10.0.0.133 (executor driver) (69/200) -26/04/01 08:43:47 INFO Executor: Running task 76.0 in stage 17.0 (TID 610) -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:47 INFO Executor: Finished task 69.0 in stage 17.0 (TID 603). 6401 bytes result sent to driver -26/04/01 08:43:47 INFO TaskSetManager: Starting task 77.0 in stage 17.0 (TID 611) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:47 INFO TaskSetManager: Finished task 69.0 in stage 17.0 (TID 603) in 1881 ms on 10.0.0.133 (executor driver) (70/200) -26/04/01 08:43:47 INFO Executor: Running task 77.0 in stage 17.0 (TID 611) -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:47 INFO Executor: Finished task 70.0 in stage 17.0 (TID 604). 6401 bytes result sent to driver -26/04/01 08:43:47 INFO TaskSetManager: Starting task 78.0 in stage 17.0 (TID 612) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:47 INFO Executor: Running task 78.0 in stage 17.0 (TID 612) -26/04/01 08:43:47 INFO TaskSetManager: Finished task 70.0 in stage 17.0 (TID 604) in 1891 ms on 10.0.0.133 (executor driver) (71/200) -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:47 INFO Executor: Finished task 71.0 in stage 17.0 (TID 605). 6401 bytes result sent to driver -26/04/01 08:43:47 INFO TaskSetManager: Starting task 79.0 in stage 17.0 (TID 613) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:47 INFO TaskSetManager: Finished task 71.0 in stage 17.0 (TID 605) in 1889 ms on 10.0.0.133 (executor driver) (72/200) -26/04/01 08:43:47 INFO Executor: Running task 79.0 in stage 17.0 (TID 613) -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:48 INFO Executor: Finished task 72.0 in stage 17.0 (TID 606). 7132 bytes result sent to driver -26/04/01 08:43:48 INFO TaskSetManager: Starting task 80.0 in stage 17.0 (TID 614) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:48 INFO Executor: Running task 80.0 in stage 17.0 (TID 614) -26/04/01 08:43:48 INFO TaskSetManager: Finished task 72.0 in stage 17.0 (TID 606) in 1863 ms on 10.0.0.133 (executor driver) (73/200) -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:48 INFO Executor: Finished task 74.0 in stage 17.0 (TID 608). 7132 bytes result sent to driver -26/04/01 08:43:48 INFO TaskSetManager: Starting task 81.0 in stage 17.0 (TID 615) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:48 INFO TaskSetManager: Finished task 74.0 in stage 17.0 (TID 608) in 1849 ms on 10.0.0.133 (executor driver) (74/200) -26/04/01 08:43:48 INFO Executor: Running task 81.0 in stage 17.0 (TID 615) -26/04/01 08:43:48 INFO Executor: Finished task 73.0 in stage 17.0 (TID 607). 7132 bytes result sent to driver -26/04/01 08:43:48 INFO TaskSetManager: Starting task 82.0 in stage 17.0 (TID 616) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:48 INFO Executor: Running task 82.0 in stage 17.0 (TID 616) -26/04/01 08:43:48 INFO TaskSetManager: Finished task 73.0 in stage 17.0 (TID 607) in 1860 ms on 10.0.0.133 (executor driver) (75/200) -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:48 INFO Executor: Finished task 75.0 in stage 17.0 (TID 609). 7132 bytes result sent to driver -26/04/01 08:43:48 INFO TaskSetManager: Starting task 83.0 in stage 17.0 (TID 617) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:48 INFO Executor: Running task 83.0 in stage 17.0 (TID 617) -26/04/01 08:43:48 INFO TaskSetManager: Finished task 75.0 in stage 17.0 (TID 609) in 1851 ms on 10.0.0.133 (executor driver) (76/200) -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:49 INFO Executor: Finished task 76.0 in stage 17.0 (TID 610). 7132 bytes result sent to driver -26/04/01 08:43:49 INFO TaskSetManager: Starting task 84.0 in stage 17.0 (TID 618) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:49 INFO TaskSetManager: Finished task 76.0 in stage 17.0 (TID 610) in 1849 ms on 10.0.0.133 (executor driver) (77/200) -26/04/01 08:43:49 INFO Executor: Running task 84.0 in stage 17.0 (TID 618) -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:49 INFO Executor: Finished task 78.0 in stage 17.0 (TID 612). 7132 bytes result sent to driver -26/04/01 08:43:49 INFO TaskSetManager: Starting task 85.0 in stage 17.0 (TID 619) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:49 INFO Executor: Running task 85.0 in stage 17.0 (TID 619) -26/04/01 08:43:49 INFO TaskSetManager: Finished task 78.0 in stage 17.0 (TID 612) in 1843 ms on 10.0.0.133 (executor driver) (78/200) -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:49 INFO Executor: Finished task 77.0 in stage 17.0 (TID 611). 7132 bytes result sent to driver -26/04/01 08:43:49 INFO TaskSetManager: Starting task 86.0 in stage 17.0 (TID 620) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:49 INFO TaskSetManager: Finished task 77.0 in stage 17.0 (TID 611) in 1865 ms on 10.0.0.133 (executor driver) (79/200) -26/04/01 08:43:49 INFO Executor: Running task 86.0 in stage 17.0 (TID 620) -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:49 INFO Executor: Finished task 79.0 in stage 17.0 (TID 613). 7132 bytes result sent to driver -26/04/01 08:43:49 INFO TaskSetManager: Starting task 87.0 in stage 17.0 (TID 621) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:49 INFO TaskSetManager: Finished task 79.0 in stage 17.0 (TID 613) in 1847 ms on 10.0.0.133 (executor driver) (80/200) -26/04/01 08:43:49 INFO Executor: Running task 87.0 in stage 17.0 (TID 621) -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:50 INFO Executor: Finished task 81.0 in stage 17.0 (TID 615). 6401 bytes result sent to driver -26/04/01 08:43:50 INFO TaskSetManager: Starting task 88.0 in stage 17.0 (TID 622) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:50 INFO TaskSetManager: Finished task 81.0 in stage 17.0 (TID 615) in 1838 ms on 10.0.0.133 (executor driver) (81/200) -26/04/01 08:43:50 INFO Executor: Running task 88.0 in stage 17.0 (TID 622) -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:50 INFO Executor: Finished task 80.0 in stage 17.0 (TID 614). 6401 bytes result sent to driver -26/04/01 08:43:50 INFO TaskSetManager: Starting task 89.0 in stage 17.0 (TID 623) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:50 INFO TaskSetManager: Finished task 80.0 in stage 17.0 (TID 614) in 1854 ms on 10.0.0.133 (executor driver) (82/200) -26/04/01 08:43:50 INFO Executor: Running task 89.0 in stage 17.0 (TID 623) -26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:50 INFO Executor: Finished task 82.0 in stage 17.0 (TID 616). 6401 bytes result sent to driver -26/04/01 08:43:50 INFO TaskSetManager: Starting task 90.0 in stage 17.0 (TID 624) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:50 INFO TaskSetManager: Finished task 82.0 in stage 17.0 (TID 616) in 1865 ms on 10.0.0.133 (executor driver) (83/200) -26/04/01 08:43:50 INFO Executor: Running task 90.0 in stage 17.0 (TID 624) -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:50 INFO Executor: Finished task 83.0 in stage 17.0 (TID 617). 6401 bytes result sent to driver -26/04/01 08:43:50 INFO TaskSetManager: Starting task 91.0 in stage 17.0 (TID 625) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:50 INFO TaskSetManager: Finished task 83.0 in stage 17.0 (TID 617) in 1856 ms on 10.0.0.133 (executor driver) (84/200) -26/04/01 08:43:50 INFO Executor: Running task 91.0 in stage 17.0 (TID 625) -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:51 INFO Executor: Finished task 84.0 in stage 17.0 (TID 618). 6401 bytes result sent to driver -26/04/01 08:43:51 INFO TaskSetManager: Starting task 92.0 in stage 17.0 (TID 626) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:51 INFO Executor: Running task 92.0 in stage 17.0 (TID 626) -26/04/01 08:43:51 INFO TaskSetManager: Finished task 84.0 in stage 17.0 (TID 618) in 1852 ms on 10.0.0.133 (executor driver) (85/200) -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:51 INFO Executor: Finished task 85.0 in stage 17.0 (TID 619). 6401 bytes result sent to driver -26/04/01 08:43:51 INFO TaskSetManager: Starting task 93.0 in stage 17.0 (TID 627) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:51 INFO Executor: Running task 93.0 in stage 17.0 (TID 627) -26/04/01 08:43:51 INFO TaskSetManager: Finished task 85.0 in stage 17.0 (TID 619) in 1864 ms on 10.0.0.133 (executor driver) (86/200) -26/04/01 08:43:51 INFO Executor: Finished task 86.0 in stage 17.0 (TID 620). 6401 bytes result sent to driver -26/04/01 08:43:51 INFO TaskSetManager: Starting task 94.0 in stage 17.0 (TID 628) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:51 INFO TaskSetManager: Finished task 86.0 in stage 17.0 (TID 620) in 1863 ms on 10.0.0.133 (executor driver) (87/200) -26/04/01 08:43:51 INFO Executor: Running task 94.0 in stage 17.0 (TID 628) -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:51 INFO Executor: Finished task 87.0 in stage 17.0 (TID 621). 6401 bytes result sent to driver -26/04/01 08:43:51 INFO TaskSetManager: Starting task 95.0 in stage 17.0 (TID 629) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:51 INFO Executor: Running task 95.0 in stage 17.0 (TID 629) -26/04/01 08:43:51 INFO TaskSetManager: Finished task 87.0 in stage 17.0 (TID 621) in 1850 ms on 10.0.0.133 (executor driver) (88/200) -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:52 INFO Executor: Finished task 89.0 in stage 17.0 (TID 623). 6401 bytes result sent to driver -26/04/01 08:43:52 INFO TaskSetManager: Starting task 96.0 in stage 17.0 (TID 630) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:52 INFO TaskSetManager: Finished task 89.0 in stage 17.0 (TID 623) in 1842 ms on 10.0.0.133 (executor driver) (89/200) -26/04/01 08:43:52 INFO Executor: Running task 96.0 in stage 17.0 (TID 630) -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:52 INFO Executor: Finished task 88.0 in stage 17.0 (TID 622). 6401 bytes result sent to driver -26/04/01 08:43:52 INFO TaskSetManager: Starting task 97.0 in stage 17.0 (TID 631) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:52 INFO TaskSetManager: Finished task 88.0 in stage 17.0 (TID 622) in 1860 ms on 10.0.0.133 (executor driver) (90/200) -26/04/01 08:43:52 INFO Executor: Running task 97.0 in stage 17.0 (TID 631) -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:52 INFO Executor: Finished task 90.0 in stage 17.0 (TID 624). 6401 bytes result sent to driver -26/04/01 08:43:52 INFO TaskSetManager: Starting task 98.0 in stage 17.0 (TID 632) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:52 INFO Executor: Running task 98.0 in stage 17.0 (TID 632) -26/04/01 08:43:52 INFO TaskSetManager: Finished task 90.0 in stage 17.0 (TID 624) in 1856 ms on 10.0.0.133 (executor driver) (91/200) -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:52 INFO Executor: Finished task 91.0 in stage 17.0 (TID 625). 6401 bytes result sent to driver -26/04/01 08:43:52 INFO TaskSetManager: Starting task 99.0 in stage 17.0 (TID 633) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:52 INFO Executor: Running task 99.0 in stage 17.0 (TID 633) -26/04/01 08:43:52 INFO TaskSetManager: Finished task 91.0 in stage 17.0 (TID 625) in 1850 ms on 10.0.0.133 (executor driver) (92/200) -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:53 INFO Executor: Finished task 92.0 in stage 17.0 (TID 626). 6401 bytes result sent to driver -26/04/01 08:43:53 INFO TaskSetManager: Starting task 100.0 in stage 17.0 (TID 634) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:53 INFO TaskSetManager: Finished task 92.0 in stage 17.0 (TID 626) in 1866 ms on 10.0.0.133 (executor driver) (93/200) -26/04/01 08:43:53 INFO Executor: Running task 100.0 in stage 17.0 (TID 634) -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:53 INFO Executor: Finished task 93.0 in stage 17.0 (TID 627). 6401 bytes result sent to driver -26/04/01 08:43:53 INFO TaskSetManager: Starting task 101.0 in stage 17.0 (TID 635) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:53 INFO TaskSetManager: Finished task 93.0 in stage 17.0 (TID 627) in 1861 ms on 10.0.0.133 (executor driver) (94/200) -26/04/01 08:43:53 INFO Executor: Running task 101.0 in stage 17.0 (TID 635) -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:53 INFO Executor: Finished task 94.0 in stage 17.0 (TID 628). 6401 bytes result sent to driver -26/04/01 08:43:53 INFO TaskSetManager: Starting task 102.0 in stage 17.0 (TID 636) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:53 INFO TaskSetManager: Finished task 94.0 in stage 17.0 (TID 628) in 1875 ms on 10.0.0.133 (executor driver) (95/200) -26/04/01 08:43:53 INFO Executor: Running task 102.0 in stage 17.0 (TID 636) -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:53 INFO Executor: Finished task 95.0 in stage 17.0 (TID 629). 6401 bytes result sent to driver -26/04/01 08:43:53 INFO TaskSetManager: Starting task 103.0 in stage 17.0 (TID 637) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:53 INFO TaskSetManager: Finished task 95.0 in stage 17.0 (TID 629) in 1860 ms on 10.0.0.133 (executor driver) (96/200) -26/04/01 08:43:53 INFO Executor: Running task 103.0 in stage 17.0 (TID 637) -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:54 INFO Executor: Finished task 96.0 in stage 17.0 (TID 630). 6401 bytes result sent to driver -26/04/01 08:43:54 INFO TaskSetManager: Starting task 104.0 in stage 17.0 (TID 638) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:54 INFO TaskSetManager: Finished task 96.0 in stage 17.0 (TID 630) in 1863 ms on 10.0.0.133 (executor driver) (97/200) -26/04/01 08:43:54 INFO Executor: Running task 104.0 in stage 17.0 (TID 638) -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:54 INFO Executor: Finished task 97.0 in stage 17.0 (TID 631). 6401 bytes result sent to driver -26/04/01 08:43:54 INFO TaskSetManager: Starting task 105.0 in stage 17.0 (TID 639) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:54 INFO TaskSetManager: Finished task 97.0 in stage 17.0 (TID 631) in 1861 ms on 10.0.0.133 (executor driver) (98/200) -26/04/01 08:43:54 INFO Executor: Running task 105.0 in stage 17.0 (TID 639) -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:54 INFO Executor: Finished task 99.0 in stage 17.0 (TID 633). 6401 bytes result sent to driver -26/04/01 08:43:54 INFO TaskSetManager: Starting task 106.0 in stage 17.0 (TID 640) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:54 INFO TaskSetManager: Finished task 99.0 in stage 17.0 (TID 633) in 1845 ms on 10.0.0.133 (executor driver) (99/200) -26/04/01 08:43:54 INFO Executor: Running task 106.0 in stage 17.0 (TID 640) -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:54 INFO Executor: Finished task 98.0 in stage 17.0 (TID 632). 6401 bytes result sent to driver -26/04/01 08:43:54 INFO TaskSetManager: Starting task 107.0 in stage 17.0 (TID 641) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:54 INFO TaskSetManager: Finished task 98.0 in stage 17.0 (TID 632) in 1853 ms on 10.0.0.133 (executor driver) (100/200) -26/04/01 08:43:54 INFO Executor: Running task 107.0 in stage 17.0 (TID 641) -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:54 INFO Executor: Finished task 100.0 in stage 17.0 (TID 634). 6401 bytes result sent to driver -26/04/01 08:43:54 INFO TaskSetManager: Starting task 108.0 in stage 17.0 (TID 642) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:54 INFO TaskSetManager: Finished task 100.0 in stage 17.0 (TID 634) in 1866 ms on 10.0.0.133 (executor driver) (101/200) -26/04/01 08:43:54 INFO Executor: Running task 108.0 in stage 17.0 (TID 642) -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:55 INFO Executor: Finished task 101.0 in stage 17.0 (TID 635). 6401 bytes result sent to driver -26/04/01 08:43:55 INFO TaskSetManager: Starting task 109.0 in stage 17.0 (TID 643) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:55 INFO Executor: Running task 109.0 in stage 17.0 (TID 643) -26/04/01 08:43:55 INFO TaskSetManager: Finished task 101.0 in stage 17.0 (TID 635) in 1859 ms on 10.0.0.133 (executor driver) (102/200) -26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:55 INFO Executor: Finished task 102.0 in stage 17.0 (TID 636). 6401 bytes result sent to driver -26/04/01 08:43:55 INFO TaskSetManager: Starting task 110.0 in stage 17.0 (TID 644) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:55 INFO Executor: Running task 110.0 in stage 17.0 (TID 644) -26/04/01 08:43:55 INFO TaskSetManager: Finished task 102.0 in stage 17.0 (TID 636) in 1858 ms on 10.0.0.133 (executor driver) (103/200) -26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:55 INFO Executor: Finished task 103.0 in stage 17.0 (TID 637). 6401 bytes result sent to driver -26/04/01 08:43:55 INFO TaskSetManager: Starting task 111.0 in stage 17.0 (TID 645) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:55 INFO Executor: Running task 111.0 in stage 17.0 (TID 645) -26/04/01 08:43:55 INFO TaskSetManager: Finished task 103.0 in stage 17.0 (TID 637) in 1850 ms on 10.0.0.133 (executor driver) (104/200) -26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:56 INFO Executor: Finished task 105.0 in stage 17.0 (TID 639). 6401 bytes result sent to driver -26/04/01 08:43:56 INFO TaskSetManager: Starting task 112.0 in stage 17.0 (TID 646) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:56 INFO TaskSetManager: Finished task 105.0 in stage 17.0 (TID 639) in 1855 ms on 10.0.0.133 (executor driver) (105/200) -26/04/01 08:43:56 INFO Executor: Running task 112.0 in stage 17.0 (TID 646) -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:56 INFO Executor: Finished task 104.0 in stage 17.0 (TID 638). 6401 bytes result sent to driver -26/04/01 08:43:56 INFO TaskSetManager: Starting task 113.0 in stage 17.0 (TID 647) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:56 INFO Executor: Running task 113.0 in stage 17.0 (TID 647) -26/04/01 08:43:56 INFO TaskSetManager: Finished task 104.0 in stage 17.0 (TID 638) in 1864 ms on 10.0.0.133 (executor driver) (106/200) -26/04/01 08:43:56 INFO Executor: Finished task 106.0 in stage 17.0 (TID 640). 6401 bytes result sent to driver -26/04/01 08:43:56 INFO TaskSetManager: Starting task 114.0 in stage 17.0 (TID 648) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:56 INFO TaskSetManager: Finished task 106.0 in stage 17.0 (TID 640) in 1844 ms on 10.0.0.133 (executor driver) (107/200) -26/04/01 08:43:56 INFO Executor: Running task 114.0 in stage 17.0 (TID 648) -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:56 INFO Executor: Finished task 107.0 in stage 17.0 (TID 641). 6401 bytes result sent to driver -26/04/01 08:43:56 INFO TaskSetManager: Starting task 115.0 in stage 17.0 (TID 649) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:56 INFO Executor: Running task 115.0 in stage 17.0 (TID 649) -26/04/01 08:43:56 INFO TaskSetManager: Finished task 107.0 in stage 17.0 (TID 641) in 1862 ms on 10.0.0.133 (executor driver) (108/200) -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:56 INFO Executor: Finished task 108.0 in stage 17.0 (TID 642). 6401 bytes result sent to driver -26/04/01 08:43:56 INFO TaskSetManager: Starting task 116.0 in stage 17.0 (TID 650) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:56 INFO TaskSetManager: Finished task 108.0 in stage 17.0 (TID 642) in 1857 ms on 10.0.0.133 (executor driver) (109/200) -26/04/01 08:43:56 INFO Executor: Running task 116.0 in stage 17.0 (TID 650) -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:56 INFO Executor: Finished task 109.0 in stage 17.0 (TID 643). 6401 bytes result sent to driver -26/04/01 08:43:56 INFO TaskSetManager: Starting task 117.0 in stage 17.0 (TID 651) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:56 INFO Executor: Running task 117.0 in stage 17.0 (TID 651) -26/04/01 08:43:56 INFO TaskSetManager: Finished task 109.0 in stage 17.0 (TID 643) in 1856 ms on 10.0.0.133 (executor driver) (110/200) -26/04/01 08:43:56 INFO Executor: Finished task 110.0 in stage 17.0 (TID 644). 6401 bytes result sent to driver -26/04/01 08:43:56 INFO TaskSetManager: Starting task 118.0 in stage 17.0 (TID 652) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:56 INFO Executor: Running task 118.0 in stage 17.0 (TID 652) -26/04/01 08:43:56 INFO TaskSetManager: Finished task 110.0 in stage 17.0 (TID 644) in 1843 ms on 10.0.0.133 (executor driver) (111/200) -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (76.0 MiB) non-empty blocks including 208 (76.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (76.0 MiB) non-empty blocks including 208 (76.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:57 INFO Executor: Finished task 111.0 in stage 17.0 (TID 645). 6401 bytes result sent to driver -26/04/01 08:43:57 INFO TaskSetManager: Starting task 119.0 in stage 17.0 (TID 653) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:57 INFO TaskSetManager: Finished task 111.0 in stage 17.0 (TID 645) in 1852 ms on 10.0.0.133 (executor driver) (112/200) -26/04/01 08:43:57 INFO Executor: Running task 119.0 in stage 17.0 (TID 653) -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:57 INFO Executor: Finished task 114.0 in stage 17.0 (TID 648). 6401 bytes result sent to driver -26/04/01 08:43:57 INFO TaskSetManager: Starting task 120.0 in stage 17.0 (TID 654) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:57 INFO TaskSetManager: Finished task 114.0 in stage 17.0 (TID 648) in 1858 ms on 10.0.0.133 (executor driver) (113/200) -26/04/01 08:43:57 INFO Executor: Running task 120.0 in stage 17.0 (TID 654) -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:57 INFO Executor: Finished task 113.0 in stage 17.0 (TID 647). 6401 bytes result sent to driver -26/04/01 08:43:57 INFO TaskSetManager: Starting task 121.0 in stage 17.0 (TID 655) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:57 INFO TaskSetManager: Finished task 113.0 in stage 17.0 (TID 647) in 1862 ms on 10.0.0.133 (executor driver) (114/200) -26/04/01 08:43:57 INFO Executor: Running task 121.0 in stage 17.0 (TID 655) -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:57 INFO Executor: Finished task 112.0 in stage 17.0 (TID 646). 6401 bytes result sent to driver -26/04/01 08:43:57 INFO TaskSetManager: Starting task 122.0 in stage 17.0 (TID 656) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:57 INFO Executor: Running task 122.0 in stage 17.0 (TID 656) -26/04/01 08:43:57 INFO TaskSetManager: Finished task 112.0 in stage 17.0 (TID 646) in 1865 ms on 10.0.0.133 (executor driver) (115/200) -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:57 INFO Executor: Finished task 115.0 in stage 17.0 (TID 649). 6401 bytes result sent to driver -26/04/01 08:43:57 INFO TaskSetManager: Starting task 123.0 in stage 17.0 (TID 657) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:57 INFO TaskSetManager: Finished task 115.0 in stage 17.0 (TID 649) in 1852 ms on 10.0.0.133 (executor driver) (116/200) -26/04/01 08:43:57 INFO Executor: Running task 123.0 in stage 17.0 (TID 657) -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:58 INFO Executor: Finished task 116.0 in stage 17.0 (TID 650). 7132 bytes result sent to driver -26/04/01 08:43:58 INFO TaskSetManager: Starting task 124.0 in stage 17.0 (TID 658) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:58 INFO TaskSetManager: Finished task 116.0 in stage 17.0 (TID 650) in 1859 ms on 10.0.0.133 (executor driver) (117/200) -26/04/01 08:43:58 INFO Executor: Running task 124.0 in stage 17.0 (TID 658) -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:58 INFO Executor: Finished task 117.0 in stage 17.0 (TID 651). 7132 bytes result sent to driver -26/04/01 08:43:58 INFO TaskSetManager: Starting task 125.0 in stage 17.0 (TID 659) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:58 INFO Executor: Running task 125.0 in stage 17.0 (TID 659) -26/04/01 08:43:58 INFO TaskSetManager: Finished task 117.0 in stage 17.0 (TID 651) in 1858 ms on 10.0.0.133 (executor driver) (118/200) -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:58 INFO Executor: Finished task 118.0 in stage 17.0 (TID 652). 7132 bytes result sent to driver -26/04/01 08:43:58 INFO TaskSetManager: Starting task 126.0 in stage 17.0 (TID 660) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:58 INFO Executor: Running task 126.0 in stage 17.0 (TID 660) -26/04/01 08:43:58 INFO TaskSetManager: Finished task 118.0 in stage 17.0 (TID 652) in 1883 ms on 10.0.0.133 (executor driver) (119/200) -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:58 INFO Executor: Finished task 119.0 in stage 17.0 (TID 653). 7132 bytes result sent to driver -26/04/01 08:43:58 INFO TaskSetManager: Starting task 127.0 in stage 17.0 (TID 661) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:58 INFO TaskSetManager: Finished task 119.0 in stage 17.0 (TID 653) in 1851 ms on 10.0.0.133 (executor driver) (120/200) -26/04/01 08:43:58 INFO Executor: Running task 127.0 in stage 17.0 (TID 661) -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:59 INFO Executor: Finished task 120.0 in stage 17.0 (TID 654). 7132 bytes result sent to driver -26/04/01 08:43:59 INFO TaskSetManager: Starting task 128.0 in stage 17.0 (TID 662) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:59 INFO TaskSetManager: Finished task 120.0 in stage 17.0 (TID 654) in 1868 ms on 10.0.0.133 (executor driver) (121/200) -26/04/01 08:43:59 INFO Executor: Running task 128.0 in stage 17.0 (TID 662) -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:59 INFO Executor: Finished task 121.0 in stage 17.0 (TID 655). 7132 bytes result sent to driver -26/04/01 08:43:59 INFO TaskSetManager: Starting task 129.0 in stage 17.0 (TID 663) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:59 INFO Executor: Running task 129.0 in stage 17.0 (TID 663) -26/04/01 08:43:59 INFO TaskSetManager: Finished task 121.0 in stage 17.0 (TID 655) in 1870 ms on 10.0.0.133 (executor driver) (122/200) -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:59 INFO Executor: Finished task 122.0 in stage 17.0 (TID 656). 7132 bytes result sent to driver -26/04/01 08:43:59 INFO TaskSetManager: Starting task 130.0 in stage 17.0 (TID 664) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:59 INFO TaskSetManager: Finished task 122.0 in stage 17.0 (TID 656) in 1875 ms on 10.0.0.133 (executor driver) (123/200) -26/04/01 08:43:59 INFO Executor: Running task 130.0 in stage 17.0 (TID 664) -26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:59 INFO Executor: Finished task 123.0 in stage 17.0 (TID 657). 7132 bytes result sent to driver -26/04/01 08:43:59 INFO TaskSetManager: Starting task 131.0 in stage 17.0 (TID 665) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9624 bytes) -26/04/01 08:43:59 INFO Executor: Running task 131.0 in stage 17.0 (TID 665) -26/04/01 08:43:59 INFO TaskSetManager: Finished task 123.0 in stage 17.0 (TID 657) in 1868 ms on 10.0.0.133 (executor driver) (124/200) -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:43:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:43:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:00 INFO Executor: Finished task 124.0 in stage 17.0 (TID 658). 6401 bytes result sent to driver -26/04/01 08:44:00 INFO TaskSetManager: Starting task 132.0 in stage 17.0 (TID 666) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:00 INFO Executor: Running task 132.0 in stage 17.0 (TID 666) -26/04/01 08:44:00 INFO TaskSetManager: Finished task 124.0 in stage 17.0 (TID 658) in 1864 ms on 10.0.0.133 (executor driver) (125/200) -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:00 INFO Executor: Finished task 126.0 in stage 17.0 (TID 660). 6401 bytes result sent to driver -26/04/01 08:44:00 INFO TaskSetManager: Starting task 133.0 in stage 17.0 (TID 667) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:00 INFO Executor: Finished task 125.0 in stage 17.0 (TID 659). 6401 bytes result sent to driver -26/04/01 08:44:00 INFO Executor: Running task 133.0 in stage 17.0 (TID 667) -26/04/01 08:44:00 INFO TaskSetManager: Starting task 134.0 in stage 17.0 (TID 668) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:00 INFO Executor: Running task 134.0 in stage 17.0 (TID 668) -26/04/01 08:44:00 INFO TaskSetManager: Finished task 126.0 in stage 17.0 (TID 660) in 1854 ms on 10.0.0.133 (executor driver) (126/200) -26/04/01 08:44:00 INFO TaskSetManager: Finished task 125.0 in stage 17.0 (TID 659) in 1880 ms on 10.0.0.133 (executor driver) (127/200) -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:00 INFO Executor: Finished task 127.0 in stage 17.0 (TID 661). 6401 bytes result sent to driver -26/04/01 08:44:00 INFO TaskSetManager: Starting task 135.0 in stage 17.0 (TID 669) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:00 INFO Executor: Running task 135.0 in stage 17.0 (TID 669) -26/04/01 08:44:00 INFO TaskSetManager: Finished task 127.0 in stage 17.0 (TID 661) in 1852 ms on 10.0.0.133 (executor driver) (128/200) -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:01 INFO Executor: Finished task 128.0 in stage 17.0 (TID 662). 6401 bytes result sent to driver -26/04/01 08:44:01 INFO TaskSetManager: Starting task 136.0 in stage 17.0 (TID 670) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:01 INFO TaskSetManager: Finished task 128.0 in stage 17.0 (TID 662) in 1873 ms on 10.0.0.133 (executor driver) (129/200) -26/04/01 08:44:01 INFO Executor: Running task 136.0 in stage 17.0 (TID 670) -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:01 INFO Executor: Finished task 129.0 in stage 17.0 (TID 663). 6401 bytes result sent to driver -26/04/01 08:44:01 INFO TaskSetManager: Starting task 137.0 in stage 17.0 (TID 671) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:01 INFO TaskSetManager: Finished task 129.0 in stage 17.0 (TID 663) in 1869 ms on 10.0.0.133 (executor driver) (130/200) -26/04/01 08:44:01 INFO Executor: Running task 137.0 in stage 17.0 (TID 671) -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:01 INFO Executor: Finished task 131.0 in stage 17.0 (TID 665). 6401 bytes result sent to driver -26/04/01 08:44:01 INFO TaskSetManager: Starting task 138.0 in stage 17.0 (TID 672) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:01 INFO Executor: Running task 138.0 in stage 17.0 (TID 672) -26/04/01 08:44:01 INFO TaskSetManager: Finished task 131.0 in stage 17.0 (TID 665) in 1873 ms on 10.0.0.133 (executor driver) (131/200) -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:01 INFO Executor: Finished task 130.0 in stage 17.0 (TID 664). 6401 bytes result sent to driver -26/04/01 08:44:01 INFO TaskSetManager: Starting task 139.0 in stage 17.0 (TID 673) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:01 INFO TaskSetManager: Finished task 130.0 in stage 17.0 (TID 664) in 1884 ms on 10.0.0.133 (executor driver) (132/200) -26/04/01 08:44:01 INFO Executor: Running task 139.0 in stage 17.0 (TID 673) -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:02 INFO Executor: Finished task 132.0 in stage 17.0 (TID 666). 6401 bytes result sent to driver -26/04/01 08:44:02 INFO TaskSetManager: Starting task 140.0 in stage 17.0 (TID 674) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:02 INFO TaskSetManager: Finished task 132.0 in stage 17.0 (TID 666) in 1865 ms on 10.0.0.133 (executor driver) (133/200) -26/04/01 08:44:02 INFO Executor: Running task 140.0 in stage 17.0 (TID 674) -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:02 INFO Executor: Finished task 133.0 in stage 17.0 (TID 667). 6401 bytes result sent to driver -26/04/01 08:44:02 INFO TaskSetManager: Starting task 141.0 in stage 17.0 (TID 675) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:02 INFO TaskSetManager: Finished task 133.0 in stage 17.0 (TID 667) in 1870 ms on 10.0.0.133 (executor driver) (134/200) -26/04/01 08:44:02 INFO Executor: Running task 141.0 in stage 17.0 (TID 675) -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:02 INFO Executor: Finished task 134.0 in stage 17.0 (TID 668). 6401 bytes result sent to driver -26/04/01 08:44:02 INFO TaskSetManager: Starting task 142.0 in stage 17.0 (TID 676) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:02 INFO TaskSetManager: Finished task 134.0 in stage 17.0 (TID 668) in 1881 ms on 10.0.0.133 (executor driver) (135/200) -26/04/01 08:44:02 INFO Executor: Running task 142.0 in stage 17.0 (TID 676) -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:02 INFO Executor: Finished task 135.0 in stage 17.0 (TID 669). 6401 bytes result sent to driver -26/04/01 08:44:02 INFO TaskSetManager: Starting task 143.0 in stage 17.0 (TID 677) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:02 INFO TaskSetManager: Finished task 135.0 in stage 17.0 (TID 669) in 1884 ms on 10.0.0.133 (executor driver) (136/200) -26/04/01 08:44:02 INFO Executor: Running task 143.0 in stage 17.0 (TID 677) -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:03 INFO Executor: Finished task 136.0 in stage 17.0 (TID 670). 6401 bytes result sent to driver -26/04/01 08:44:03 INFO TaskSetManager: Starting task 144.0 in stage 17.0 (TID 678) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:03 INFO TaskSetManager: Finished task 136.0 in stage 17.0 (TID 670) in 1888 ms on 10.0.0.133 (executor driver) (137/200) -26/04/01 08:44:03 INFO Executor: Running task 144.0 in stage 17.0 (TID 678) -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:03 INFO Executor: Finished task 138.0 in stage 17.0 (TID 672). 6401 bytes result sent to driver -26/04/01 08:44:03 INFO TaskSetManager: Starting task 145.0 in stage 17.0 (TID 679) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:03 INFO TaskSetManager: Finished task 138.0 in stage 17.0 (TID 672) in 1880 ms on 10.0.0.133 (executor driver) (138/200) -26/04/01 08:44:03 INFO Executor: Running task 145.0 in stage 17.0 (TID 679) -26/04/01 08:44:03 INFO Executor: Finished task 137.0 in stage 17.0 (TID 671). 6401 bytes result sent to driver -26/04/01 08:44:03 INFO TaskSetManager: Starting task 146.0 in stage 17.0 (TID 680) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:03 INFO TaskSetManager: Finished task 137.0 in stage 17.0 (TID 671) in 1892 ms on 10.0.0.133 (executor driver) (139/200) -26/04/01 08:44:03 INFO Executor: Running task 146.0 in stage 17.0 (TID 680) -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:03 INFO Executor: Finished task 139.0 in stage 17.0 (TID 673). 6401 bytes result sent to driver -26/04/01 08:44:03 INFO TaskSetManager: Starting task 147.0 in stage 17.0 (TID 681) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:03 INFO Executor: Running task 147.0 in stage 17.0 (TID 681) -26/04/01 08:44:03 INFO TaskSetManager: Finished task 139.0 in stage 17.0 (TID 673) in 1887 ms on 10.0.0.133 (executor driver) (140/200) -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:04 INFO Executor: Finished task 140.0 in stage 17.0 (TID 674). 6401 bytes result sent to driver -26/04/01 08:44:04 INFO TaskSetManager: Starting task 148.0 in stage 17.0 (TID 682) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:04 INFO Executor: Running task 148.0 in stage 17.0 (TID 682) -26/04/01 08:44:04 INFO TaskSetManager: Finished task 140.0 in stage 17.0 (TID 674) in 1875 ms on 10.0.0.133 (executor driver) (141/200) -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:04 INFO Executor: Finished task 141.0 in stage 17.0 (TID 675). 6401 bytes result sent to driver -26/04/01 08:44:04 INFO TaskSetManager: Starting task 149.0 in stage 17.0 (TID 683) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:04 INFO Executor: Running task 149.0 in stage 17.0 (TID 683) -26/04/01 08:44:04 INFO TaskSetManager: Finished task 141.0 in stage 17.0 (TID 675) in 1874 ms on 10.0.0.133 (executor driver) (142/200) -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:04 INFO Executor: Finished task 143.0 in stage 17.0 (TID 677). 6401 bytes result sent to driver -26/04/01 08:44:04 INFO TaskSetManager: Starting task 150.0 in stage 17.0 (TID 684) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:04 INFO Executor: Running task 150.0 in stage 17.0 (TID 684) -26/04/01 08:44:04 INFO TaskSetManager: Finished task 143.0 in stage 17.0 (TID 677) in 1863 ms on 10.0.0.133 (executor driver) (143/200) -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:04 INFO Executor: Finished task 142.0 in stage 17.0 (TID 676). 6401 bytes result sent to driver -26/04/01 08:44:04 INFO TaskSetManager: Starting task 151.0 in stage 17.0 (TID 685) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:04 INFO Executor: Running task 151.0 in stage 17.0 (TID 685) -26/04/01 08:44:04 INFO TaskSetManager: Finished task 142.0 in stage 17.0 (TID 676) in 1883 ms on 10.0.0.133 (executor driver) (144/200) -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:05 INFO Executor: Finished task 144.0 in stage 17.0 (TID 678). 6401 bytes result sent to driver -26/04/01 08:44:05 INFO TaskSetManager: Starting task 152.0 in stage 17.0 (TID 686) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:05 INFO Executor: Running task 152.0 in stage 17.0 (TID 686) -26/04/01 08:44:05 INFO TaskSetManager: Finished task 144.0 in stage 17.0 (TID 678) in 1851 ms on 10.0.0.133 (executor driver) (145/200) -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:05 INFO Executor: Finished task 145.0 in stage 17.0 (TID 679). 6401 bytes result sent to driver -26/04/01 08:44:05 INFO TaskSetManager: Starting task 153.0 in stage 17.0 (TID 687) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:05 INFO Executor: Running task 153.0 in stage 17.0 (TID 687) -26/04/01 08:44:05 INFO TaskSetManager: Finished task 145.0 in stage 17.0 (TID 679) in 1856 ms on 10.0.0.133 (executor driver) (146/200) -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:05 INFO Executor: Finished task 146.0 in stage 17.0 (TID 680). 6401 bytes result sent to driver -26/04/01 08:44:05 INFO TaskSetManager: Starting task 154.0 in stage 17.0 (TID 688) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:05 INFO Executor: Running task 154.0 in stage 17.0 (TID 688) -26/04/01 08:44:05 INFO TaskSetManager: Finished task 146.0 in stage 17.0 (TID 680) in 1858 ms on 10.0.0.133 (executor driver) (147/200) -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:05 INFO Executor: Finished task 147.0 in stage 17.0 (TID 681). 6401 bytes result sent to driver -26/04/01 08:44:05 INFO TaskSetManager: Starting task 155.0 in stage 17.0 (TID 689) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:05 INFO TaskSetManager: Finished task 147.0 in stage 17.0 (TID 681) in 1861 ms on 10.0.0.133 (executor driver) (148/200) -26/04/01 08:44:05 INFO Executor: Running task 155.0 in stage 17.0 (TID 689) -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:06 INFO Executor: Finished task 148.0 in stage 17.0 (TID 682). 6401 bytes result sent to driver -26/04/01 08:44:06 INFO TaskSetManager: Starting task 156.0 in stage 17.0 (TID 690) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:06 INFO TaskSetManager: Finished task 148.0 in stage 17.0 (TID 682) in 1859 ms on 10.0.0.133 (executor driver) (149/200) -26/04/01 08:44:06 INFO Executor: Running task 156.0 in stage 17.0 (TID 690) -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:06 INFO Executor: Finished task 149.0 in stage 17.0 (TID 683). 6401 bytes result sent to driver -26/04/01 08:44:06 INFO TaskSetManager: Starting task 157.0 in stage 17.0 (TID 691) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:06 INFO TaskSetManager: Finished task 149.0 in stage 17.0 (TID 683) in 1877 ms on 10.0.0.133 (executor driver) (150/200) -26/04/01 08:44:06 INFO Executor: Running task 157.0 in stage 17.0 (TID 691) -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:06 INFO Executor: Finished task 150.0 in stage 17.0 (TID 684). 6401 bytes result sent to driver -26/04/01 08:44:06 INFO TaskSetManager: Starting task 158.0 in stage 17.0 (TID 692) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:06 INFO TaskSetManager: Finished task 150.0 in stage 17.0 (TID 684) in 1862 ms on 10.0.0.133 (executor driver) (151/200) -26/04/01 08:44:06 INFO Executor: Running task 158.0 in stage 17.0 (TID 692) -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:06 INFO Executor: Finished task 151.0 in stage 17.0 (TID 685). 6401 bytes result sent to driver -26/04/01 08:44:06 INFO TaskSetManager: Starting task 159.0 in stage 17.0 (TID 693) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:06 INFO Executor: Running task 159.0 in stage 17.0 (TID 693) -26/04/01 08:44:06 INFO TaskSetManager: Finished task 151.0 in stage 17.0 (TID 685) in 1866 ms on 10.0.0.133 (executor driver) (152/200) -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:07 INFO Executor: Finished task 152.0 in stage 17.0 (TID 686). 6401 bytes result sent to driver -26/04/01 08:44:07 INFO TaskSetManager: Starting task 160.0 in stage 17.0 (TID 694) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:07 INFO Executor: Running task 160.0 in stage 17.0 (TID 694) -26/04/01 08:44:07 INFO TaskSetManager: Finished task 152.0 in stage 17.0 (TID 686) in 1859 ms on 10.0.0.133 (executor driver) (153/200) -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:07 INFO Executor: Finished task 154.0 in stage 17.0 (TID 688). 6401 bytes result sent to driver -26/04/01 08:44:07 INFO TaskSetManager: Starting task 161.0 in stage 17.0 (TID 695) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:07 INFO Executor: Finished task 153.0 in stage 17.0 (TID 687). 6401 bytes result sent to driver -26/04/01 08:44:07 INFO Executor: Running task 161.0 in stage 17.0 (TID 695) -26/04/01 08:44:07 INFO TaskSetManager: Finished task 154.0 in stage 17.0 (TID 688) in 1858 ms on 10.0.0.133 (executor driver) (154/200) -26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:07 INFO TaskSetManager: Starting task 162.0 in stage 17.0 (TID 696) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:07 INFO TaskSetManager: Finished task 153.0 in stage 17.0 (TID 687) in 1861 ms on 10.0.0.133 (executor driver) (155/200) -26/04/01 08:44:07 INFO Executor: Running task 162.0 in stage 17.0 (TID 696) -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:07 INFO Executor: Finished task 155.0 in stage 17.0 (TID 689). 6401 bytes result sent to driver -26/04/01 08:44:07 INFO TaskSetManager: Starting task 163.0 in stage 17.0 (TID 697) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:07 INFO TaskSetManager: Finished task 155.0 in stage 17.0 (TID 689) in 1847 ms on 10.0.0.133 (executor driver) (156/200) -26/04/01 08:44:07 INFO Executor: Running task 163.0 in stage 17.0 (TID 697) -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:08 INFO Executor: Finished task 156.0 in stage 17.0 (TID 690). 6401 bytes result sent to driver -26/04/01 08:44:08 INFO TaskSetManager: Starting task 164.0 in stage 17.0 (TID 698) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:08 INFO TaskSetManager: Finished task 156.0 in stage 17.0 (TID 690) in 1870 ms on 10.0.0.133 (executor driver) (157/200) -26/04/01 08:44:08 INFO Executor: Running task 164.0 in stage 17.0 (TID 698) -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:08 INFO Executor: Finished task 157.0 in stage 17.0 (TID 691). 6401 bytes result sent to driver -26/04/01 08:44:08 INFO TaskSetManager: Starting task 165.0 in stage 17.0 (TID 699) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:08 INFO TaskSetManager: Finished task 157.0 in stage 17.0 (TID 691) in 1856 ms on 10.0.0.133 (executor driver) (158/200) -26/04/01 08:44:08 INFO Executor: Running task 165.0 in stage 17.0 (TID 699) -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:08 INFO Executor: Finished task 158.0 in stage 17.0 (TID 692). 6401 bytes result sent to driver -26/04/01 08:44:08 INFO TaskSetManager: Starting task 166.0 in stage 17.0 (TID 700) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:08 INFO TaskSetManager: Finished task 158.0 in stage 17.0 (TID 692) in 1875 ms on 10.0.0.133 (executor driver) (159/200) -26/04/01 08:44:08 INFO Executor: Running task 166.0 in stage 17.0 (TID 700) -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:08 INFO Executor: Finished task 159.0 in stage 17.0 (TID 693). 6401 bytes result sent to driver -26/04/01 08:44:08 INFO TaskSetManager: Starting task 167.0 in stage 17.0 (TID 701) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:08 INFO TaskSetManager: Finished task 159.0 in stage 17.0 (TID 693) in 1878 ms on 10.0.0.133 (executor driver) (160/200) -26/04/01 08:44:08 INFO Executor: Running task 167.0 in stage 17.0 (TID 701) -26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:09 INFO Executor: Finished task 161.0 in stage 17.0 (TID 695). 7132 bytes result sent to driver -26/04/01 08:44:09 INFO TaskSetManager: Starting task 168.0 in stage 17.0 (TID 702) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:09 INFO Executor: Running task 168.0 in stage 17.0 (TID 702) -26/04/01 08:44:09 INFO TaskSetManager: Finished task 161.0 in stage 17.0 (TID 695) in 1864 ms on 10.0.0.133 (executor driver) (161/200) -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:09 INFO Executor: Finished task 160.0 in stage 17.0 (TID 694). 7132 bytes result sent to driver -26/04/01 08:44:09 INFO TaskSetManager: Starting task 169.0 in stage 17.0 (TID 703) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:09 INFO Executor: Running task 169.0 in stage 17.0 (TID 703) -26/04/01 08:44:09 INFO TaskSetManager: Finished task 160.0 in stage 17.0 (TID 694) in 1888 ms on 10.0.0.133 (executor driver) (162/200) -26/04/01 08:44:09 INFO Executor: Finished task 162.0 in stage 17.0 (TID 696). 7132 bytes result sent to driver -26/04/01 08:44:09 INFO TaskSetManager: Starting task 170.0 in stage 17.0 (TID 704) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:09 INFO Executor: Running task 170.0 in stage 17.0 (TID 704) -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:09 INFO TaskSetManager: Finished task 162.0 in stage 17.0 (TID 696) in 1878 ms on 10.0.0.133 (executor driver) (163/200) -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:09 INFO Executor: Finished task 163.0 in stage 17.0 (TID 697). 7132 bytes result sent to driver -26/04/01 08:44:09 INFO TaskSetManager: Starting task 171.0 in stage 17.0 (TID 705) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:09 INFO Executor: Running task 171.0 in stage 17.0 (TID 705) -26/04/01 08:44:09 INFO TaskSetManager: Finished task 163.0 in stage 17.0 (TID 697) in 1876 ms on 10.0.0.133 (executor driver) (164/200) -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:09 INFO Executor: Finished task 164.0 in stage 17.0 (TID 698). 7132 bytes result sent to driver -26/04/01 08:44:09 INFO TaskSetManager: Starting task 172.0 in stage 17.0 (TID 706) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:09 INFO TaskSetManager: Finished task 164.0 in stage 17.0 (TID 698) in 1878 ms on 10.0.0.133 (executor driver) (165/200) -26/04/01 08:44:09 INFO Executor: Running task 172.0 in stage 17.0 (TID 706) -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:10 INFO Executor: Finished task 165.0 in stage 17.0 (TID 699). 7132 bytes result sent to driver -26/04/01 08:44:10 INFO TaskSetManager: Starting task 173.0 in stage 17.0 (TID 707) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:10 INFO TaskSetManager: Finished task 165.0 in stage 17.0 (TID 699) in 1868 ms on 10.0.0.133 (executor driver) (166/200) -26/04/01 08:44:10 INFO Executor: Running task 173.0 in stage 17.0 (TID 707) -26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:10 INFO Executor: Finished task 167.0 in stage 17.0 (TID 701). 7132 bytes result sent to driver -26/04/01 08:44:10 INFO TaskSetManager: Starting task 174.0 in stage 17.0 (TID 708) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:10 INFO TaskSetManager: Finished task 167.0 in stage 17.0 (TID 701) in 1851 ms on 10.0.0.133 (executor driver) (167/200) -26/04/01 08:44:10 INFO Executor: Running task 174.0 in stage 17.0 (TID 708) -26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:10 INFO Executor: Finished task 166.0 in stage 17.0 (TID 700). 7132 bytes result sent to driver -26/04/01 08:44:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:10 INFO TaskSetManager: Starting task 175.0 in stage 17.0 (TID 709) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:10 INFO TaskSetManager: Finished task 166.0 in stage 17.0 (TID 700) in 1871 ms on 10.0.0.133 (executor driver) (168/200) -26/04/01 08:44:10 INFO Executor: Running task 175.0 in stage 17.0 (TID 709) -26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO Executor: Finished task 168.0 in stage 17.0 (TID 702). 6401 bytes result sent to driver -26/04/01 08:44:11 INFO TaskSetManager: Starting task 176.0 in stage 17.0 (TID 710) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:11 INFO TaskSetManager: Finished task 168.0 in stage 17.0 (TID 702) in 1855 ms on 10.0.0.133 (executor driver) (169/200) -26/04/01 08:44:11 INFO Executor: Running task 176.0 in stage 17.0 (TID 710) -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO Executor: Finished task 170.0 in stage 17.0 (TID 704). 6401 bytes result sent to driver -26/04/01 08:44:11 INFO TaskSetManager: Starting task 177.0 in stage 17.0 (TID 711) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:11 INFO Executor: Running task 177.0 in stage 17.0 (TID 711) -26/04/01 08:44:11 INFO TaskSetManager: Finished task 170.0 in stage 17.0 (TID 704) in 1871 ms on 10.0.0.133 (executor driver) (170/200) -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO Executor: Finished task 169.0 in stage 17.0 (TID 703). 6401 bytes result sent to driver -26/04/01 08:44:11 INFO TaskSetManager: Starting task 178.0 in stage 17.0 (TID 712) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:11 INFO TaskSetManager: Finished task 169.0 in stage 17.0 (TID 703) in 1873 ms on 10.0.0.133 (executor driver) (171/200) -26/04/01 08:44:11 INFO Executor: Running task 178.0 in stage 17.0 (TID 712) -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO Executor: Finished task 171.0 in stage 17.0 (TID 705). 6401 bytes result sent to driver -26/04/01 08:44:11 INFO TaskSetManager: Starting task 179.0 in stage 17.0 (TID 713) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:11 INFO TaskSetManager: Finished task 171.0 in stage 17.0 (TID 705) in 1872 ms on 10.0.0.133 (executor driver) (172/200) -26/04/01 08:44:11 INFO Executor: Running task 179.0 in stage 17.0 (TID 713) -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO Executor: Finished task 172.0 in stage 17.0 (TID 706). 6401 bytes result sent to driver -26/04/01 08:44:11 INFO TaskSetManager: Starting task 180.0 in stage 17.0 (TID 714) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:11 INFO TaskSetManager: Finished task 172.0 in stage 17.0 (TID 706) in 1852 ms on 10.0.0.133 (executor driver) (173/200) -26/04/01 08:44:11 INFO Executor: Running task 180.0 in stage 17.0 (TID 714) -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO Executor: Finished task 173.0 in stage 17.0 (TID 707). 6401 bytes result sent to driver -26/04/01 08:44:11 INFO TaskSetManager: Starting task 181.0 in stage 17.0 (TID 715) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:11 INFO Executor: Running task 181.0 in stage 17.0 (TID 715) -26/04/01 08:44:11 INFO TaskSetManager: Finished task 173.0 in stage 17.0 (TID 707) in 1852 ms on 10.0.0.133 (executor driver) (174/200) -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO Executor: Finished task 174.0 in stage 17.0 (TID 708). 6401 bytes result sent to driver -26/04/01 08:44:11 INFO TaskSetManager: Starting task 182.0 in stage 17.0 (TID 716) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:11 INFO TaskSetManager: Finished task 174.0 in stage 17.0 (TID 708) in 1843 ms on 10.0.0.133 (executor driver) (175/200) -26/04/01 08:44:11 INFO Executor: Running task 182.0 in stage 17.0 (TID 716) -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO Executor: Finished task 175.0 in stage 17.0 (TID 709). 6401 bytes result sent to driver -26/04/01 08:44:11 INFO TaskSetManager: Starting task 183.0 in stage 17.0 (TID 717) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:11 INFO TaskSetManager: Finished task 175.0 in stage 17.0 (TID 709) in 1850 ms on 10.0.0.133 (executor driver) (176/200) -26/04/01 08:44:11 INFO Executor: Running task 183.0 in stage 17.0 (TID 717) -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:12 INFO Executor: Finished task 176.0 in stage 17.0 (TID 710). 6401 bytes result sent to driver -26/04/01 08:44:12 INFO TaskSetManager: Starting task 184.0 in stage 17.0 (TID 718) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:12 INFO TaskSetManager: Finished task 176.0 in stage 17.0 (TID 710) in 1833 ms on 10.0.0.133 (executor driver) (177/200) -26/04/01 08:44:12 INFO Executor: Running task 184.0 in stage 17.0 (TID 718) -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:12 INFO Executor: Finished task 177.0 in stage 17.0 (TID 711). 6401 bytes result sent to driver -26/04/01 08:44:12 INFO TaskSetManager: Starting task 185.0 in stage 17.0 (TID 719) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:12 INFO TaskSetManager: Finished task 177.0 in stage 17.0 (TID 711) in 1858 ms on 10.0.0.133 (executor driver) (178/200) -26/04/01 08:44:12 INFO Executor: Running task 185.0 in stage 17.0 (TID 719) -26/04/01 08:44:12 INFO Executor: Finished task 178.0 in stage 17.0 (TID 712). 6401 bytes result sent to driver -26/04/01 08:44:12 INFO TaskSetManager: Starting task 186.0 in stage 17.0 (TID 720) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:12 INFO Executor: Running task 186.0 in stage 17.0 (TID 720) -26/04/01 08:44:12 INFO TaskSetManager: Finished task 178.0 in stage 17.0 (TID 712) in 1858 ms on 10.0.0.133 (executor driver) (179/200) -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (77.0 MiB) non-empty blocks including 208 (77.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (77.0 MiB) non-empty blocks including 208 (77.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:12 INFO Executor: Finished task 179.0 in stage 17.0 (TID 713). 6401 bytes result sent to driver -26/04/01 08:44:12 INFO TaskSetManager: Starting task 187.0 in stage 17.0 (TID 721) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:12 INFO Executor: Running task 187.0 in stage 17.0 (TID 721) -26/04/01 08:44:12 INFO TaskSetManager: Finished task 179.0 in stage 17.0 (TID 713) in 1867 ms on 10.0.0.133 (executor driver) (180/200) -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:13 INFO Executor: Finished task 180.0 in stage 17.0 (TID 714). 6401 bytes result sent to driver -26/04/01 08:44:13 INFO TaskSetManager: Starting task 188.0 in stage 17.0 (TID 722) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:13 INFO TaskSetManager: Finished task 180.0 in stage 17.0 (TID 714) in 1848 ms on 10.0.0.133 (executor driver) (181/200) -26/04/01 08:44:13 INFO Executor: Running task 188.0 in stage 17.0 (TID 722) -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:13 INFO Executor: Finished task 181.0 in stage 17.0 (TID 715). 6401 bytes result sent to driver -26/04/01 08:44:13 INFO TaskSetManager: Starting task 189.0 in stage 17.0 (TID 723) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:13 INFO Executor: Running task 189.0 in stage 17.0 (TID 723) -26/04/01 08:44:13 INFO TaskSetManager: Finished task 181.0 in stage 17.0 (TID 715) in 1883 ms on 10.0.0.133 (executor driver) (182/200) -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:13 INFO Executor: Finished task 182.0 in stage 17.0 (TID 716). 6401 bytes result sent to driver -26/04/01 08:44:13 INFO TaskSetManager: Starting task 190.0 in stage 17.0 (TID 724) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:13 INFO Executor: Running task 190.0 in stage 17.0 (TID 724) -26/04/01 08:44:13 INFO TaskSetManager: Finished task 182.0 in stage 17.0 (TID 716) in 1881 ms on 10.0.0.133 (executor driver) (183/200) -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:13 INFO Executor: Finished task 183.0 in stage 17.0 (TID 717). 6401 bytes result sent to driver -26/04/01 08:44:13 INFO TaskSetManager: Starting task 191.0 in stage 17.0 (TID 725) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:13 INFO TaskSetManager: Finished task 183.0 in stage 17.0 (TID 717) in 1874 ms on 10.0.0.133 (executor driver) (184/200) -26/04/01 08:44:13 INFO Executor: Running task 191.0 in stage 17.0 (TID 725) -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:14 INFO Executor: Finished task 184.0 in stage 17.0 (TID 718). 6401 bytes result sent to driver -26/04/01 08:44:14 INFO TaskSetManager: Starting task 192.0 in stage 17.0 (TID 726) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:14 INFO Executor: Running task 192.0 in stage 17.0 (TID 726) -26/04/01 08:44:14 INFO TaskSetManager: Finished task 184.0 in stage 17.0 (TID 718) in 1873 ms on 10.0.0.133 (executor driver) (185/200) -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:14 INFO Executor: Finished task 186.0 in stage 17.0 (TID 720). 6401 bytes result sent to driver -26/04/01 08:44:14 INFO TaskSetManager: Starting task 193.0 in stage 17.0 (TID 727) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:14 INFO Executor: Running task 193.0 in stage 17.0 (TID 727) -26/04/01 08:44:14 INFO TaskSetManager: Finished task 186.0 in stage 17.0 (TID 720) in 1881 ms on 10.0.0.133 (executor driver) (186/200) -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:14 INFO Executor: Finished task 185.0 in stage 17.0 (TID 719). 6401 bytes result sent to driver -26/04/01 08:44:14 INFO TaskSetManager: Starting task 194.0 in stage 17.0 (TID 728) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:14 INFO Executor: Running task 194.0 in stage 17.0 (TID 728) -26/04/01 08:44:14 INFO TaskSetManager: Finished task 185.0 in stage 17.0 (TID 719) in 1894 ms on 10.0.0.133 (executor driver) (187/200) -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:14 INFO Executor: Finished task 187.0 in stage 17.0 (TID 721). 6401 bytes result sent to driver -26/04/01 08:44:14 INFO TaskSetManager: Starting task 195.0 in stage 17.0 (TID 729) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:14 INFO TaskSetManager: Finished task 187.0 in stage 17.0 (TID 721) in 1907 ms on 10.0.0.133 (executor driver) (188/200) -26/04/01 08:44:14 INFO Executor: Running task 195.0 in stage 17.0 (TID 729) -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:15 INFO Executor: Finished task 188.0 in stage 17.0 (TID 722). 6401 bytes result sent to driver -26/04/01 08:44:15 INFO TaskSetManager: Starting task 196.0 in stage 17.0 (TID 730) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:15 INFO Executor: Running task 196.0 in stage 17.0 (TID 730) -26/04/01 08:44:15 INFO TaskSetManager: Finished task 188.0 in stage 17.0 (TID 722) in 1916 ms on 10.0.0.133 (executor driver) (189/200) -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:15 INFO Executor: Finished task 189.0 in stage 17.0 (TID 723). 6401 bytes result sent to driver -26/04/01 08:44:15 INFO TaskSetManager: Starting task 197.0 in stage 17.0 (TID 731) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:15 INFO TaskSetManager: Finished task 189.0 in stage 17.0 (TID 723) in 1899 ms on 10.0.0.133 (executor driver) (190/200) -26/04/01 08:44:15 INFO Executor: Running task 197.0 in stage 17.0 (TID 731) -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:15 INFO Executor: Finished task 190.0 in stage 17.0 (TID 724). 6401 bytes result sent to driver -26/04/01 08:44:15 INFO TaskSetManager: Starting task 198.0 in stage 17.0 (TID 732) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:15 INFO Executor: Running task 198.0 in stage 17.0 (TID 732) -26/04/01 08:44:15 INFO TaskSetManager: Finished task 190.0 in stage 17.0 (TID 724) in 1900 ms on 10.0.0.133 (executor driver) (191/200) -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:15 INFO Executor: Finished task 191.0 in stage 17.0 (TID 725). 6401 bytes result sent to driver -26/04/01 08:44:15 INFO TaskSetManager: Starting task 199.0 in stage 17.0 (TID 733) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9624 bytes) -26/04/01 08:44:15 INFO TaskSetManager: Finished task 191.0 in stage 17.0 (TID 725) in 1898 ms on 10.0.0.133 (executor driver) (192/200) -26/04/01 08:44:15 INFO Executor: Running task 199.0 in stage 17.0 (TID 733) -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:16 INFO Executor: Finished task 192.0 in stage 17.0 (TID 726). 6401 bytes result sent to driver -26/04/01 08:44:16 INFO TaskSetManager: Finished task 192.0 in stage 17.0 (TID 726) in 1870 ms on 10.0.0.133 (executor driver) (193/200) -26/04/01 08:44:16 INFO Executor: Finished task 194.0 in stage 17.0 (TID 728). 6401 bytes result sent to driver -26/04/01 08:44:16 INFO TaskSetManager: Finished task 194.0 in stage 17.0 (TID 728) in 1860 ms on 10.0.0.133 (executor driver) (194/200) -26/04/01 08:44:16 INFO Executor: Finished task 193.0 in stage 17.0 (TID 727). 6401 bytes result sent to driver -26/04/01 08:44:16 INFO TaskSetManager: Finished task 193.0 in stage 17.0 (TID 727) in 1874 ms on 10.0.0.133 (executor driver) (195/200) -26/04/01 08:44:16 INFO Executor: Finished task 195.0 in stage 17.0 (TID 729). 6487 bytes result sent to driver -26/04/01 08:44:16 INFO TaskSetManager: Finished task 195.0 in stage 17.0 (TID 729) in 1870 ms on 10.0.0.133 (executor driver) (196/200) -26/04/01 08:44:17 INFO Executor: Finished task 196.0 in stage 17.0 (TID 730). 6444 bytes result sent to driver -26/04/01 08:44:17 INFO TaskSetManager: Finished task 196.0 in stage 17.0 (TID 730) in 1865 ms on 10.0.0.133 (executor driver) (197/200) -26/04/01 08:44:17 INFO Executor: Finished task 197.0 in stage 17.0 (TID 731). 6444 bytes result sent to driver -26/04/01 08:44:17 INFO TaskSetManager: Finished task 197.0 in stage 17.0 (TID 731) in 1867 ms on 10.0.0.133 (executor driver) (198/200) -26/04/01 08:44:17 INFO Executor: Finished task 198.0 in stage 17.0 (TID 732). 6444 bytes result sent to driver -26/04/01 08:44:17 INFO TaskSetManager: Finished task 198.0 in stage 17.0 (TID 732) in 1864 ms on 10.0.0.133 (executor driver) (199/200) -26/04/01 08:44:17 INFO Executor: Finished task 199.0 in stage 17.0 (TID 733). 6444 bytes result sent to driver -26/04/01 08:44:17 INFO TaskSetManager: Finished task 199.0 in stage 17.0 (TID 733) in 1863 ms on 10.0.0.133 (executor driver) (200/200) -26/04/01 08:44:17 INFO TaskSchedulerImpl: Removed TaskSet 17.0, whose tasks have all completed, from pool -26/04/01 08:44:17 INFO DAGScheduler: ShuffleMapStage 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 47.905 s -26/04/01 08:44:17 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:44:17 INFO DAGScheduler: running: Set() -26/04/01 08:44:17 INFO DAGScheduler: waiting: Set() -26/04/01 08:44:17 INFO DAGScheduler: failed: Set() -26/04/01 08:44:17 INFO ShufflePartitionsUtil: For shuffle(5, 2), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 08:44:17 INFO DAGScheduler: Registering RDD 43 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 6 -26/04/01 08:44:17 INFO DAGScheduler: Got map stage job 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 14 output partitions -26/04/01 08:44:17 INFO DAGScheduler: Final stage: ShuffleMapStage 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:44:17 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 19, ShuffleMapStage 20) -26/04/01 08:44:17 INFO DAGScheduler: Missing parents: List() -26/04/01 08:44:17 INFO DAGScheduler: Submitting ShuffleMapStage 21 (MapPartitionsRDD[43] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:44:17 INFO MemoryStore: Block broadcast_24 stored as values in memory (estimated size 24.1 KiB, free 8.6 GiB) -26/04/01 08:44:17 INFO MemoryStore: Block broadcast_24_piece0 stored as bytes in memory (estimated size 9.8 KiB, free 8.6 GiB) -26/04/01 08:44:17 INFO BlockManagerInfo: Added broadcast_24_piece0 in memory on 10.0.0.133:59303 (size: 9.8 KiB, free: 8.6 GiB) -26/04/01 08:44:17 INFO SparkContext: Created broadcast 24 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:44:17 INFO DAGScheduler: Submitting 14 missing tasks from ShuffleMapStage 21 (MapPartitionsRDD[43] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)) -26/04/01 08:44:17 INFO TaskSchedulerImpl: Adding task set 21.0 with 14 tasks resource profile 0 -26/04/01 08:44:17 INFO TaskSetManager: Starting task 0.0 in stage 21.0 (TID 734) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:17 INFO TaskSetManager: Starting task 1.0 in stage 21.0 (TID 735) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:17 INFO TaskSetManager: Starting task 2.0 in stage 21.0 (TID 736) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:17 INFO TaskSetManager: Starting task 3.0 in stage 21.0 (TID 737) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:17 INFO TaskSetManager: Starting task 4.0 in stage 21.0 (TID 738) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:17 INFO TaskSetManager: Starting task 5.0 in stage 21.0 (TID 739) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:17 INFO TaskSetManager: Starting task 6.0 in stage 21.0 (TID 740) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:17 INFO TaskSetManager: Starting task 7.0 in stage 21.0 (TID 741) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:17 INFO Executor: Running task 1.0 in stage 21.0 (TID 735) -26/04/01 08:44:17 INFO Executor: Running task 2.0 in stage 21.0 (TID 736) -26/04/01 08:44:17 INFO Executor: Running task 7.0 in stage 21.0 (TID 741) -26/04/01 08:44:17 INFO Executor: Running task 0.0 in stage 21.0 (TID 734) -26/04/01 08:44:17 INFO Executor: Running task 5.0 in stage 21.0 (TID 739) -26/04/01 08:44:17 INFO Executor: Running task 3.0 in stage 21.0 (TID 737) -26/04/01 08:44:17 INFO Executor: Running task 6.0 in stage 21.0 (TID 740) -26/04/01 08:44:17 INFO Executor: Running task 4.0 in stage 21.0 (TID 738) -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.0 MiB) non-empty blocks including 200 (61.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.3 MiB) non-empty blocks including 200 (61.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.6 MiB) non-empty blocks including 200 (61.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (618.3 KiB) non-empty blocks including 8 (618.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (619.7 KiB) non-empty blocks including 8 (619.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (624.5 KiB) non-empty blocks including 8 (624.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (621.6 KiB) non-empty blocks including 8 (621.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (619.1 KiB) non-empty blocks including 8 (619.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (618.2 KiB) non-empty blocks including 8 (618.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.6 MiB) non-empty blocks including 200 (61.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.0 MiB) non-empty blocks including 200 (61.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 200 (61.3 MiB) non-empty blocks including 200 (61.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (618.3 KiB) non-empty blocks including 8 (618.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (619.7 KiB) non-empty blocks including 8 (619.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (624.5 KiB) non-empty blocks including 8 (624.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (619.1 KiB) non-empty blocks including 8 (619.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (618.2 KiB) non-empty blocks including 8 (618.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Getting 8 (621.6 KiB) non-empty blocks including 8 (621.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:19 INFO BlockManagerInfo: Removed broadcast_23_piece0 on 10.0.0.133:59303 in memory (size: 11.0 KiB, free: 8.6 GiB) -26/04/01 08:44:27 INFO Executor: Finished task 7.0 in stage 21.0 (TID 741). 9594 bytes result sent to driver -26/04/01 08:44:27 INFO TaskSetManager: Starting task 8.0 in stage 21.0 (TID 742) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:27 INFO Executor: Running task 8.0 in stage 21.0 (TID 742) -26/04/01 08:44:27 INFO TaskSetManager: Finished task 7.0 in stage 21.0 (TID 741) in 9887 ms on 10.0.0.133 (executor driver) (1/14) -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (623.7 KiB) non-empty blocks including 8 (623.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (623.7 KiB) non-empty blocks including 8 (623.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:27 INFO Executor: Finished task 0.0 in stage 21.0 (TID 734). 9594 bytes result sent to driver -26/04/01 08:44:27 INFO TaskSetManager: Starting task 9.0 in stage 21.0 (TID 743) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:27 INFO Executor: Running task 9.0 in stage 21.0 (TID 743) -26/04/01 08:44:27 INFO TaskSetManager: Finished task 0.0 in stage 21.0 (TID 734) in 9947 ms on 10.0.0.133 (executor driver) (2/14) -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (623.6 KiB) non-empty blocks including 8 (623.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (623.6 KiB) non-empty blocks including 8 (623.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:27 INFO Executor: Finished task 5.0 in stage 21.0 (TID 739). 9594 bytes result sent to driver -26/04/01 08:44:27 INFO TaskSetManager: Starting task 10.0 in stage 21.0 (TID 744) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:27 INFO Executor: Running task 10.0 in stage 21.0 (TID 744) -26/04/01 08:44:27 INFO TaskSetManager: Finished task 5.0 in stage 21.0 (TID 739) in 10048 ms on 10.0.0.133 (executor driver) (3/14) -26/04/01 08:44:27 INFO Executor: Finished task 6.0 in stage 21.0 (TID 740). 9594 bytes result sent to driver -26/04/01 08:44:27 INFO TaskSetManager: Starting task 11.0 in stage 21.0 (TID 745) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:27 INFO TaskSetManager: Finished task 6.0 in stage 21.0 (TID 740) in 10049 ms on 10.0.0.133 (executor driver) (4/14) -26/04/01 08:44:27 INFO Executor: Running task 11.0 in stage 21.0 (TID 745) -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.2 MiB) non-empty blocks including 200 (61.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (622.2 KiB) non-empty blocks including 8 (622.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (617.6 KiB) non-empty blocks including 8 (617.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.2 MiB) non-empty blocks including 200 (61.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (622.2 KiB) non-empty blocks including 8 (622.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (617.6 KiB) non-empty blocks including 8 (617.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:27 INFO Executor: Finished task 4.0 in stage 21.0 (TID 738). 9594 bytes result sent to driver -26/04/01 08:44:27 INFO TaskSetManager: Starting task 12.0 in stage 21.0 (TID 746) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:27 INFO TaskSetManager: Finished task 4.0 in stage 21.0 (TID 738) in 10095 ms on 10.0.0.133 (executor driver) (5/14) -26/04/01 08:44:27 INFO Executor: Running task 12.0 in stage 21.0 (TID 746) -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (622.0 KiB) non-empty blocks including 8 (622.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (622.0 KiB) non-empty blocks including 8 (622.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:27 INFO Executor: Finished task 2.0 in stage 21.0 (TID 736). 9594 bytes result sent to driver -26/04/01 08:44:27 INFO TaskSetManager: Starting task 13.0 in stage 21.0 (TID 747) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:27 INFO TaskSetManager: Finished task 2.0 in stage 21.0 (TID 736) in 10111 ms on 10.0.0.133 (executor driver) (6/14) -26/04/01 08:44:27 INFO Executor: Running task 13.0 in stage 21.0 (TID 747) -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (20.4 MiB) non-empty blocks including 200 (20.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (207.3 KiB) non-empty blocks including 8 (207.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 200 (20.4 MiB) non-empty blocks including 200 (20.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Getting 8 (207.3 KiB) non-empty blocks including 8 (207.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:27 INFO Executor: Finished task 3.0 in stage 21.0 (TID 737). 9594 bytes result sent to driver -26/04/01 08:44:27 INFO TaskSetManager: Finished task 3.0 in stage 21.0 (TID 737) in 10162 ms on 10.0.0.133 (executor driver) (7/14) -26/04/01 08:44:27 INFO Executor: Finished task 1.0 in stage 21.0 (TID 735). 9594 bytes result sent to driver -26/04/01 08:44:27 INFO TaskSetManager: Finished task 1.0 in stage 21.0 (TID 735) in 10193 ms on 10.0.0.133 (executor driver) (8/14) -26/04/01 08:44:30 INFO Executor: Finished task 13.0 in stage 21.0 (TID 747). 9594 bytes result sent to driver -26/04/01 08:44:30 INFO TaskSetManager: Finished task 13.0 in stage 21.0 (TID 747) in 2979 ms on 10.0.0.133 (executor driver) (9/14) -26/04/01 08:44:36 INFO Executor: Finished task 10.0 in stage 21.0 (TID 744). 9594 bytes result sent to driver -26/04/01 08:44:36 INFO TaskSetManager: Finished task 10.0 in stage 21.0 (TID 744) in 9283 ms on 10.0.0.133 (executor driver) (10/14) -26/04/01 08:44:36 INFO Executor: Finished task 8.0 in stage 21.0 (TID 742). 9594 bytes result sent to driver -26/04/01 08:44:36 INFO TaskSetManager: Finished task 8.0 in stage 21.0 (TID 742) in 9452 ms on 10.0.0.133 (executor driver) (11/14) -26/04/01 08:44:36 INFO Executor: Finished task 9.0 in stage 21.0 (TID 743). 9594 bytes result sent to driver -26/04/01 08:44:36 INFO TaskSetManager: Finished task 9.0 in stage 21.0 (TID 743) in 9400 ms on 10.0.0.133 (executor driver) (12/14) -26/04/01 08:44:36 INFO Executor: Finished task 12.0 in stage 21.0 (TID 746). 9594 bytes result sent to driver -26/04/01 08:44:36 INFO TaskSetManager: Finished task 12.0 in stage 21.0 (TID 746) in 9261 ms on 10.0.0.133 (executor driver) (13/14) -26/04/01 08:44:36 INFO Executor: Finished task 11.0 in stage 21.0 (TID 745). 9594 bytes result sent to driver -26/04/01 08:44:36 INFO TaskSetManager: Finished task 11.0 in stage 21.0 (TID 745) in 9335 ms on 10.0.0.133 (executor driver) (14/14) -26/04/01 08:44:36 INFO TaskSchedulerImpl: Removed TaskSet 21.0, whose tasks have all completed, from pool -26/04/01 08:44:36 INFO DAGScheduler: ShuffleMapStage 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 19.386 s -26/04/01 08:44:36 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:44:36 INFO DAGScheduler: running: Set() -26/04/01 08:44:36 INFO DAGScheduler: waiting: Set() -26/04/01 08:44:36 INFO DAGScheduler: failed: Set() -26/04/01 08:44:36 INFO ShufflePartitionsUtil: For shuffle(6, 3), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 08:44:36 INFO DAGScheduler: Registering RDD 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 7 -26/04/01 08:44:36 INFO DAGScheduler: Got map stage job 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 29 output partitions -26/04/01 08:44:36 INFO DAGScheduler: Final stage: ShuffleMapStage 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:44:36 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 25, ShuffleMapStage 26) -26/04/01 08:44:36 INFO DAGScheduler: Missing parents: List() -26/04/01 08:44:36 INFO DAGScheduler: Submitting ShuffleMapStage 27 (MapPartitionsRDD[47] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:44:36 INFO MemoryStore: Block broadcast_25 stored as values in memory (estimated size 28.9 KiB, free 8.6 GiB) -26/04/01 08:44:36 INFO MemoryStore: Block broadcast_25_piece0 stored as bytes in memory (estimated size 10.3 KiB, free 8.6 GiB) -26/04/01 08:44:36 INFO BlockManagerInfo: Added broadcast_25_piece0 in memory on 10.0.0.133:59303 (size: 10.3 KiB, free: 8.6 GiB) -26/04/01 08:44:36 INFO SparkContext: Created broadcast 25 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:44:36 INFO DAGScheduler: Submitting 29 missing tasks from ShuffleMapStage 27 (MapPartitionsRDD[47] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:44:36 INFO TaskSchedulerImpl: Adding task set 27.0 with 29 tasks resource profile 0 -26/04/01 08:44:36 INFO TaskSetManager: Starting task 0.0 in stage 27.0 (TID 748) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:36 INFO TaskSetManager: Starting task 1.0 in stage 27.0 (TID 749) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:36 INFO TaskSetManager: Starting task 2.0 in stage 27.0 (TID 750) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:36 INFO TaskSetManager: Starting task 3.0 in stage 27.0 (TID 751) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:36 INFO TaskSetManager: Starting task 4.0 in stage 27.0 (TID 752) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:36 INFO TaskSetManager: Starting task 5.0 in stage 27.0 (TID 753) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:36 INFO TaskSetManager: Starting task 6.0 in stage 27.0 (TID 754) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:36 INFO TaskSetManager: Starting task 7.0 in stage 27.0 (TID 755) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:36 INFO Executor: Running task 0.0 in stage 27.0 (TID 748) -26/04/01 08:44:36 INFO Executor: Running task 2.0 in stage 27.0 (TID 750) -26/04/01 08:44:36 INFO Executor: Running task 1.0 in stage 27.0 (TID 749) -26/04/01 08:44:36 INFO Executor: Running task 5.0 in stage 27.0 (TID 753) -26/04/01 08:44:36 INFO Executor: Running task 3.0 in stage 27.0 (TID 751) -26/04/01 08:44:36 INFO Executor: Running task 6.0 in stage 27.0 (TID 754) -26/04/01 08:44:36 INFO Executor: Running task 7.0 in stage 27.0 (TID 755) -26/04/01 08:44:36 INFO Executor: Running task 4.0 in stage 27.0 (TID 752) -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.6 MiB) non-empty blocks including 14 (23.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.2 MiB) non-empty blocks including 14 (23.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.2 MiB) non-empty blocks including 14 (23.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.6 MiB) non-empty blocks including 14 (23.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:46 INFO Executor: Finished task 5.0 in stage 27.0 (TID 753). 9594 bytes result sent to driver -26/04/01 08:44:46 INFO TaskSetManager: Starting task 8.0 in stage 27.0 (TID 756) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:46 INFO Executor: Running task 8.0 in stage 27.0 (TID 756) -26/04/01 08:44:46 INFO TaskSetManager: Finished task 5.0 in stage 27.0 (TID 753) in 9399 ms on 10.0.0.133 (executor driver) (1/29) -26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:47 INFO Executor: Finished task 7.0 in stage 27.0 (TID 755). 9594 bytes result sent to driver -26/04/01 08:44:47 INFO TaskSetManager: Starting task 9.0 in stage 27.0 (TID 757) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:47 INFO Executor: Running task 9.0 in stage 27.0 (TID 757) -26/04/01 08:44:47 INFO TaskSetManager: Finished task 7.0 in stage 27.0 (TID 755) in 10074 ms on 10.0.0.133 (executor driver) (2/29) -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:47 INFO Executor: Finished task 4.0 in stage 27.0 (TID 752). 9594 bytes result sent to driver -26/04/01 08:44:47 INFO TaskSetManager: Starting task 10.0 in stage 27.0 (TID 758) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:47 INFO Executor: Running task 10.0 in stage 27.0 (TID 758) -26/04/01 08:44:47 INFO TaskSetManager: Finished task 4.0 in stage 27.0 (TID 752) in 10242 ms on 10.0.0.133 (executor driver) (3/29) -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:47 INFO Executor: Finished task 3.0 in stage 27.0 (TID 751). 9594 bytes result sent to driver -26/04/01 08:44:47 INFO TaskSetManager: Starting task 11.0 in stage 27.0 (TID 759) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:47 INFO TaskSetManager: Finished task 3.0 in stage 27.0 (TID 751) in 10315 ms on 10.0.0.133 (executor driver) (4/29) -26/04/01 08:44:47 INFO Executor: Running task 11.0 in stage 27.0 (TID 759) -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:47 INFO Executor: Finished task 6.0 in stage 27.0 (TID 754). 9594 bytes result sent to driver -26/04/01 08:44:47 INFO TaskSetManager: Starting task 12.0 in stage 27.0 (TID 760) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:47 INFO TaskSetManager: Finished task 6.0 in stage 27.0 (TID 754) in 10328 ms on 10.0.0.133 (executor driver) (5/29) -26/04/01 08:44:47 INFO Executor: Running task 12.0 in stage 27.0 (TID 760) -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:47 INFO Executor: Finished task 2.0 in stage 27.0 (TID 750). 9594 bytes result sent to driver -26/04/01 08:44:47 INFO TaskSetManager: Starting task 13.0 in stage 27.0 (TID 761) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:47 INFO TaskSetManager: Finished task 2.0 in stage 27.0 (TID 750) in 10839 ms on 10.0.0.133 (executor driver) (6/29) -26/04/01 08:44:47 INFO Executor: Running task 13.0 in stage 27.0 (TID 761) -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:47 INFO Executor: Finished task 0.0 in stage 27.0 (TID 748). 9594 bytes result sent to driver -26/04/01 08:44:47 INFO TaskSetManager: Starting task 14.0 in stage 27.0 (TID 762) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:47 INFO TaskSetManager: Finished task 0.0 in stage 27.0 (TID 748) in 10894 ms on 10.0.0.133 (executor driver) (7/29) -26/04/01 08:44:47 INFO Executor: Running task 14.0 in stage 27.0 (TID 762) -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:47 INFO Executor: Finished task 1.0 in stage 27.0 (TID 749). 9594 bytes result sent to driver -26/04/01 08:44:47 INFO TaskSetManager: Starting task 15.0 in stage 27.0 (TID 763) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:47 INFO TaskSetManager: Finished task 1.0 in stage 27.0 (TID 749) in 10902 ms on 10.0.0.133 (executor driver) (8/29) -26/04/01 08:44:47 INFO Executor: Running task 15.0 in stage 27.0 (TID 763) -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:55 INFO Executor: Finished task 8.0 in stage 27.0 (TID 756). 9594 bytes result sent to driver -26/04/01 08:44:55 INFO TaskSetManager: Starting task 16.0 in stage 27.0 (TID 764) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:55 INFO Executor: Running task 16.0 in stage 27.0 (TID 764) -26/04/01 08:44:55 INFO TaskSetManager: Finished task 8.0 in stage 27.0 (TID 756) in 9022 ms on 10.0.0.133 (executor driver) (9/29) -26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:56 INFO Executor: Finished task 9.0 in stage 27.0 (TID 757). 9594 bytes result sent to driver -26/04/01 08:44:56 INFO TaskSetManager: Starting task 17.0 in stage 27.0 (TID 765) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:56 INFO Executor: Running task 17.0 in stage 27.0 (TID 765) -26/04/01 08:44:56 INFO TaskSetManager: Finished task 9.0 in stage 27.0 (TID 757) in 9059 ms on 10.0.0.133 (executor driver) (10/29) -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:56 INFO Executor: Finished task 11.0 in stage 27.0 (TID 759). 9594 bytes result sent to driver -26/04/01 08:44:56 INFO TaskSetManager: Starting task 18.0 in stage 27.0 (TID 766) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:56 INFO Executor: Running task 18.0 in stage 27.0 (TID 766) -26/04/01 08:44:56 INFO TaskSetManager: Finished task 11.0 in stage 27.0 (TID 759) in 9606 ms on 10.0.0.133 (executor driver) (11/29) -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:57 INFO Executor: Finished task 10.0 in stage 27.0 (TID 758). 9594 bytes result sent to driver -26/04/01 08:44:57 INFO TaskSetManager: Starting task 19.0 in stage 27.0 (TID 767) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:57 INFO TaskSetManager: Finished task 10.0 in stage 27.0 (TID 758) in 9859 ms on 10.0.0.133 (executor driver) (12/29) -26/04/01 08:44:57 INFO Executor: Running task 19.0 in stage 27.0 (TID 767) -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:57 INFO Executor: Finished task 12.0 in stage 27.0 (TID 760). 9594 bytes result sent to driver -26/04/01 08:44:57 INFO TaskSetManager: Starting task 20.0 in stage 27.0 (TID 768) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:57 INFO Executor: Running task 20.0 in stage 27.0 (TID 768) -26/04/01 08:44:57 INFO TaskSetManager: Finished task 12.0 in stage 27.0 (TID 760) in 9836 ms on 10.0.0.133 (executor driver) (13/29) -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:57 INFO Executor: Finished task 15.0 in stage 27.0 (TID 763). 9594 bytes result sent to driver -26/04/01 08:44:57 INFO Executor: Finished task 14.0 in stage 27.0 (TID 762). 9594 bytes result sent to driver -26/04/01 08:44:57 INFO TaskSetManager: Starting task 21.0 in stage 27.0 (TID 769) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:57 INFO Executor: Running task 21.0 in stage 27.0 (TID 769) -26/04/01 08:44:57 INFO TaskSetManager: Starting task 22.0 in stage 27.0 (TID 770) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:57 INFO TaskSetManager: Finished task 15.0 in stage 27.0 (TID 763) in 9658 ms on 10.0.0.133 (executor driver) (14/29) -26/04/01 08:44:57 INFO Executor: Running task 22.0 in stage 27.0 (TID 770) -26/04/01 08:44:57 INFO TaskSetManager: Finished task 14.0 in stage 27.0 (TID 762) in 9667 ms on 10.0.0.133 (executor driver) (15/29) -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:57 INFO Executor: Finished task 13.0 in stage 27.0 (TID 761). 9594 bytes result sent to driver -26/04/01 08:44:57 INFO TaskSetManager: Starting task 23.0 in stage 27.0 (TID 771) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9428 bytes) -26/04/01 08:44:57 INFO TaskSetManager: Finished task 13.0 in stage 27.0 (TID 761) in 9735 ms on 10.0.0.133 (executor driver) (16/29) -26/04/01 08:44:57 INFO Executor: Running task 23.0 in stage 27.0 (TID 771) -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:44:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:44:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:04 INFO Executor: Finished task 16.0 in stage 27.0 (TID 764). 9594 bytes result sent to driver -26/04/01 08:45:04 INFO TaskSetManager: Starting task 24.0 in stage 27.0 (TID 772) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9428 bytes) -26/04/01 08:45:04 INFO Executor: Running task 24.0 in stage 27.0 (TID 772) -26/04/01 08:45:04 INFO TaskSetManager: Finished task 16.0 in stage 27.0 (TID 764) in 8861 ms on 10.0.0.133 (executor driver) (17/29) -26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:05 INFO Executor: Finished task 17.0 in stage 27.0 (TID 765). 9594 bytes result sent to driver -26/04/01 08:45:05 INFO TaskSetManager: Starting task 25.0 in stage 27.0 (TID 773) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9428 bytes) -26/04/01 08:45:05 INFO TaskSetManager: Finished task 17.0 in stage 27.0 (TID 765) in 9168 ms on 10.0.0.133 (executor driver) (18/29) -26/04/01 08:45:05 INFO Executor: Running task 25.0 in stage 27.0 (TID 773) -26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:06 INFO Executor: Finished task 18.0 in stage 27.0 (TID 766). 9594 bytes result sent to driver -26/04/01 08:45:06 INFO TaskSetManager: Starting task 26.0 in stage 27.0 (TID 774) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9428 bytes) -26/04/01 08:45:06 INFO Executor: Running task 26.0 in stage 27.0 (TID 774) -26/04/01 08:45:06 INFO TaskSetManager: Finished task 18.0 in stage 27.0 (TID 766) in 9253 ms on 10.0.0.133 (executor driver) (19/29) -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:06 INFO Executor: Finished task 20.0 in stage 27.0 (TID 768). 9594 bytes result sent to driver -26/04/01 08:45:06 INFO TaskSetManager: Starting task 27.0 in stage 27.0 (TID 775) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9428 bytes) -26/04/01 08:45:06 INFO Executor: Running task 27.0 in stage 27.0 (TID 775) -26/04/01 08:45:06 INFO TaskSetManager: Finished task 20.0 in stage 27.0 (TID 768) in 9211 ms on 10.0.0.133 (executor driver) (20/29) -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:06 INFO Executor: Finished task 19.0 in stage 27.0 (TID 767). 9594 bytes result sent to driver -26/04/01 08:45:06 INFO TaskSetManager: Starting task 28.0 in stage 27.0 (TID 776) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9428 bytes) -26/04/01 08:45:06 INFO TaskSetManager: Finished task 19.0 in stage 27.0 (TID 767) in 9525 ms on 10.0.0.133 (executor driver) (21/29) -26/04/01 08:45:06 INFO Executor: Running task 28.0 in stage 27.0 (TID 776) -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 14 (13.4 MiB) non-empty blocks including 14 (13.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 37 (21.8 MiB) non-empty blocks including 37 (21.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 14 (13.4 MiB) non-empty blocks including 14 (13.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Getting 37 (21.8 MiB) non-empty blocks including 37 (21.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:06 INFO Executor: Finished task 21.0 in stage 27.0 (TID 769). 9594 bytes result sent to driver -26/04/01 08:45:06 INFO TaskSetManager: Finished task 21.0 in stage 27.0 (TID 769) in 9189 ms on 10.0.0.133 (executor driver) (22/29) -26/04/01 08:45:06 INFO Executor: Finished task 22.0 in stage 27.0 (TID 770). 9594 bytes result sent to driver -26/04/01 08:45:06 INFO TaskSetManager: Finished task 22.0 in stage 27.0 (TID 770) in 9363 ms on 10.0.0.133 (executor driver) (23/29) -26/04/01 08:45:07 INFO Executor: Finished task 23.0 in stage 27.0 (TID 771). 9594 bytes result sent to driver -26/04/01 08:45:07 INFO TaskSetManager: Finished task 23.0 in stage 27.0 (TID 771) in 9516 ms on 10.0.0.133 (executor driver) (24/29) -26/04/01 08:45:11 INFO Executor: Finished task 28.0 in stage 27.0 (TID 776). 9594 bytes result sent to driver -26/04/01 08:45:11 INFO TaskSetManager: Finished task 28.0 in stage 27.0 (TID 776) in 5264 ms on 10.0.0.133 (executor driver) (25/29) -26/04/01 08:45:13 INFO Executor: Finished task 24.0 in stage 27.0 (TID 772). 9594 bytes result sent to driver -26/04/01 08:45:13 INFO TaskSetManager: Finished task 24.0 in stage 27.0 (TID 772) in 8823 ms on 10.0.0.133 (executor driver) (26/29) -26/04/01 08:45:14 INFO Executor: Finished task 25.0 in stage 27.0 (TID 773). 9594 bytes result sent to driver -26/04/01 08:45:14 INFO TaskSetManager: Finished task 25.0 in stage 27.0 (TID 773) in 8795 ms on 10.0.0.133 (executor driver) (27/29) -26/04/01 08:45:15 INFO Executor: Finished task 26.0 in stage 27.0 (TID 774). 9594 bytes result sent to driver -26/04/01 08:45:15 INFO TaskSetManager: Finished task 26.0 in stage 27.0 (TID 774) in 8852 ms on 10.0.0.133 (executor driver) (28/29) -26/04/01 08:45:15 INFO Executor: Finished task 27.0 in stage 27.0 (TID 775). 9594 bytes result sent to driver -26/04/01 08:45:15 INFO TaskSetManager: Finished task 27.0 in stage 27.0 (TID 775) in 8855 ms on 10.0.0.133 (executor driver) (29/29) -26/04/01 08:45:15 INFO TaskSchedulerImpl: Removed TaskSet 27.0, whose tasks have all completed, from pool -26/04/01 08:45:15 INFO DAGScheduler: ShuffleMapStage 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 38.232 s -26/04/01 08:45:15 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:45:15 INFO DAGScheduler: running: Set() -26/04/01 08:45:15 INFO DAGScheduler: waiting: Set() -26/04/01 08:45:15 INFO DAGScheduler: failed: Set() -26/04/01 08:45:15 INFO ShufflePartitionsUtil: For shuffle(7, 4), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 08:45:15 INFO DAGScheduler: Registering RDD 52 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 8 -26/04/01 08:45:15 INFO DAGScheduler: Got map stage job 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 34 output partitions -26/04/01 08:45:15 INFO DAGScheduler: Final stage: ShuffleMapStage 35 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:45:15 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 33, ShuffleMapStage 34) -26/04/01 08:45:15 INFO DAGScheduler: Missing parents: List() -26/04/01 08:45:15 INFO DAGScheduler: Submitting ShuffleMapStage 35 (MapPartitionsRDD[52] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:45:15 INFO MemoryStore: Block broadcast_26 stored as values in memory (estimated size 44.4 KiB, free 8.6 GiB) -26/04/01 08:45:15 INFO MemoryStore: Block broadcast_26_piece0 stored as bytes in memory (estimated size 12.6 KiB, free 8.6 GiB) -26/04/01 08:45:15 INFO BlockManagerInfo: Added broadcast_26_piece0 in memory on 10.0.0.133:59303 (size: 12.6 KiB, free: 8.6 GiB) -26/04/01 08:45:15 INFO SparkContext: Created broadcast 26 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:45:15 INFO DAGScheduler: Submitting 34 missing tasks from ShuffleMapStage 35 (MapPartitionsRDD[52] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:45:15 INFO TaskSchedulerImpl: Adding task set 35.0 with 34 tasks resource profile 0 -26/04/01 08:45:15 INFO TaskSetManager: Starting task 0.0 in stage 35.0 (TID 777) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:15 INFO TaskSetManager: Starting task 1.0 in stage 35.0 (TID 778) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:15 INFO TaskSetManager: Starting task 2.0 in stage 35.0 (TID 779) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:15 INFO TaskSetManager: Starting task 3.0 in stage 35.0 (TID 780) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:15 INFO TaskSetManager: Starting task 4.0 in stage 35.0 (TID 781) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:15 INFO TaskSetManager: Starting task 5.0 in stage 35.0 (TID 782) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:15 INFO TaskSetManager: Starting task 6.0 in stage 35.0 (TID 783) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:15 INFO TaskSetManager: Starting task 7.0 in stage 35.0 (TID 784) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:15 INFO Executor: Running task 0.0 in stage 35.0 (TID 777) -26/04/01 08:45:15 INFO Executor: Running task 1.0 in stage 35.0 (TID 778) -26/04/01 08:45:15 INFO Executor: Running task 3.0 in stage 35.0 (TID 780) -26/04/01 08:45:15 INFO Executor: Running task 2.0 in stage 35.0 (TID 779) -26/04/01 08:45:15 INFO Executor: Running task 5.0 in stage 35.0 (TID 782) -26/04/01 08:45:15 INFO Executor: Running task 7.0 in stage 35.0 (TID 784) -26/04/01 08:45:15 INFO Executor: Running task 4.0 in stage 35.0 (TID 781) -26/04/01 08:45:15 INFO Executor: Running task 6.0 in stage 35.0 (TID 783) -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO Executor: Finished task 0.0 in stage 35.0 (TID 777). 10455 bytes result sent to driver -26/04/01 08:45:22 INFO TaskSetManager: Starting task 8.0 in stage 35.0 (TID 785) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:22 INFO TaskSetManager: Finished task 0.0 in stage 35.0 (TID 777) in 7148 ms on 10.0.0.133 (executor driver) (1/34) -26/04/01 08:45:22 INFO Executor: Running task 8.0 in stage 35.0 (TID 785) -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO Executor: Finished task 3.0 in stage 35.0 (TID 780). 10455 bytes result sent to driver -26/04/01 08:45:22 INFO TaskSetManager: Starting task 9.0 in stage 35.0 (TID 786) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:22 INFO TaskSetManager: Finished task 3.0 in stage 35.0 (TID 780) in 7424 ms on 10.0.0.133 (executor driver) (2/34) -26/04/01 08:45:22 INFO Executor: Running task 9.0 in stage 35.0 (TID 786) -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO Executor: Finished task 1.0 in stage 35.0 (TID 778). 10455 bytes result sent to driver -26/04/01 08:45:22 INFO TaskSetManager: Starting task 10.0 in stage 35.0 (TID 787) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:22 INFO Executor: Running task 10.0 in stage 35.0 (TID 787) -26/04/01 08:45:22 INFO TaskSetManager: Finished task 1.0 in stage 35.0 (TID 778) in 7433 ms on 10.0.0.133 (executor driver) (3/34) -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO Executor: Finished task 2.0 in stage 35.0 (TID 779). 10455 bytes result sent to driver -26/04/01 08:45:22 INFO TaskSetManager: Starting task 11.0 in stage 35.0 (TID 788) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:22 INFO TaskSetManager: Finished task 2.0 in stage 35.0 (TID 779) in 7501 ms on 10.0.0.133 (executor driver) (4/34) -26/04/01 08:45:22 INFO Executor: Running task 11.0 in stage 35.0 (TID 788) -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO Executor: Finished task 6.0 in stage 35.0 (TID 783). 10455 bytes result sent to driver -26/04/01 08:45:22 INFO TaskSetManager: Starting task 12.0 in stage 35.0 (TID 789) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:22 INFO TaskSetManager: Finished task 6.0 in stage 35.0 (TID 783) in 7539 ms on 10.0.0.133 (executor driver) (5/34) -26/04/01 08:45:22 INFO Executor: Running task 12.0 in stage 35.0 (TID 789) -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO Executor: Finished task 7.0 in stage 35.0 (TID 784). 10455 bytes result sent to driver -26/04/01 08:45:22 INFO TaskSetManager: Starting task 13.0 in stage 35.0 (TID 790) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:22 INFO Executor: Running task 13.0 in stage 35.0 (TID 790) -26/04/01 08:45:22 INFO TaskSetManager: Finished task 7.0 in stage 35.0 (TID 784) in 7559 ms on 10.0.0.133 (executor driver) (6/34) -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO Executor: Finished task 4.0 in stage 35.0 (TID 781). 10455 bytes result sent to driver -26/04/01 08:45:22 INFO TaskSetManager: Starting task 14.0 in stage 35.0 (TID 791) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:22 INFO TaskSetManager: Finished task 4.0 in stage 35.0 (TID 781) in 7573 ms on 10.0.0.133 (executor driver) (7/34) -26/04/01 08:45:22 INFO Executor: Running task 14.0 in stage 35.0 (TID 791) -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO Executor: Finished task 5.0 in stage 35.0 (TID 782). 10455 bytes result sent to driver -26/04/01 08:45:22 INFO TaskSetManager: Starting task 15.0 in stage 35.0 (TID 792) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:22 INFO Executor: Running task 15.0 in stage 35.0 (TID 792) -26/04/01 08:45:22 INFO TaskSetManager: Finished task 5.0 in stage 35.0 (TID 782) in 7576 ms on 10.0.0.133 (executor driver) (8/34) -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO Executor: Finished task 8.0 in stage 35.0 (TID 785). 10455 bytes result sent to driver -26/04/01 08:45:29 INFO TaskSetManager: Starting task 16.0 in stage 35.0 (TID 793) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:29 INFO Executor: Running task 16.0 in stage 35.0 (TID 793) -26/04/01 08:45:29 INFO TaskSetManager: Finished task 8.0 in stage 35.0 (TID 785) in 6922 ms on 10.0.0.133 (executor driver) (9/34) -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO Executor: Finished task 9.0 in stage 35.0 (TID 786). 10455 bytes result sent to driver -26/04/01 08:45:29 INFO TaskSetManager: Starting task 17.0 in stage 35.0 (TID 794) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:29 INFO Executor: Running task 17.0 in stage 35.0 (TID 794) -26/04/01 08:45:29 INFO TaskSetManager: Finished task 9.0 in stage 35.0 (TID 786) in 6863 ms on 10.0.0.133 (executor driver) (10/34) -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO Executor: Finished task 10.0 in stage 35.0 (TID 787). 10455 bytes result sent to driver -26/04/01 08:45:29 INFO TaskSetManager: Starting task 18.0 in stage 35.0 (TID 795) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:29 INFO TaskSetManager: Finished task 10.0 in stage 35.0 (TID 787) in 6919 ms on 10.0.0.133 (executor driver) (11/34) -26/04/01 08:45:29 INFO Executor: Running task 18.0 in stage 35.0 (TID 795) -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO Executor: Finished task 11.0 in stage 35.0 (TID 788). 10455 bytes result sent to driver -26/04/01 08:45:29 INFO TaskSetManager: Starting task 19.0 in stage 35.0 (TID 796) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:29 INFO TaskSetManager: Finished task 11.0 in stage 35.0 (TID 788) in 6907 ms on 10.0.0.133 (executor driver) (12/34) -26/04/01 08:45:29 INFO Executor: Running task 19.0 in stage 35.0 (TID 796) -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO Executor: Finished task 12.0 in stage 35.0 (TID 789). 10455 bytes result sent to driver -26/04/01 08:45:29 INFO TaskSetManager: Starting task 20.0 in stage 35.0 (TID 797) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:29 INFO Executor: Running task 20.0 in stage 35.0 (TID 797) -26/04/01 08:45:29 INFO TaskSetManager: Finished task 12.0 in stage 35.0 (TID 789) in 6937 ms on 10.0.0.133 (executor driver) (13/34) -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO Executor: Finished task 13.0 in stage 35.0 (TID 790). 10455 bytes result sent to driver -26/04/01 08:45:29 INFO TaskSetManager: Starting task 21.0 in stage 35.0 (TID 798) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:29 INFO TaskSetManager: Finished task 13.0 in stage 35.0 (TID 790) in 6989 ms on 10.0.0.133 (executor driver) (14/34) -26/04/01 08:45:29 INFO Executor: Running task 21.0 in stage 35.0 (TID 798) -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO Executor: Finished task 14.0 in stage 35.0 (TID 791). 10455 bytes result sent to driver -26/04/01 08:45:29 INFO TaskSetManager: Starting task 22.0 in stage 35.0 (TID 799) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:29 INFO Executor: Running task 22.0 in stage 35.0 (TID 799) -26/04/01 08:45:29 INFO TaskSetManager: Finished task 14.0 in stage 35.0 (TID 791) in 7033 ms on 10.0.0.133 (executor driver) (15/34) -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO Executor: Finished task 15.0 in stage 35.0 (TID 792). 10455 bytes result sent to driver -26/04/01 08:45:29 INFO TaskSetManager: Starting task 23.0 in stage 35.0 (TID 800) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:29 INFO Executor: Running task 23.0 in stage 35.0 (TID 800) -26/04/01 08:45:29 INFO TaskSetManager: Finished task 15.0 in stage 35.0 (TID 792) in 7050 ms on 10.0.0.133 (executor driver) (16/34) -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO Executor: Finished task 16.0 in stage 35.0 (TID 793). 9724 bytes result sent to driver -26/04/01 08:45:36 INFO TaskSetManager: Starting task 24.0 in stage 35.0 (TID 801) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:36 INFO Executor: Running task 24.0 in stage 35.0 (TID 801) -26/04/01 08:45:36 INFO TaskSetManager: Finished task 16.0 in stage 35.0 (TID 793) in 6896 ms on 10.0.0.133 (executor driver) (17/34) -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO Executor: Finished task 17.0 in stage 35.0 (TID 794). 9724 bytes result sent to driver -26/04/01 08:45:36 INFO TaskSetManager: Starting task 25.0 in stage 35.0 (TID 802) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:36 INFO TaskSetManager: Finished task 17.0 in stage 35.0 (TID 794) in 6801 ms on 10.0.0.133 (executor driver) (18/34) -26/04/01 08:45:36 INFO Executor: Running task 25.0 in stage 35.0 (TID 802) -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO Executor: Finished task 18.0 in stage 35.0 (TID 795). 9724 bytes result sent to driver -26/04/01 08:45:36 INFO TaskSetManager: Starting task 26.0 in stage 35.0 (TID 803) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:36 INFO Executor: Running task 26.0 in stage 35.0 (TID 803) -26/04/01 08:45:36 INFO TaskSetManager: Finished task 18.0 in stage 35.0 (TID 795) in 6844 ms on 10.0.0.133 (executor driver) (19/34) -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO Executor: Finished task 19.0 in stage 35.0 (TID 796). 9724 bytes result sent to driver -26/04/01 08:45:36 INFO TaskSetManager: Starting task 27.0 in stage 35.0 (TID 804) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:36 INFO TaskSetManager: Finished task 19.0 in stage 35.0 (TID 796) in 6862 ms on 10.0.0.133 (executor driver) (20/34) -26/04/01 08:45:36 INFO Executor: Running task 27.0 in stage 35.0 (TID 804) -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO Executor: Finished task 20.0 in stage 35.0 (TID 797). 9724 bytes result sent to driver -26/04/01 08:45:36 INFO TaskSetManager: Starting task 28.0 in stage 35.0 (TID 805) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:36 INFO TaskSetManager: Finished task 20.0 in stage 35.0 (TID 797) in 6876 ms on 10.0.0.133 (executor driver) (21/34) -26/04/01 08:45:36 INFO Executor: Running task 28.0 in stage 35.0 (TID 805) -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO Executor: Finished task 21.0 in stage 35.0 (TID 798). 9724 bytes result sent to driver -26/04/01 08:45:36 INFO TaskSetManager: Starting task 29.0 in stage 35.0 (TID 806) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:36 INFO TaskSetManager: Finished task 21.0 in stage 35.0 (TID 798) in 6812 ms on 10.0.0.133 (executor driver) (22/34) -26/04/01 08:45:36 INFO Executor: Running task 29.0 in stage 35.0 (TID 806) -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO Executor: Finished task 22.0 in stage 35.0 (TID 799). 9724 bytes result sent to driver -26/04/01 08:45:36 INFO TaskSetManager: Starting task 30.0 in stage 35.0 (TID 807) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:36 INFO Executor: Running task 30.0 in stage 35.0 (TID 807) -26/04/01 08:45:36 INFO TaskSetManager: Finished task 22.0 in stage 35.0 (TID 799) in 6887 ms on 10.0.0.133 (executor driver) (23/34) -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO Executor: Finished task 23.0 in stage 35.0 (TID 800). 9724 bytes result sent to driver -26/04/01 08:45:36 INFO TaskSetManager: Starting task 31.0 in stage 35.0 (TID 808) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:36 INFO Executor: Running task 31.0 in stage 35.0 (TID 808) -26/04/01 08:45:36 INFO TaskSetManager: Finished task 23.0 in stage 35.0 (TID 800) in 6910 ms on 10.0.0.133 (executor driver) (24/34) -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:43 INFO Executor: Finished task 24.0 in stage 35.0 (TID 801). 10455 bytes result sent to driver -26/04/01 08:45:43 INFO TaskSetManager: Starting task 32.0 in stage 35.0 (TID 809) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:43 INFO Executor: Running task 32.0 in stage 35.0 (TID 809) -26/04/01 08:45:43 INFO TaskSetManager: Finished task 24.0 in stage 35.0 (TID 801) in 6800 ms on 10.0.0.133 (executor driver) (25/34) -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:43 INFO Executor: Finished task 25.0 in stage 35.0 (TID 802). 10455 bytes result sent to driver -26/04/01 08:45:43 INFO TaskSetManager: Starting task 33.0 in stage 35.0 (TID 810) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9644 bytes) -26/04/01 08:45:43 INFO Executor: Running task 33.0 in stage 35.0 (TID 810) -26/04/01 08:45:43 INFO TaskSetManager: Finished task 25.0 in stage 35.0 (TID 802) in 6900 ms on 10.0.0.133 (executor driver) (26/34) -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 29 (8.1 MiB) non-empty blocks including 29 (8.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 64 (12.0 MiB) non-empty blocks including 64 (12.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 29 (8.1 MiB) non-empty blocks including 29 (8.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Getting 64 (12.0 MiB) non-empty blocks including 64 (12.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:43 INFO Executor: Finished task 26.0 in stage 35.0 (TID 803). 10455 bytes result sent to driver -26/04/01 08:45:43 INFO TaskSetManager: Finished task 26.0 in stage 35.0 (TID 803) in 6838 ms on 10.0.0.133 (executor driver) (27/34) -26/04/01 08:45:43 INFO Executor: Finished task 27.0 in stage 35.0 (TID 804). 10455 bytes result sent to driver -26/04/01 08:45:43 INFO TaskSetManager: Finished task 27.0 in stage 35.0 (TID 804) in 6835 ms on 10.0.0.133 (executor driver) (28/34) -26/04/01 08:45:43 INFO Executor: Finished task 28.0 in stage 35.0 (TID 805). 10455 bytes result sent to driver -26/04/01 08:45:43 INFO TaskSetManager: Finished task 28.0 in stage 35.0 (TID 805) in 6896 ms on 10.0.0.133 (executor driver) (29/34) -26/04/01 08:45:43 INFO Executor: Finished task 29.0 in stage 35.0 (TID 806). 10455 bytes result sent to driver -26/04/01 08:45:43 INFO TaskSetManager: Finished task 29.0 in stage 35.0 (TID 806) in 6889 ms on 10.0.0.133 (executor driver) (30/34) -26/04/01 08:45:43 INFO Executor: Finished task 30.0 in stage 35.0 (TID 807). 10455 bytes result sent to driver -26/04/01 08:45:43 INFO TaskSetManager: Finished task 30.0 in stage 35.0 (TID 807) in 6994 ms on 10.0.0.133 (executor driver) (31/34) -26/04/01 08:45:43 INFO Executor: Finished task 31.0 in stage 35.0 (TID 808). 10455 bytes result sent to driver -26/04/01 08:45:43 INFO TaskSetManager: Finished task 31.0 in stage 35.0 (TID 808) in 6962 ms on 10.0.0.133 (executor driver) (32/34) -26/04/01 08:45:45 INFO Executor: Finished task 33.0 in stage 35.0 (TID 810). 9724 bytes result sent to driver -26/04/01 08:45:45 INFO TaskSetManager: Finished task 33.0 in stage 35.0 (TID 810) in 2160 ms on 10.0.0.133 (executor driver) (33/34) -26/04/01 08:45:49 INFO Executor: Finished task 32.0 in stage 35.0 (TID 809). 10455 bytes result sent to driver -26/04/01 08:45:49 INFO TaskSetManager: Finished task 32.0 in stage 35.0 (TID 809) in 6613 ms on 10.0.0.133 (executor driver) (34/34) -26/04/01 08:45:49 INFO TaskSchedulerImpl: Removed TaskSet 35.0, whose tasks have all completed, from pool -26/04/01 08:45:49 INFO DAGScheduler: ShuffleMapStage 35 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 34.381 s -26/04/01 08:45:49 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:45:49 INFO DAGScheduler: running: Set() -26/04/01 08:45:49 INFO DAGScheduler: waiting: Set() -26/04/01 08:45:49 INFO DAGScheduler: failed: Set() -26/04/01 08:45:49 INFO ShufflePartitionsUtil: For shuffle(8), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 08:45:49 INFO CodeGenerator: Code generated in 65.74125 ms -26/04/01 08:45:49 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:45:49 INFO DAGScheduler: Got job 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 2 output partitions -26/04/01 08:45:49 INFO DAGScheduler: Final stage: ResultStage 44 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:45:49 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 43) -26/04/01 08:45:49 INFO DAGScheduler: Missing parents: List() -26/04/01 08:45:49 INFO DAGScheduler: Submitting ResultStage 44 (MapPartitionsRDD[57] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:45:49 INFO MemoryStore: Block broadcast_27 stored as values in memory (estimated size 20.9 KiB, free 8.6 GiB) -26/04/01 08:45:49 INFO MemoryStore: Block broadcast_27_piece0 stored as bytes in memory (estimated size 8.4 KiB, free 8.6 GiB) -26/04/01 08:45:49 INFO BlockManagerInfo: Added broadcast_27_piece0 in memory on 10.0.0.133:59303 (size: 8.4 KiB, free: 8.6 GiB) -26/04/01 08:45:49 INFO SparkContext: Created broadcast 27 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:45:49 INFO DAGScheduler: Submitting 2 missing tasks from ResultStage 44 (MapPartitionsRDD[57] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1)) -26/04/01 08:45:49 INFO TaskSchedulerImpl: Adding task set 44.0 with 2 tasks resource profile 0 -26/04/01 08:45:49 INFO TaskSetManager: Starting task 0.0 in stage 44.0 (TID 811) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) -26/04/01 08:45:49 INFO TaskSetManager: Starting task 1.0 in stage 44.0 (TID 812) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9415 bytes) -26/04/01 08:45:49 INFO Executor: Running task 1.0 in stage 44.0 (TID 812) -26/04/01 08:45:49 INFO Executor: Running task 0.0 in stage 44.0 (TID 811) -26/04/01 08:45:49 INFO ShuffleBlockFetcherIterator: Getting 34 (1039.8 KiB) non-empty blocks including 34 (1039.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:49 INFO ShuffleBlockFetcherIterator: Getting 34 (1186.0 KiB) non-empty blocks including 34 (1186.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:49 INFO CodeGenerator: Code generated in 7.21075 ms -26/04/01 08:45:50 INFO Executor: Finished task 0.0 in stage 44.0 (TID 811). 8901 bytes result sent to driver -26/04/01 08:45:50 INFO TaskSetManager: Finished task 0.0 in stage 44.0 (TID 811) in 522 ms on 10.0.0.133 (executor driver) (1/2) -26/04/01 08:45:50 INFO Executor: Finished task 1.0 in stage 44.0 (TID 812). 9919 bytes result sent to driver -26/04/01 08:45:50 INFO TaskSetManager: Finished task 1.0 in stage 44.0 (TID 812) in 580 ms on 10.0.0.133 (executor driver) (2/2) -26/04/01 08:45:50 INFO TaskSchedulerImpl: Removed TaskSet 44.0, whose tasks have all completed, from pool -26/04/01 08:45:50 INFO DAGScheduler: ResultStage 44 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.582 s -26/04/01 08:45:50 INFO DAGScheduler: Job 19 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:45:50 INFO TaskSchedulerImpl: Killing all running tasks in stage 44: Stage finished -26/04/01 08:45:50 INFO DAGScheduler: Job 19 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.585016 s -26/04/01 08:45:50 INFO DAGScheduler: Registering RDD 58 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 9 -26/04/01 08:45:50 INFO DAGScheduler: Got map stage job 20 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 2 output partitions -26/04/01 08:45:50 INFO DAGScheduler: Final stage: ShuffleMapStage 53 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:45:50 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 52) -26/04/01 08:45:50 INFO DAGScheduler: Missing parents: List() -26/04/01 08:45:50 INFO DAGScheduler: Submitting ShuffleMapStage 53 (MapPartitionsRDD[58] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:45:50 INFO MemoryStore: Block broadcast_28 stored as values in memory (estimated size 31.4 KiB, free 8.6 GiB) -26/04/01 08:45:50 INFO MemoryStore: Block broadcast_28_piece0 stored as bytes in memory (estimated size 10.6 KiB, free 8.6 GiB) -26/04/01 08:45:50 INFO BlockManagerInfo: Added broadcast_28_piece0 in memory on 10.0.0.133:59303 (size: 10.6 KiB, free: 8.6 GiB) -26/04/01 08:45:50 INFO SparkContext: Created broadcast 28 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:45:50 INFO DAGScheduler: Submitting 2 missing tasks from ShuffleMapStage 53 (MapPartitionsRDD[58] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1)) -26/04/01 08:45:50 INFO TaskSchedulerImpl: Adding task set 53.0 with 2 tasks resource profile 0 -26/04/01 08:45:50 INFO TaskSetManager: Starting task 0.0 in stage 53.0 (TID 813) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9404 bytes) -26/04/01 08:45:50 INFO TaskSetManager: Starting task 1.0 in stage 53.0 (TID 814) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9404 bytes) -26/04/01 08:45:50 INFO Executor: Running task 0.0 in stage 53.0 (TID 813) -26/04/01 08:45:50 INFO Executor: Running task 1.0 in stage 53.0 (TID 814) -26/04/01 08:45:50 INFO CodeGenerator: Code generated in 3.566542 ms -26/04/01 08:45:50 INFO ShuffleBlockFetcherIterator: Getting 34 (1186.0 KiB) non-empty blocks including 34 (1186.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:50 INFO ShuffleBlockFetcherIterator: Getting 34 (1039.8 KiB) non-empty blocks including 34 (1039.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:50 INFO Executor: Finished task 0.0 in stage 53.0 (TID 813). 6566 bytes result sent to driver -26/04/01 08:45:50 INFO TaskSetManager: Finished task 0.0 in stage 53.0 (TID 813) in 491 ms on 10.0.0.133 (executor driver) (1/2) -26/04/01 08:45:50 INFO Executor: Finished task 1.0 in stage 53.0 (TID 814). 6566 bytes result sent to driver -26/04/01 08:45:50 INFO TaskSetManager: Finished task 1.0 in stage 53.0 (TID 814) in 555 ms on 10.0.0.133 (executor driver) (2/2) -26/04/01 08:45:50 INFO TaskSchedulerImpl: Removed TaskSet 53.0, whose tasks have all completed, from pool -26/04/01 08:45:50 INFO DAGScheduler: ShuffleMapStage 53 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.558 s -26/04/01 08:45:50 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:45:50 INFO DAGScheduler: running: Set() -26/04/01 08:45:50 INFO DAGScheduler: waiting: Set() -26/04/01 08:45:50 INFO DAGScheduler: failed: Set() -26/04/01 08:45:50 INFO ShufflePartitionsUtil: For shuffle(9), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 08:45:50 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:45:50 INFO DAGScheduler: Got job 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 08:45:50 INFO DAGScheduler: Final stage: ResultStage 63 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:45:50 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 62) -26/04/01 08:45:50 INFO DAGScheduler: Missing parents: List() -26/04/01 08:45:50 INFO DAGScheduler: Submitting ResultStage 63 (MapPartitionsRDD[62] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:45:50 INFO MemoryStore: Block broadcast_29 stored as values in memory (estimated size 28.6 KiB, free 8.6 GiB) -26/04/01 08:45:50 INFO MemoryStore: Block broadcast_29_piece0 stored as bytes in memory (estimated size 10.4 KiB, free 8.6 GiB) -26/04/01 08:45:50 INFO BlockManagerInfo: Added broadcast_29_piece0 in memory on 10.0.0.133:59303 (size: 10.4 KiB, free: 8.6 GiB) -26/04/01 08:45:50 INFO SparkContext: Created broadcast 29 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:45:50 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 63 (MapPartitionsRDD[62] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:45:50 INFO TaskSchedulerImpl: Adding task set 63.0 with 1 tasks resource profile 0 -26/04/01 08:45:50 INFO TaskSetManager: Starting task 0.0 in stage 63.0 (TID 815) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) -26/04/01 08:45:50 INFO Executor: Running task 0.0 in stage 63.0 (TID 815) -26/04/01 08:45:50 INFO ShuffleBlockFetcherIterator: Getting 2 (76.4 KiB) non-empty blocks including 2 (76.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:45:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:45:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=4096 MB, memoryFraction=1.0, memoryLimit=4096 MB, memoryLimitPerTask=512 MB -26/04/01 08:45:50 INFO Executor: Finished task 0.0 in stage 63.0 (TID 815). 8247 bytes result sent to driver -26/04/01 08:45:50 INFO TaskSetManager: Finished task 0.0 in stage 63.0 (TID 815) in 47 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:45:50 INFO TaskSchedulerImpl: Removed TaskSet 63.0, whose tasks have all completed, from pool -26/04/01 08:45:50 INFO DAGScheduler: ResultStage 63 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.048 s -26/04/01 08:45:50 INFO DAGScheduler: Job 21 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:45:50 INFO TaskSchedulerImpl: Killing all running tasks in stage 63: Stage finished -26/04/01 08:45:50 INFO DAGScheduler: Job 21 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.049918 s -26/04/01 08:45:50 INFO SparkContext: SparkContext is stopping with exitCode 0. -26/04/01 08:45:50 INFO CometDriverPlugin: CometDriverPlugin shutdown -26/04/01 08:45:50 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! -26/04/01 08:45:53 INFO MemoryStore: MemoryStore cleared -26/04/01 08:45:53 INFO BlockManager: BlockManager stopped -26/04/01 08:45:53 INFO BlockManagerMaster: BlockManagerMaster stopped -26/04/01 08:45:53 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! -26/04/01 08:45:53 INFO SparkContext: Successfully stopped SparkContext -26/04/01 08:45:53 INFO ShutdownHookManager: Shutdown hook called -26/04/01 08:45:53 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-cae9fc0e-ad8f-4143-bb25-84be77849f6a -26/04/01 08:45:54 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-3fd37116-6cd4-4238-bd4c-70dc836beec5 -26/04/01 08:45:54 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-cae9fc0e-ad8f-4143-bb25-84be77849f6a/pyspark-f8d215d3-6f69-482a-bccc-77152a4751f3 - 384.09 real 3098.02 user 34.23 sys - 8939782144 maximum resident set size - 0 average shared memory size - 0 average unshared data size - 0 average unshared stack size - 626326 page reclaims - 80 page faults - 0 swaps - 0 block input operations - 0 block output operations - 1278 messages sent - 2365 messages received - 37 signals received - 109370 voluntary context switches - 938768 involuntary context switches - 43620594854966 instructions retired - 11293084515693 cycles elapsed - 3963704144 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.log b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.log deleted file mode 100644 index 1779ac4349..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.log +++ /dev/null @@ -1,76 +0,0 @@ -Registering table customer from /opt/tpch/sf100/customer -Registering table lineitem from /opt/tpch/sf100/lineitem -Registering table nation from /opt/tpch/sf100/nation -Registering table orders from /opt/tpch/sf100/orders -Registering table part from /opt/tpch/sf100/part -Registering table partsupp from /opt/tpch/sf100/partsupp -Registering table region from /opt/tpch/sf100/region -Registering table supplier from /opt/tpch/sf100/supplier - -============================================================ -Starting iteration 1 of 1 -============================================================ - -Running query 1 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q1.sql -Executing: -- CometBench-H query 1 derived from TPC-H query 1 under the terms of the TPC Fair Use Policy. --- TP... -== Physical Plan == -AdaptiveSparkPlan (9) -+- CometSort (8) - +- CometExchange (7) - +- CometHashAggregate (6) - +- CometExchange (5) - +- CometHashAggregate (4) - +- CometProject (3) - +- CometFilter (2) - +- CometNativeScan parquet (1) - - -(1) CometNativeScan parquet -Output [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] -PushedFilters: [IsNotNull(l_shipdate), LessThanOrEqual(l_shipdate,1998-09-24)] -ReadSchema: struct - -(2) CometFilter -Input [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] -Condition : (isnotnull(l_shipdate#26) AND (l_shipdate#26 <= 1998-09-24)) - -(3) CometProject -Input [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] -Arguments: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25], [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25] - -(4) CometHashAggregate -Input [6]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25] -Keys [2]: [l_returnflag#24, l_linestatus#25] -Functions [8]: [partial_sum(l_quantity#20), partial_sum(l_extendedprice#21), partial_sum((l_extendedprice#21 * (1 - l_discount#22))), partial_sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23))), partial_avg(l_quantity#20), partial_avg(l_extendedprice#21), partial_avg(l_discount#22), partial_count(1)] - -(5) CometExchange -Input [17]: [l_returnflag#24, l_linestatus#25, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, count#172L, sum#173, count#174L, sum#175, count#176L, count#177L] -Arguments: hashpartitioning(l_returnflag#24, l_linestatus#25, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=83] - -(6) CometHashAggregate -Input [17]: [l_returnflag#24, l_linestatus#25, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, count#172L, sum#173, count#174L, sum#175, count#176L, count#177L] -Keys [2]: [l_returnflag#24, l_linestatus#25] -Functions [8]: [sum(l_quantity#20), sum(l_extendedprice#21), sum((l_extendedprice#21 * (1 - l_discount#22))), sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23))), avg(l_quantity#20), avg(l_extendedprice#21), avg(l_discount#22), count(1)] - -(7) CometExchange -Input [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] -Arguments: rangepartitioning(l_returnflag#24 ASC NULLS FIRST, l_linestatus#25 ASC NULLS FIRST, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=85] - -(8) CometSort -Input [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] -Arguments: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L], [l_returnflag#24 ASC NULLS FIRST, l_linestatus#25 ASC NULLS FIRST] - -(9) AdaptiveSparkPlan -Output [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] -Arguments: isFinalPlan=false - - -Query 1 returned 4 rows, hash=6839cc802df6220ed2efdaaaa48373a6 -Query 1 took 330.29 seconds - -Iteration 1 took 330.29 seconds - -Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/comet-offheap8g-q1-tpch-1775046669770.json diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.time b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.time deleted file mode 100644 index 29c47eddbd..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q1.time +++ /dev/null @@ -1,1629 +0,0 @@ -26/04/01 06:25:34 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) -26/04/01 06:25:34 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address -26/04/01 06:25:34 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable -26/04/01 06:25:35 INFO SparkContext: Running Spark version 3.5.8 -26/04/01 06:25:35 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 06:25:35 INFO SparkContext: Java version 17.0.17 -26/04/01 06:25:35 INFO ResourceUtils: ============================================================== -26/04/01 06:25:35 INFO ResourceUtils: No custom resources configured for spark.driver. -26/04/01 06:25:35 INFO ResourceUtils: ============================================================== -26/04/01 06:25:35 INFO SparkContext: Submitted application: comet-offheap8g-q1 benchmark derived from tpch -26/04/01 06:25:35 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 8192, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) -26/04/01 06:25:35 INFO ResourceProfile: Limiting resource is cpu -26/04/01 06:25:35 INFO ResourceProfileManager: Added ResourceProfile id: 0 -26/04/01 06:25:35 INFO SecurityManager: Changing view acls to: andy -26/04/01 06:25:35 INFO SecurityManager: Changing modify acls to: andy -26/04/01 06:25:35 INFO SecurityManager: Changing view acls groups to: -26/04/01 06:25:35 INFO SecurityManager: Changing modify acls groups to: -26/04/01 06:25:35 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY -26/04/01 06:25:35 INFO Utils: Successfully started service 'sparkDriver' on port 58416. -26/04/01 06:25:35 INFO SparkEnv: Registering MapOutputTracker -26/04/01 06:25:35 INFO SparkEnv: Registering BlockManagerMaster -26/04/01 06:25:35 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information -26/04/01 06:25:35 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up -26/04/01 06:25:35 INFO SparkEnv: Registering BlockManagerMasterHeartbeat -26/04/01 06:25:35 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-c231e1ae-1a96-48c7-b30d-283875835f41 -26/04/01 06:25:35 INFO MemoryStore: MemoryStore started with capacity 12.6 GiB -26/04/01 06:25:35 INFO SparkEnv: Registering OutputCommitCoordinator -26/04/01 06:25:35 INFO SparkContext: Added JAR file:///Users/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar at spark://10.0.0.133:58416/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775046335237 -26/04/01 06:25:35 INFO CometDriverPlugin: CometDriverPlugin init -26/04/01 06:25:35 INFO CometDriverPlugin: Setting spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions -26/04/01 06:25:35 INFO CometDriverPlugin: Comet metrics reporting is disabled. Set spark.comet.metrics.enabled=true to enable. -26/04/01 06:25:35 INFO CometDriverPlugin: Comet is running in unified memory mode and sharing off-heap memory with Spark -26/04/01 06:25:35 INFO DriverPluginContainer: Initialized driver component for plugin org.apache.spark.CometPlugin. -26/04/01 06:25:35 INFO Executor: Starting executor ID driver on host 10.0.0.133 -26/04/01 06:25:35 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 06:25:35 INFO Executor: Java version 17.0.17 -26/04/01 06:25:35 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' -26/04/01 06:25:35 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@17670b56 for default. -26/04/01 06:25:35 INFO Executor: Fetching spark://10.0.0.133:58416/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775046335237 -26/04/01 06:25:35 INFO TransportClientFactory: Successfully created connection to /10.0.0.133:58416 after 8 ms (0 ms spent in bootstraps) -26/04/01 06:25:35 INFO Utils: Fetching spark://10.0.0.133:58416/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-9d0b1c01-f2bb-4fb8-875a-59c49514dd0e/userFiles-79baa501-cc0c-488c-9287-530627dc7d52/fetchFileTemp4495643812795777425.tmp -26/04/01 06:25:35 INFO Executor: Adding file:/private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-9d0b1c01-f2bb-4fb8-875a-59c49514dd0e/userFiles-79baa501-cc0c-488c-9287-530627dc7d52/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to class loader default -26/04/01 06:25:35 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 58418. -26/04/01 06:25:35 INFO NettyBlockTransferService: Server created on 10.0.0.133:58418 -26/04/01 06:25:35 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy -26/04/01 06:25:35 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 58418, None) -26/04/01 06:25:35 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:58418 with 12.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 58418, None) -26/04/01 06:25:35 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 58418, None) -26/04/01 06:25:35 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 58418, None) -26/04/01 06:25:35 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. -26/04/01 06:25:35 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. -26/04/01 06:25:35 INFO InMemoryFileIndex: It took 16 ms to list leaf files for 1 paths. -26/04/01 06:25:36 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:25:36 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:25:36 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:25:36 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:25:36 INFO DAGScheduler: Missing parents: List() -26/04/01 06:25:36 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:25:36 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:25:36 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) -26/04/01 06:25:36 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:58418 (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:25:36 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:25:36 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:25:36 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 -26/04/01 06:25:36 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 06:25:36 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) -26/04/01 06:25:36 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 2041 bytes result sent to driver -26/04/01 06:25:36 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 118 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:25:36 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool -26/04/01 06:25:36 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.371 s -26/04/01 06:25:36 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:25:36 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished -26/04/01 06:25:36 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.384166 s -26/04/01 06:25:36 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:58418 in memory (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO NativeBase: Couldn't locate log file from either COMET_CONF_DIR or comet.log.file.path. Using default log configuration with INFO log level which emits to stderr -26/04/01 06:25:39 INFO core/src/lib.rs: Comet native library version 0.15.0 initialized -26/04/01 06:25:39 INFO NativeBase: Setting system property arrow.enable_unsafe_memory_access to true -26/04/01 06:25:39 INFO NativeBase: Setting system property arrow.enable_null_check_for_get to false -26/04/01 06:25:39 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. -26/04/01 06:25:39 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:25:39 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:25:39 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() -26/04/01 06:25:39 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:58418 (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:25:39 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 -26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) -26/04/01 06:25:39 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2430 bytes result sent to driver -26/04/01 06:25:39 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 17 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:25:39 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool -26/04/01 06:25:39 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.026 s -26/04/01 06:25:39 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:25:39 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished -26/04/01 06:25:39 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.027665 s -26/04/01 06:25:39 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 06:25:39 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:25:39 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:25:39 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() -26/04/01 06:25:39 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:58418 (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:25:39 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 -26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) -26/04/01 06:25:39 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver -26/04/01 06:25:39 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:25:39 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool -26/04/01 06:25:39 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s -26/04/01 06:25:39 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:25:39 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished -26/04/01 06:25:39 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.016430 s -26/04/01 06:25:39 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. -26/04/01 06:25:39 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:25:39 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:25:39 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() -26/04/01 06:25:39 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:58418 (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:25:39 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 -26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) -26/04/01 06:25:39 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver -26/04/01 06:25:39 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:25:39 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool -26/04/01 06:25:39 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s -26/04/01 06:25:39 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:25:39 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished -26/04/01 06:25:39 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.015428 s -26/04/01 06:25:39 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. -26/04/01 06:25:39 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:25:39 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:25:39 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() -26/04/01 06:25:39 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:58418 (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:58418 in memory (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:25:39 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 -26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9299 bytes) -26/04/01 06:25:39 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:58418 in memory (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) -26/04/01 06:25:39 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:58418 in memory (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver -26/04/01 06:25:39 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 7 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:25:39 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool -26/04/01 06:25:39 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.016 s -26/04/01 06:25:39 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:25:39 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished -26/04/01 06:25:39 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.017624 s -26/04/01 06:25:39 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 06:25:39 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:25:39 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:25:39 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() -26/04/01 06:25:39 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:58418 (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:58418 in memory (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:25:39 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 -26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) -26/04/01 06:25:39 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver -26/04/01 06:25:39 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:25:39 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool -26/04/01 06:25:39 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s -26/04/01 06:25:39 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:25:39 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished -26/04/01 06:25:39 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.015451 s -26/04/01 06:25:39 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. -26/04/01 06:25:39 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:25:39 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:25:39 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() -26/04/01 06:25:39 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:58418 in memory (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:58418 (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:25:39 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 -26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) -26/04/01 06:25:39 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver -26/04/01 06:25:39 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:25:39 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool -26/04/01 06:25:39 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 06:25:39 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:25:39 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished -26/04/01 06:25:39 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.014321 s -26/04/01 06:25:39 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 06:25:39 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:25:39 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:25:39 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() -26/04/01 06:25:39 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:58418 (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:25:39 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:58418 in memory (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 -26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) -26/04/01 06:25:39 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver -26/04/01 06:25:39 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:25:39 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool -26/04/01 06:25:39 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s -26/04/01 06:25:39 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:25:39 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished -26/04/01 06:25:39 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.015871 s -26/04/01 06:25:39 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_shipdate),LessThanOrEqual(l_shipdate,1998-09-24) -26/04/01 06:25:39 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_shipdate#26),(l_shipdate#26 <= 1998-09-24) -26/04/01 06:25:39 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:58418 in memory (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:25:39 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:58418 (size: 35.4 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:25:39 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:25:39 INFO DAGScheduler: Registering RDD 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 -26/04/01 06:25:39 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions -26/04/01 06:25:39 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:25:39 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:25:39 INFO DAGScheduler: Missing parents: List() -26/04/01 06:25:39 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 46.8 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 10.1 KiB, free 12.6 GiB) -26/04/01 06:25:39 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:58418 (size: 10.1 KiB, free: 12.6 GiB) -26/04/01 06:25:39 INFO SparkContext: Created broadcast 9 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:25:39 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 06:25:39 INFO TaskSchedulerImpl: Adding task set 8.0 with 208 tasks resource profile 0 -26/04/01 06:25:39 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:25:39 INFO TaskSetManager: Starting task 1.0 in stage 8.0 (TID 9) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:25:39 INFO TaskSetManager: Starting task 2.0 in stage 8.0 (TID 10) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:25:39 INFO TaskSetManager: Starting task 3.0 in stage 8.0 (TID 11) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:25:39 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) -26/04/01 06:25:39 INFO Executor: Running task 1.0 in stage 8.0 (TID 9) -26/04/01 06:25:39 INFO Executor: Running task 3.0 in stage 8.0 (TID 11) -26/04/01 06:25:39 INFO Executor: Running task 2.0 in stage 8.0 (TID 10) -26/04/01 06:25:39 INFO BaseAllocator: Debug mode disabled. Enable with the VM option -Darrow.memory.debug.allocator=true. -26/04/01 06:25:39 INFO DefaultAllocationManagerOption: allocation manager type not specified, using netty as the default type -26/04/01 06:25:39 INFO CheckAllocator: Using DefaultAllocationManager at memory/unsafe/DefaultAllocationManagerFactory.class -26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:39 INFO core/src/execution/jni_api.rs: Comet tokio runtime: using spark.executor.cores=4 worker threads -26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:46 INFO Executor: Finished task 1.0 in stage 8.0 (TID 9). 6830 bytes result sent to driver -26/04/01 06:25:46 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 6830 bytes result sent to driver -26/04/01 06:25:46 INFO Executor: Finished task 3.0 in stage 8.0 (TID 11). 6830 bytes result sent to driver -26/04/01 06:25:46 INFO Executor: Finished task 2.0 in stage 8.0 (TID 10). 6830 bytes result sent to driver -26/04/01 06:25:46 INFO TaskSetManager: Starting task 4.0 in stage 8.0 (TID 12) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:25:46 INFO Executor: Running task 4.0 in stage 8.0 (TID 12) -26/04/01 06:25:46 INFO TaskSetManager: Starting task 5.0 in stage 8.0 (TID 13) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:25:46 INFO Executor: Running task 5.0 in stage 8.0 (TID 13) -26/04/01 06:25:46 INFO TaskSetManager: Starting task 6.0 in stage 8.0 (TID 14) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:25:46 INFO TaskSetManager: Finished task 3.0 in stage 8.0 (TID 11) in 6895 ms on 10.0.0.133 (executor driver) (1/208) -26/04/01 06:25:46 INFO Executor: Running task 6.0 in stage 8.0 (TID 14) -26/04/01 06:25:46 INFO TaskSetManager: Starting task 7.0 in stage 8.0 (TID 15) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:25:46 INFO Executor: Running task 7.0 in stage 8.0 (TID 15) -26/04/01 06:25:46 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 6898 ms on 10.0.0.133 (executor driver) (2/208) -26/04/01 06:25:46 INFO TaskSetManager: Finished task 1.0 in stage 8.0 (TID 9) in 6898 ms on 10.0.0.133 (executor driver) (3/208) -26/04/01 06:25:46 INFO TaskSetManager: Finished task 2.0 in stage 8.0 (TID 10) in 6898 ms on 10.0.0.133 (executor driver) (4/208) -26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:53 INFO Executor: Finished task 7.0 in stage 8.0 (TID 15). 6830 bytes result sent to driver -26/04/01 06:25:53 INFO TaskSetManager: Starting task 8.0 in stage 8.0 (TID 16) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:25:53 INFO Executor: Finished task 5.0 in stage 8.0 (TID 13). 6830 bytes result sent to driver -26/04/01 06:25:53 INFO Executor: Running task 8.0 in stage 8.0 (TID 16) -26/04/01 06:25:53 INFO TaskSetManager: Starting task 9.0 in stage 8.0 (TID 17) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:25:53 INFO TaskSetManager: Finished task 7.0 in stage 8.0 (TID 15) in 6548 ms on 10.0.0.133 (executor driver) (5/208) -26/04/01 06:25:53 INFO Executor: Running task 9.0 in stage 8.0 (TID 17) -26/04/01 06:25:53 INFO TaskSetManager: Finished task 5.0 in stage 8.0 (TID 13) in 6549 ms on 10.0.0.133 (executor driver) (6/208) -26/04/01 06:25:53 INFO Executor: Finished task 4.0 in stage 8.0 (TID 12). 6830 bytes result sent to driver -26/04/01 06:25:53 INFO TaskSetManager: Starting task 10.0 in stage 8.0 (TID 18) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:25:53 INFO Executor: Running task 10.0 in stage 8.0 (TID 18) -26/04/01 06:25:53 INFO Executor: Finished task 6.0 in stage 8.0 (TID 14). 6830 bytes result sent to driver -26/04/01 06:25:53 INFO TaskSetManager: Finished task 4.0 in stage 8.0 (TID 12) in 6552 ms on 10.0.0.133 (executor driver) (7/208) -26/04/01 06:25:53 INFO TaskSetManager: Starting task 11.0 in stage 8.0 (TID 19) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:25:53 INFO Executor: Running task 11.0 in stage 8.0 (TID 19) -26/04/01 06:25:53 INFO TaskSetManager: Finished task 6.0 in stage 8.0 (TID 14) in 6552 ms on 10.0.0.133 (executor driver) (8/208) -26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:59 INFO Executor: Finished task 9.0 in stage 8.0 (TID 17). 6830 bytes result sent to driver -26/04/01 06:25:59 INFO TaskSetManager: Starting task 12.0 in stage 8.0 (TID 20) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:25:59 INFO Executor: Finished task 11.0 in stage 8.0 (TID 19). 6830 bytes result sent to driver -26/04/01 06:25:59 INFO Executor: Running task 12.0 in stage 8.0 (TID 20) -26/04/01 06:25:59 INFO TaskSetManager: Starting task 13.0 in stage 8.0 (TID 21) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:25:59 INFO Executor: Running task 13.0 in stage 8.0 (TID 21) -26/04/01 06:25:59 INFO Executor: Finished task 8.0 in stage 8.0 (TID 16). 6830 bytes result sent to driver -26/04/01 06:25:59 INFO TaskSetManager: Finished task 9.0 in stage 8.0 (TID 17) in 6542 ms on 10.0.0.133 (executor driver) (9/208) -26/04/01 06:25:59 INFO TaskSetManager: Finished task 11.0 in stage 8.0 (TID 19) in 6540 ms on 10.0.0.133 (executor driver) (10/208) -26/04/01 06:25:59 INFO TaskSetManager: Starting task 14.0 in stage 8.0 (TID 22) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:25:59 INFO Executor: Running task 14.0 in stage 8.0 (TID 22) -26/04/01 06:25:59 INFO TaskSetManager: Finished task 8.0 in stage 8.0 (TID 16) in 6543 ms on 10.0.0.133 (executor driver) (11/208) -26/04/01 06:25:59 INFO Executor: Finished task 10.0 in stage 8.0 (TID 18). 6830 bytes result sent to driver -26/04/01 06:25:59 INFO TaskSetManager: Starting task 15.0 in stage 8.0 (TID 23) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:25:59 INFO Executor: Running task 15.0 in stage 8.0 (TID 23) -26/04/01 06:25:59 INFO TaskSetManager: Finished task 10.0 in stage 8.0 (TID 18) in 6542 ms on 10.0.0.133 (executor driver) (12/208) -26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:25:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:06 INFO Executor: Finished task 14.0 in stage 8.0 (TID 22). 6787 bytes result sent to driver -26/04/01 06:26:06 INFO TaskSetManager: Starting task 16.0 in stage 8.0 (TID 24) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:06 INFO Executor: Running task 16.0 in stage 8.0 (TID 24) -26/04/01 06:26:06 INFO TaskSetManager: Finished task 14.0 in stage 8.0 (TID 22) in 6550 ms on 10.0.0.133 (executor driver) (13/208) -26/04/01 06:26:06 INFO Executor: Finished task 15.0 in stage 8.0 (TID 23). 6787 bytes result sent to driver -26/04/01 06:26:06 INFO TaskSetManager: Starting task 17.0 in stage 8.0 (TID 25) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:06 INFO Executor: Running task 17.0 in stage 8.0 (TID 25) -26/04/01 06:26:06 INFO TaskSetManager: Finished task 15.0 in stage 8.0 (TID 23) in 6551 ms on 10.0.0.133 (executor driver) (14/208) -26/04/01 06:26:06 INFO Executor: Finished task 13.0 in stage 8.0 (TID 21). 6830 bytes result sent to driver -26/04/01 06:26:06 INFO TaskSetManager: Starting task 18.0 in stage 8.0 (TID 26) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:06 INFO TaskSetManager: Finished task 13.0 in stage 8.0 (TID 21) in 6554 ms on 10.0.0.133 (executor driver) (15/208) -26/04/01 06:26:06 INFO Executor: Running task 18.0 in stage 8.0 (TID 26) -26/04/01 06:26:06 INFO Executor: Finished task 12.0 in stage 8.0 (TID 20). 6830 bytes result sent to driver -26/04/01 06:26:06 INFO TaskSetManager: Starting task 19.0 in stage 8.0 (TID 27) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:26:06 INFO Executor: Running task 19.0 in stage 8.0 (TID 27) -26/04/01 06:26:06 INFO TaskSetManager: Finished task 12.0 in stage 8.0 (TID 20) in 6554 ms on 10.0.0.133 (executor driver) (16/208) -26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:12 INFO Executor: Finished task 18.0 in stage 8.0 (TID 26). 6787 bytes result sent to driver -26/04/01 06:26:12 INFO Executor: Finished task 19.0 in stage 8.0 (TID 27). 6787 bytes result sent to driver -26/04/01 06:26:12 INFO Executor: Finished task 17.0 in stage 8.0 (TID 25). 6830 bytes result sent to driver -26/04/01 06:26:12 INFO TaskSetManager: Starting task 20.0 in stage 8.0 (TID 28) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:26:12 INFO Executor: Finished task 16.0 in stage 8.0 (TID 24). 6787 bytes result sent to driver -26/04/01 06:26:12 INFO Executor: Running task 20.0 in stage 8.0 (TID 28) -26/04/01 06:26:12 INFO TaskSetManager: Starting task 21.0 in stage 8.0 (TID 29) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:26:12 INFO TaskSetManager: Finished task 18.0 in stage 8.0 (TID 26) in 6547 ms on 10.0.0.133 (executor driver) (17/208) -26/04/01 06:26:12 INFO Executor: Running task 21.0 in stage 8.0 (TID 29) -26/04/01 06:26:12 INFO TaskSetManager: Finished task 19.0 in stage 8.0 (TID 27) in 6546 ms on 10.0.0.133 (executor driver) (18/208) -26/04/01 06:26:12 INFO TaskSetManager: Starting task 22.0 in stage 8.0 (TID 30) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:26:12 INFO TaskSetManager: Starting task 23.0 in stage 8.0 (TID 31) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:26:12 INFO Executor: Running task 22.0 in stage 8.0 (TID 30) -26/04/01 06:26:12 INFO TaskSetManager: Finished task 17.0 in stage 8.0 (TID 25) in 6547 ms on 10.0.0.133 (executor driver) (19/208) -26/04/01 06:26:12 INFO Executor: Running task 23.0 in stage 8.0 (TID 31) -26/04/01 06:26:12 INFO TaskSetManager: Finished task 16.0 in stage 8.0 (TID 24) in 6550 ms on 10.0.0.133 (executor driver) (20/208) -26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:19 INFO Executor: Finished task 23.0 in stage 8.0 (TID 31). 6830 bytes result sent to driver -26/04/01 06:26:19 INFO Executor: Finished task 22.0 in stage 8.0 (TID 30). 6830 bytes result sent to driver -26/04/01 06:26:19 INFO TaskSetManager: Starting task 24.0 in stage 8.0 (TID 32) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:26:19 INFO Executor: Finished task 21.0 in stage 8.0 (TID 29). 6830 bytes result sent to driver -26/04/01 06:26:19 INFO TaskSetManager: Starting task 25.0 in stage 8.0 (TID 33) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:26:19 INFO Executor: Running task 24.0 in stage 8.0 (TID 32) -26/04/01 06:26:19 INFO TaskSetManager: Finished task 23.0 in stage 8.0 (TID 31) in 6530 ms on 10.0.0.133 (executor driver) (21/208) -26/04/01 06:26:19 INFO Executor: Running task 25.0 in stage 8.0 (TID 33) -26/04/01 06:26:19 INFO TaskSetManager: Finished task 22.0 in stage 8.0 (TID 30) in 6531 ms on 10.0.0.133 (executor driver) (22/208) -26/04/01 06:26:19 INFO TaskSetManager: Starting task 26.0 in stage 8.0 (TID 34) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:19 INFO TaskSetManager: Finished task 21.0 in stage 8.0 (TID 29) in 6532 ms on 10.0.0.133 (executor driver) (23/208) -26/04/01 06:26:19 INFO Executor: Running task 26.0 in stage 8.0 (TID 34) -26/04/01 06:26:19 INFO Executor: Finished task 20.0 in stage 8.0 (TID 28). 6830 bytes result sent to driver -26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:19 INFO TaskSetManager: Starting task 27.0 in stage 8.0 (TID 35) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:19 INFO Executor: Running task 27.0 in stage 8.0 (TID 35) -26/04/01 06:26:19 INFO TaskSetManager: Finished task 20.0 in stage 8.0 (TID 28) in 6534 ms on 10.0.0.133 (executor driver) (24/208) -26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:25 INFO Executor: Finished task 25.0 in stage 8.0 (TID 33). 6873 bytes result sent to driver -26/04/01 06:26:25 INFO Executor: Finished task 27.0 in stage 8.0 (TID 35). 6873 bytes result sent to driver -26/04/01 06:26:25 INFO TaskSetManager: Starting task 28.0 in stage 8.0 (TID 36) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:25 INFO TaskSetManager: Starting task 29.0 in stage 8.0 (TID 37) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:25 INFO Executor: Running task 28.0 in stage 8.0 (TID 36) -26/04/01 06:26:25 INFO Executor: Running task 29.0 in stage 8.0 (TID 37) -26/04/01 06:26:25 INFO TaskSetManager: Finished task 25.0 in stage 8.0 (TID 33) in 6463 ms on 10.0.0.133 (executor driver) (25/208) -26/04/01 06:26:25 INFO TaskSetManager: Finished task 27.0 in stage 8.0 (TID 35) in 6461 ms on 10.0.0.133 (executor driver) (26/208) -26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:25 INFO Executor: Finished task 24.0 in stage 8.0 (TID 32). 6830 bytes result sent to driver -26/04/01 06:26:25 INFO Executor: Finished task 26.0 in stage 8.0 (TID 34). 6830 bytes result sent to driver -26/04/01 06:26:25 INFO TaskSetManager: Starting task 30.0 in stage 8.0 (TID 38) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:26:25 INFO Executor: Running task 30.0 in stage 8.0 (TID 38) -26/04/01 06:26:25 INFO TaskSetManager: Starting task 31.0 in stage 8.0 (TID 39) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:26:25 INFO TaskSetManager: Finished task 24.0 in stage 8.0 (TID 32) in 6469 ms on 10.0.0.133 (executor driver) (27/208) -26/04/01 06:26:25 INFO Executor: Running task 31.0 in stage 8.0 (TID 39) -26/04/01 06:26:25 INFO TaskSetManager: Finished task 26.0 in stage 8.0 (TID 34) in 6468 ms on 10.0.0.133 (executor driver) (28/208) -26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:32 INFO Executor: Finished task 28.0 in stage 8.0 (TID 36). 6787 bytes result sent to driver -26/04/01 06:26:32 INFO Executor: Finished task 30.0 in stage 8.0 (TID 38). 6787 bytes result sent to driver -26/04/01 06:26:32 INFO TaskSetManager: Starting task 32.0 in stage 8.0 (TID 40) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:32 INFO TaskSetManager: Starting task 33.0 in stage 8.0 (TID 41) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:32 INFO Executor: Running task 32.0 in stage 8.0 (TID 40) -26/04/01 06:26:32 INFO TaskSetManager: Finished task 28.0 in stage 8.0 (TID 36) in 6475 ms on 10.0.0.133 (executor driver) (29/208) -26/04/01 06:26:32 INFO Executor: Running task 33.0 in stage 8.0 (TID 41) -26/04/01 06:26:32 INFO TaskSetManager: Finished task 30.0 in stage 8.0 (TID 38) in 6470 ms on 10.0.0.133 (executor driver) (30/208) -26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:32 INFO Executor: Finished task 29.0 in stage 8.0 (TID 37). 6787 bytes result sent to driver -26/04/01 06:26:32 INFO TaskSetManager: Starting task 34.0 in stage 8.0 (TID 42) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:32 INFO TaskSetManager: Finished task 29.0 in stage 8.0 (TID 37) in 6485 ms on 10.0.0.133 (executor driver) (31/208) -26/04/01 06:26:32 INFO Executor: Running task 34.0 in stage 8.0 (TID 42) -26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:32 INFO Executor: Finished task 31.0 in stage 8.0 (TID 39). 6787 bytes result sent to driver -26/04/01 06:26:32 INFO TaskSetManager: Starting task 35.0 in stage 8.0 (TID 43) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:32 INFO TaskSetManager: Finished task 31.0 in stage 8.0 (TID 39) in 6484 ms on 10.0.0.133 (executor driver) (32/208) -26/04/01 06:26:32 INFO Executor: Running task 35.0 in stage 8.0 (TID 43) -26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:38 INFO Executor: Finished task 32.0 in stage 8.0 (TID 40). 6787 bytes result sent to driver -26/04/01 06:26:38 INFO Executor: Finished task 33.0 in stage 8.0 (TID 41). 6787 bytes result sent to driver -26/04/01 06:26:38 INFO TaskSetManager: Starting task 36.0 in stage 8.0 (TID 44) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:26:38 INFO Executor: Running task 36.0 in stage 8.0 (TID 44) -26/04/01 06:26:38 INFO TaskSetManager: Starting task 37.0 in stage 8.0 (TID 45) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:26:38 INFO TaskSetManager: Finished task 32.0 in stage 8.0 (TID 40) in 6461 ms on 10.0.0.133 (executor driver) (33/208) -26/04/01 06:26:38 INFO Executor: Running task 37.0 in stage 8.0 (TID 45) -26/04/01 06:26:38 INFO TaskSetManager: Finished task 33.0 in stage 8.0 (TID 41) in 6461 ms on 10.0.0.133 (executor driver) (34/208) -26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:38 INFO Executor: Finished task 34.0 in stage 8.0 (TID 42). 6787 bytes result sent to driver -26/04/01 06:26:38 INFO TaskSetManager: Starting task 38.0 in stage 8.0 (TID 46) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:38 INFO Executor: Running task 38.0 in stage 8.0 (TID 46) -26/04/01 06:26:38 INFO TaskSetManager: Finished task 34.0 in stage 8.0 (TID 42) in 6457 ms on 10.0.0.133 (executor driver) (35/208) -26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:38 INFO Executor: Finished task 35.0 in stage 8.0 (TID 43). 6787 bytes result sent to driver -26/04/01 06:26:38 INFO TaskSetManager: Starting task 39.0 in stage 8.0 (TID 47) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:38 INFO Executor: Running task 39.0 in stage 8.0 (TID 47) -26/04/01 06:26:38 INFO TaskSetManager: Finished task 35.0 in stage 8.0 (TID 43) in 6472 ms on 10.0.0.133 (executor driver) (36/208) -26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:45 INFO Executor: Finished task 37.0 in stage 8.0 (TID 45). 6787 bytes result sent to driver -26/04/01 06:26:45 INFO TaskSetManager: Starting task 40.0 in stage 8.0 (TID 48) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:45 INFO Executor: Finished task 36.0 in stage 8.0 (TID 44). 6787 bytes result sent to driver -26/04/01 06:26:45 INFO TaskSetManager: Finished task 37.0 in stage 8.0 (TID 45) in 6538 ms on 10.0.0.133 (executor driver) (37/208) -26/04/01 06:26:45 INFO Executor: Running task 40.0 in stage 8.0 (TID 48) -26/04/01 06:26:45 INFO TaskSetManager: Starting task 41.0 in stage 8.0 (TID 49) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:45 INFO Executor: Running task 41.0 in stage 8.0 (TID 49) -26/04/01 06:26:45 INFO TaskSetManager: Finished task 36.0 in stage 8.0 (TID 44) in 6539 ms on 10.0.0.133 (executor driver) (38/208) -26/04/01 06:26:45 INFO Executor: Finished task 38.0 in stage 8.0 (TID 46). 6787 bytes result sent to driver -26/04/01 06:26:45 INFO TaskSetManager: Starting task 42.0 in stage 8.0 (TID 50) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:26:45 INFO TaskSetManager: Finished task 38.0 in stage 8.0 (TID 46) in 6534 ms on 10.0.0.133 (executor driver) (39/208) -26/04/01 06:26:45 INFO Executor: Running task 42.0 in stage 8.0 (TID 50) -26/04/01 06:26:45 INFO Executor: Finished task 39.0 in stage 8.0 (TID 47). 6787 bytes result sent to driver -26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:45 INFO TaskSetManager: Starting task 43.0 in stage 8.0 (TID 51) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:26:45 INFO Executor: Running task 43.0 in stage 8.0 (TID 51) -26/04/01 06:26:45 INFO TaskSetManager: Finished task 39.0 in stage 8.0 (TID 47) in 6515 ms on 10.0.0.133 (executor driver) (40/208) -26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:51 INFO Executor: Finished task 42.0 in stage 8.0 (TID 50). 6787 bytes result sent to driver -26/04/01 06:26:51 INFO Executor: Finished task 43.0 in stage 8.0 (TID 51). 6787 bytes result sent to driver -26/04/01 06:26:51 INFO TaskSetManager: Starting task 44.0 in stage 8.0 (TID 52) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:26:51 INFO TaskSetManager: Starting task 45.0 in stage 8.0 (TID 53) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:26:51 INFO Executor: Running task 44.0 in stage 8.0 (TID 52) -26/04/01 06:26:51 INFO TaskSetManager: Finished task 42.0 in stage 8.0 (TID 50) in 6454 ms on 10.0.0.133 (executor driver) (41/208) -26/04/01 06:26:51 INFO TaskSetManager: Finished task 43.0 in stage 8.0 (TID 51) in 6453 ms on 10.0.0.133 (executor driver) (42/208) -26/04/01 06:26:51 INFO Executor: Running task 45.0 in stage 8.0 (TID 53) -26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:51 INFO Executor: Finished task 41.0 in stage 8.0 (TID 49). 6787 bytes result sent to driver -26/04/01 06:26:51 INFO TaskSetManager: Starting task 46.0 in stage 8.0 (TID 54) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:26:51 INFO TaskSetManager: Finished task 41.0 in stage 8.0 (TID 49) in 6491 ms on 10.0.0.133 (executor driver) (43/208) -26/04/01 06:26:51 INFO Executor: Running task 46.0 in stage 8.0 (TID 54) -26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:51 INFO Executor: Finished task 40.0 in stage 8.0 (TID 48). 6787 bytes result sent to driver -26/04/01 06:26:51 INFO TaskSetManager: Starting task 47.0 in stage 8.0 (TID 55) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:26:51 INFO Executor: Running task 47.0 in stage 8.0 (TID 55) -26/04/01 06:26:51 INFO TaskSetManager: Finished task 40.0 in stage 8.0 (TID 48) in 6498 ms on 10.0.0.133 (executor driver) (44/208) -26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:58 INFO Executor: Finished task 45.0 in stage 8.0 (TID 53). 6787 bytes result sent to driver -26/04/01 06:26:58 INFO TaskSetManager: Starting task 48.0 in stage 8.0 (TID 56) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:26:58 INFO Executor: Running task 48.0 in stage 8.0 (TID 56) -26/04/01 06:26:58 INFO TaskSetManager: Finished task 45.0 in stage 8.0 (TID 53) in 6221 ms on 10.0.0.133 (executor driver) (45/208) -26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:58 INFO Executor: Finished task 47.0 in stage 8.0 (TID 55). 6787 bytes result sent to driver -26/04/01 06:26:58 INFO TaskSetManager: Starting task 49.0 in stage 8.0 (TID 57) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:26:58 INFO TaskSetManager: Finished task 47.0 in stage 8.0 (TID 55) in 6455 ms on 10.0.0.133 (executor driver) (46/208) -26/04/01 06:26:58 INFO Executor: Running task 49.0 in stage 8.0 (TID 57) -26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:58 INFO Executor: Finished task 46.0 in stage 8.0 (TID 54). 6787 bytes result sent to driver -26/04/01 06:26:58 INFO TaskSetManager: Starting task 50.0 in stage 8.0 (TID 58) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:58 INFO TaskSetManager: Finished task 46.0 in stage 8.0 (TID 54) in 6468 ms on 10.0.0.133 (executor driver) (47/208) -26/04/01 06:26:58 INFO Executor: Running task 50.0 in stage 8.0 (TID 58) -26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:58 INFO Executor: Finished task 44.0 in stage 8.0 (TID 52). 6787 bytes result sent to driver -26/04/01 06:26:58 INFO TaskSetManager: Starting task 51.0 in stage 8.0 (TID 59) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:26:58 INFO TaskSetManager: Finished task 44.0 in stage 8.0 (TID 52) in 6711 ms on 10.0.0.133 (executor driver) (48/208) -26/04/01 06:26:58 INFO Executor: Running task 51.0 in stage 8.0 (TID 59) -26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:26:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:04 INFO Executor: Finished task 48.0 in stage 8.0 (TID 56). 6787 bytes result sent to driver -26/04/01 06:27:04 INFO TaskSetManager: Starting task 52.0 in stage 8.0 (TID 60) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:04 INFO Executor: Running task 52.0 in stage 8.0 (TID 60) -26/04/01 06:27:04 INFO TaskSetManager: Finished task 48.0 in stage 8.0 (TID 56) in 6527 ms on 10.0.0.133 (executor driver) (49/208) -26/04/01 06:27:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:04 INFO Executor: Finished task 49.0 in stage 8.0 (TID 57). 6787 bytes result sent to driver -26/04/01 06:27:04 INFO TaskSetManager: Starting task 53.0 in stage 8.0 (TID 61) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:04 INFO Executor: Running task 53.0 in stage 8.0 (TID 61) -26/04/01 06:27:04 INFO TaskSetManager: Finished task 49.0 in stage 8.0 (TID 57) in 6498 ms on 10.0.0.133 (executor driver) (50/208) -26/04/01 06:27:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:04 INFO Executor: Finished task 50.0 in stage 8.0 (TID 58). 6787 bytes result sent to driver -26/04/01 06:27:04 INFO TaskSetManager: Starting task 54.0 in stage 8.0 (TID 62) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:27:04 INFO TaskSetManager: Finished task 50.0 in stage 8.0 (TID 58) in 6499 ms on 10.0.0.133 (executor driver) (51/208) -26/04/01 06:27:04 INFO Executor: Running task 54.0 in stage 8.0 (TID 62) -26/04/01 06:27:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:05 INFO Executor: Finished task 51.0 in stage 8.0 (TID 59). 6787 bytes result sent to driver -26/04/01 06:27:05 INFO TaskSetManager: Starting task 55.0 in stage 8.0 (TID 63) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:27:05 INFO Executor: Running task 55.0 in stage 8.0 (TID 63) -26/04/01 06:27:05 INFO TaskSetManager: Finished task 51.0 in stage 8.0 (TID 59) in 6536 ms on 10.0.0.133 (executor driver) (52/208) -26/04/01 06:27:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:11 INFO Executor: Finished task 52.0 in stage 8.0 (TID 60). 6787 bytes result sent to driver -26/04/01 06:27:11 INFO TaskSetManager: Starting task 56.0 in stage 8.0 (TID 64) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:11 INFO Executor: Running task 56.0 in stage 8.0 (TID 64) -26/04/01 06:27:11 INFO TaskSetManager: Finished task 52.0 in stage 8.0 (TID 60) in 6497 ms on 10.0.0.133 (executor driver) (53/208) -26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:11 INFO Executor: Finished task 53.0 in stage 8.0 (TID 61). 6787 bytes result sent to driver -26/04/01 06:27:11 INFO TaskSetManager: Starting task 57.0 in stage 8.0 (TID 65) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:11 INFO TaskSetManager: Finished task 53.0 in stage 8.0 (TID 61) in 6498 ms on 10.0.0.133 (executor driver) (54/208) -26/04/01 06:27:11 INFO Executor: Running task 57.0 in stage 8.0 (TID 65) -26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:11 INFO Executor: Finished task 54.0 in stage 8.0 (TID 62). 6787 bytes result sent to driver -26/04/01 06:27:11 INFO TaskSetManager: Starting task 58.0 in stage 8.0 (TID 66) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:11 INFO Executor: Running task 58.0 in stage 8.0 (TID 66) -26/04/01 06:27:11 INFO TaskSetManager: Finished task 54.0 in stage 8.0 (TID 62) in 6501 ms on 10.0.0.133 (executor driver) (55/208) -26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:11 INFO Executor: Finished task 55.0 in stage 8.0 (TID 63). 6787 bytes result sent to driver -26/04/01 06:27:11 INFO TaskSetManager: Starting task 59.0 in stage 8.0 (TID 67) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:11 INFO TaskSetManager: Finished task 55.0 in stage 8.0 (TID 63) in 6496 ms on 10.0.0.133 (executor driver) (56/208) -26/04/01 06:27:11 INFO Executor: Running task 59.0 in stage 8.0 (TID 67) -26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:17 INFO Executor: Finished task 56.0 in stage 8.0 (TID 64). 6787 bytes result sent to driver -26/04/01 06:27:17 INFO TaskSetManager: Starting task 60.0 in stage 8.0 (TID 68) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:27:17 INFO Executor: Running task 60.0 in stage 8.0 (TID 68) -26/04/01 06:27:17 INFO TaskSetManager: Finished task 56.0 in stage 8.0 (TID 64) in 6536 ms on 10.0.0.133 (executor driver) (57/208) -26/04/01 06:27:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:17 INFO Executor: Finished task 58.0 in stage 8.0 (TID 66). 6787 bytes result sent to driver -26/04/01 06:27:17 INFO TaskSetManager: Starting task 61.0 in stage 8.0 (TID 69) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:27:17 INFO TaskSetManager: Finished task 58.0 in stage 8.0 (TID 66) in 6525 ms on 10.0.0.133 (executor driver) (58/208) -26/04/01 06:27:17 INFO Executor: Running task 61.0 in stage 8.0 (TID 69) -26/04/01 06:27:17 INFO Executor: Finished task 57.0 in stage 8.0 (TID 65). 6787 bytes result sent to driver -26/04/01 06:27:17 INFO TaskSetManager: Starting task 62.0 in stage 8.0 (TID 70) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:17 INFO TaskSetManager: Finished task 57.0 in stage 8.0 (TID 65) in 6535 ms on 10.0.0.133 (executor driver) (59/208) -26/04/01 06:27:17 INFO Executor: Running task 62.0 in stage 8.0 (TID 70) -26/04/01 06:27:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:18 INFO Executor: Finished task 59.0 in stage 8.0 (TID 67). 6787 bytes result sent to driver -26/04/01 06:27:18 INFO TaskSetManager: Starting task 63.0 in stage 8.0 (TID 71) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:18 INFO TaskSetManager: Finished task 59.0 in stage 8.0 (TID 67) in 6537 ms on 10.0.0.133 (executor driver) (60/208) -26/04/01 06:27:18 INFO Executor: Running task 63.0 in stage 8.0 (TID 71) -26/04/01 06:27:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:24 INFO Executor: Finished task 60.0 in stage 8.0 (TID 68). 6787 bytes result sent to driver -26/04/01 06:27:24 INFO TaskSetManager: Starting task 64.0 in stage 8.0 (TID 72) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:24 INFO Executor: Running task 64.0 in stage 8.0 (TID 72) -26/04/01 06:27:24 INFO TaskSetManager: Finished task 60.0 in stage 8.0 (TID 68) in 6539 ms on 10.0.0.133 (executor driver) (61/208) -26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:24 INFO Executor: Finished task 61.0 in stage 8.0 (TID 69). 6787 bytes result sent to driver -26/04/01 06:27:24 INFO Executor: Finished task 62.0 in stage 8.0 (TID 70). 6787 bytes result sent to driver -26/04/01 06:27:24 INFO TaskSetManager: Starting task 65.0 in stage 8.0 (TID 73) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:24 INFO Executor: Running task 65.0 in stage 8.0 (TID 73) -26/04/01 06:27:24 INFO TaskSetManager: Starting task 66.0 in stage 8.0 (TID 74) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:27:24 INFO TaskSetManager: Finished task 61.0 in stage 8.0 (TID 69) in 6542 ms on 10.0.0.133 (executor driver) (62/208) -26/04/01 06:27:24 INFO Executor: Running task 66.0 in stage 8.0 (TID 74) -26/04/01 06:27:24 INFO TaskSetManager: Finished task 62.0 in stage 8.0 (TID 70) in 6542 ms on 10.0.0.133 (executor driver) (63/208) -26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:24 INFO Executor: Finished task 63.0 in stage 8.0 (TID 71). 6787 bytes result sent to driver -26/04/01 06:27:24 INFO TaskSetManager: Starting task 67.0 in stage 8.0 (TID 75) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:27:24 INFO Executor: Running task 67.0 in stage 8.0 (TID 75) -26/04/01 06:27:24 INFO TaskSetManager: Finished task 63.0 in stage 8.0 (TID 71) in 6539 ms on 10.0.0.133 (executor driver) (64/208) -26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:30 INFO Executor: Finished task 64.0 in stage 8.0 (TID 72). 6787 bytes result sent to driver -26/04/01 06:27:30 INFO TaskSetManager: Starting task 68.0 in stage 8.0 (TID 76) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:27:30 INFO TaskSetManager: Finished task 64.0 in stage 8.0 (TID 72) in 6480 ms on 10.0.0.133 (executor driver) (65/208) -26/04/01 06:27:30 INFO Executor: Running task 68.0 in stage 8.0 (TID 76) -26/04/01 06:27:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:30 INFO Executor: Finished task 66.0 in stage 8.0 (TID 74). 6787 bytes result sent to driver -26/04/01 06:27:30 INFO TaskSetManager: Starting task 69.0 in stage 8.0 (TID 77) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:27:30 INFO TaskSetManager: Finished task 66.0 in stage 8.0 (TID 74) in 6444 ms on 10.0.0.133 (executor driver) (66/208) -26/04/01 06:27:30 INFO Executor: Running task 69.0 in stage 8.0 (TID 77) -26/04/01 06:27:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:30 INFO Executor: Finished task 65.0 in stage 8.0 (TID 73). 6787 bytes result sent to driver -26/04/01 06:27:30 INFO TaskSetManager: Starting task 70.0 in stage 8.0 (TID 78) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:27:30 INFO TaskSetManager: Finished task 65.0 in stage 8.0 (TID 73) in 6486 ms on 10.0.0.133 (executor driver) (67/208) -26/04/01 06:27:30 INFO Executor: Running task 70.0 in stage 8.0 (TID 78) -26/04/01 06:27:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:31 INFO Executor: Finished task 67.0 in stage 8.0 (TID 75). 6787 bytes result sent to driver -26/04/01 06:27:31 INFO TaskSetManager: Starting task 71.0 in stage 8.0 (TID 79) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:27:31 INFO TaskSetManager: Finished task 67.0 in stage 8.0 (TID 75) in 6440 ms on 10.0.0.133 (executor driver) (68/208) -26/04/01 06:27:31 INFO Executor: Running task 71.0 in stage 8.0 (TID 79) -26/04/01 06:27:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:37 INFO Executor: Finished task 68.0 in stage 8.0 (TID 76). 6787 bytes result sent to driver -26/04/01 06:27:37 INFO TaskSetManager: Starting task 72.0 in stage 8.0 (TID 80) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:27:37 INFO TaskSetManager: Finished task 68.0 in stage 8.0 (TID 76) in 6446 ms on 10.0.0.133 (executor driver) (69/208) -26/04/01 06:27:37 INFO Executor: Running task 72.0 in stage 8.0 (TID 80) -26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:37 INFO Executor: Finished task 69.0 in stage 8.0 (TID 77). 6830 bytes result sent to driver -26/04/01 06:27:37 INFO TaskSetManager: Starting task 73.0 in stage 8.0 (TID 81) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:27:37 INFO TaskSetManager: Finished task 69.0 in stage 8.0 (TID 77) in 6444 ms on 10.0.0.133 (executor driver) (70/208) -26/04/01 06:27:37 INFO Executor: Running task 73.0 in stage 8.0 (TID 81) -26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:37 INFO Executor: Finished task 70.0 in stage 8.0 (TID 78). 6830 bytes result sent to driver -26/04/01 06:27:37 INFO TaskSetManager: Starting task 74.0 in stage 8.0 (TID 82) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:27:37 INFO Executor: Running task 74.0 in stage 8.0 (TID 82) -26/04/01 06:27:37 INFO TaskSetManager: Finished task 70.0 in stage 8.0 (TID 78) in 6455 ms on 10.0.0.133 (executor driver) (71/208) -26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:37 INFO Executor: Finished task 71.0 in stage 8.0 (TID 79). 6830 bytes result sent to driver -26/04/01 06:27:37 INFO TaskSetManager: Starting task 75.0 in stage 8.0 (TID 83) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:27:37 INFO Executor: Running task 75.0 in stage 8.0 (TID 83) -26/04/01 06:27:37 INFO TaskSetManager: Finished task 71.0 in stage 8.0 (TID 79) in 6448 ms on 10.0.0.133 (executor driver) (72/208) -26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:43 INFO Executor: Finished task 72.0 in stage 8.0 (TID 80). 6830 bytes result sent to driver -26/04/01 06:27:43 INFO TaskSetManager: Starting task 76.0 in stage 8.0 (TID 84) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:27:43 INFO TaskSetManager: Finished task 72.0 in stage 8.0 (TID 80) in 6499 ms on 10.0.0.133 (executor driver) (73/208) -26/04/01 06:27:43 INFO Executor: Running task 76.0 in stage 8.0 (TID 84) -26/04/01 06:27:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:43 INFO Executor: Finished task 73.0 in stage 8.0 (TID 81). 6787 bytes result sent to driver -26/04/01 06:27:43 INFO TaskSetManager: Starting task 77.0 in stage 8.0 (TID 85) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:27:43 INFO TaskSetManager: Finished task 73.0 in stage 8.0 (TID 81) in 6515 ms on 10.0.0.133 (executor driver) (74/208) -26/04/01 06:27:43 INFO Executor: Running task 77.0 in stage 8.0 (TID 85) -26/04/01 06:27:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:43 INFO Executor: Finished task 74.0 in stage 8.0 (TID 82). 6787 bytes result sent to driver -26/04/01 06:27:43 INFO TaskSetManager: Starting task 78.0 in stage 8.0 (TID 86) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:27:43 INFO TaskSetManager: Finished task 74.0 in stage 8.0 (TID 82) in 6498 ms on 10.0.0.133 (executor driver) (75/208) -26/04/01 06:27:43 INFO Executor: Running task 78.0 in stage 8.0 (TID 86) -26/04/01 06:27:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:44 INFO Executor: Finished task 75.0 in stage 8.0 (TID 83). 6787 bytes result sent to driver -26/04/01 06:27:44 INFO TaskSetManager: Starting task 79.0 in stage 8.0 (TID 87) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:27:44 INFO TaskSetManager: Finished task 75.0 in stage 8.0 (TID 83) in 6520 ms on 10.0.0.133 (executor driver) (76/208) -26/04/01 06:27:44 INFO Executor: Running task 79.0 in stage 8.0 (TID 87) -26/04/01 06:27:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:50 INFO Executor: Finished task 76.0 in stage 8.0 (TID 84). 6787 bytes result sent to driver -26/04/01 06:27:50 INFO TaskSetManager: Starting task 80.0 in stage 8.0 (TID 88) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:50 INFO TaskSetManager: Finished task 76.0 in stage 8.0 (TID 84) in 6507 ms on 10.0.0.133 (executor driver) (77/208) -26/04/01 06:27:50 INFO Executor: Running task 80.0 in stage 8.0 (TID 88) -26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:50 INFO Executor: Finished task 77.0 in stage 8.0 (TID 85). 6787 bytes result sent to driver -26/04/01 06:27:50 INFO TaskSetManager: Starting task 81.0 in stage 8.0 (TID 89) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:50 INFO Executor: Running task 81.0 in stage 8.0 (TID 89) -26/04/01 06:27:50 INFO TaskSetManager: Finished task 77.0 in stage 8.0 (TID 85) in 6525 ms on 10.0.0.133 (executor driver) (78/208) -26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:50 INFO Executor: Finished task 78.0 in stage 8.0 (TID 86). 6787 bytes result sent to driver -26/04/01 06:27:50 INFO TaskSetManager: Starting task 82.0 in stage 8.0 (TID 90) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:50 INFO TaskSetManager: Finished task 78.0 in stage 8.0 (TID 86) in 6507 ms on 10.0.0.133 (executor driver) (79/208) -26/04/01 06:27:50 INFO Executor: Running task 82.0 in stage 8.0 (TID 90) -26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:50 INFO Executor: Finished task 79.0 in stage 8.0 (TID 87). 6787 bytes result sent to driver -26/04/01 06:27:50 INFO TaskSetManager: Starting task 83.0 in stage 8.0 (TID 91) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:50 INFO TaskSetManager: Finished task 79.0 in stage 8.0 (TID 87) in 6525 ms on 10.0.0.133 (executor driver) (80/208) -26/04/01 06:27:50 INFO Executor: Running task 83.0 in stage 8.0 (TID 91) -26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:56 INFO Executor: Finished task 80.0 in stage 8.0 (TID 88). 6787 bytes result sent to driver -26/04/01 06:27:56 INFO TaskSetManager: Starting task 84.0 in stage 8.0 (TID 92) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:27:56 INFO Executor: Running task 84.0 in stage 8.0 (TID 92) -26/04/01 06:27:56 INFO TaskSetManager: Finished task 80.0 in stage 8.0 (TID 88) in 6534 ms on 10.0.0.133 (executor driver) (81/208) -26/04/01 06:27:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:56 INFO Executor: Finished task 81.0 in stage 8.0 (TID 89). 6787 bytes result sent to driver -26/04/01 06:27:56 INFO TaskSetManager: Starting task 85.0 in stage 8.0 (TID 93) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:27:56 INFO Executor: Running task 85.0 in stage 8.0 (TID 93) -26/04/01 06:27:56 INFO TaskSetManager: Finished task 81.0 in stage 8.0 (TID 89) in 6532 ms on 10.0.0.133 (executor driver) (82/208) -26/04/01 06:27:56 INFO Executor: Finished task 82.0 in stage 8.0 (TID 90). 6787 bytes result sent to driver -26/04/01 06:27:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:56 INFO TaskSetManager: Starting task 86.0 in stage 8.0 (TID 94) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:56 INFO TaskSetManager: Finished task 82.0 in stage 8.0 (TID 90) in 6516 ms on 10.0.0.133 (executor driver) (83/208) -26/04/01 06:27:56 INFO Executor: Running task 86.0 in stage 8.0 (TID 94) -26/04/01 06:27:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:57 INFO Executor: Finished task 83.0 in stage 8.0 (TID 91). 6787 bytes result sent to driver -26/04/01 06:27:57 INFO TaskSetManager: Starting task 87.0 in stage 8.0 (TID 95) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:27:57 INFO TaskSetManager: Finished task 83.0 in stage 8.0 (TID 91) in 6540 ms on 10.0.0.133 (executor driver) (84/208) -26/04/01 06:27:57 INFO Executor: Running task 87.0 in stage 8.0 (TID 95) -26/04/01 06:27:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:27:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:03 INFO Executor: Finished task 84.0 in stage 8.0 (TID 92). 6787 bytes result sent to driver -26/04/01 06:28:03 INFO TaskSetManager: Starting task 88.0 in stage 8.0 (TID 96) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:03 INFO TaskSetManager: Finished task 84.0 in stage 8.0 (TID 92) in 6529 ms on 10.0.0.133 (executor driver) (85/208) -26/04/01 06:28:03 INFO Executor: Running task 88.0 in stage 8.0 (TID 96) -26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:03 INFO Executor: Finished task 86.0 in stage 8.0 (TID 94). 6787 bytes result sent to driver -26/04/01 06:28:03 INFO Executor: Finished task 85.0 in stage 8.0 (TID 93). 6787 bytes result sent to driver -26/04/01 06:28:03 INFO TaskSetManager: Starting task 89.0 in stage 8.0 (TID 97) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:03 INFO Executor: Running task 89.0 in stage 8.0 (TID 97) -26/04/01 06:28:03 INFO TaskSetManager: Starting task 90.0 in stage 8.0 (TID 98) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:28:03 INFO Executor: Running task 90.0 in stage 8.0 (TID 98) -26/04/01 06:28:03 INFO TaskSetManager: Finished task 86.0 in stage 8.0 (TID 94) in 6531 ms on 10.0.0.133 (executor driver) (86/208) -26/04/01 06:28:03 INFO TaskSetManager: Finished task 85.0 in stage 8.0 (TID 93) in 6533 ms on 10.0.0.133 (executor driver) (87/208) -26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:03 INFO Executor: Finished task 87.0 in stage 8.0 (TID 95). 6787 bytes result sent to driver -26/04/01 06:28:03 INFO TaskSetManager: Starting task 91.0 in stage 8.0 (TID 99) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:28:03 INFO TaskSetManager: Finished task 87.0 in stage 8.0 (TID 95) in 6525 ms on 10.0.0.133 (executor driver) (88/208) -26/04/01 06:28:03 INFO Executor: Running task 91.0 in stage 8.0 (TID 99) -26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:09 INFO Executor: Finished task 88.0 in stage 8.0 (TID 96). 6787 bytes result sent to driver -26/04/01 06:28:09 INFO TaskSetManager: Starting task 92.0 in stage 8.0 (TID 100) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:09 INFO Executor: Running task 92.0 in stage 8.0 (TID 100) -26/04/01 06:28:09 INFO TaskSetManager: Finished task 88.0 in stage 8.0 (TID 96) in 6537 ms on 10.0.0.133 (executor driver) (89/208) -26/04/01 06:28:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:09 INFO Executor: Finished task 89.0 in stage 8.0 (TID 97). 6787 bytes result sent to driver -26/04/01 06:28:09 INFO TaskSetManager: Starting task 93.0 in stage 8.0 (TID 101) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:09 INFO TaskSetManager: Finished task 89.0 in stage 8.0 (TID 97) in 6532 ms on 10.0.0.133 (executor driver) (90/208) -26/04/01 06:28:09 INFO Executor: Running task 93.0 in stage 8.0 (TID 101) -26/04/01 06:28:09 INFO Executor: Finished task 91.0 in stage 8.0 (TID 99). 6787 bytes result sent to driver -26/04/01 06:28:09 INFO TaskSetManager: Starting task 94.0 in stage 8.0 (TID 102) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:09 INFO TaskSetManager: Finished task 91.0 in stage 8.0 (TID 99) in 6283 ms on 10.0.0.133 (executor driver) (91/208) -26/04/01 06:28:09 INFO Executor: Running task 94.0 in stage 8.0 (TID 102) -26/04/01 06:28:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:10 INFO Executor: Finished task 90.0 in stage 8.0 (TID 98). 6787 bytes result sent to driver -26/04/01 06:28:10 INFO TaskSetManager: Starting task 95.0 in stage 8.0 (TID 103) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:10 INFO Executor: Running task 95.0 in stage 8.0 (TID 103) -26/04/01 06:28:10 INFO TaskSetManager: Finished task 90.0 in stage 8.0 (TID 98) in 6782 ms on 10.0.0.133 (executor driver) (92/208) -26/04/01 06:28:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:16 INFO Executor: Finished task 92.0 in stage 8.0 (TID 100). 6787 bytes result sent to driver -26/04/01 06:28:16 INFO TaskSetManager: Starting task 96.0 in stage 8.0 (TID 104) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:28:16 INFO Executor: Running task 96.0 in stage 8.0 (TID 104) -26/04/01 06:28:16 INFO TaskSetManager: Finished task 92.0 in stage 8.0 (TID 100) in 6540 ms on 10.0.0.133 (executor driver) (93/208) -26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:16 INFO Executor: Finished task 93.0 in stage 8.0 (TID 101). 6787 bytes result sent to driver -26/04/01 06:28:16 INFO TaskSetManager: Starting task 97.0 in stage 8.0 (TID 105) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:28:16 INFO TaskSetManager: Finished task 93.0 in stage 8.0 (TID 101) in 6542 ms on 10.0.0.133 (executor driver) (94/208) -26/04/01 06:28:16 INFO Executor: Running task 97.0 in stage 8.0 (TID 105) -26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:16 INFO Executor: Finished task 94.0 in stage 8.0 (TID 102). 6787 bytes result sent to driver -26/04/01 06:28:16 INFO TaskSetManager: Starting task 98.0 in stage 8.0 (TID 106) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:28:16 INFO TaskSetManager: Finished task 94.0 in stage 8.0 (TID 102) in 6544 ms on 10.0.0.133 (executor driver) (95/208) -26/04/01 06:28:16 INFO Executor: Running task 98.0 in stage 8.0 (TID 106) -26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:16 INFO Executor: Finished task 95.0 in stage 8.0 (TID 103). 6787 bytes result sent to driver -26/04/01 06:28:16 INFO TaskSetManager: Starting task 99.0 in stage 8.0 (TID 107) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:28:16 INFO TaskSetManager: Finished task 95.0 in stage 8.0 (TID 103) in 6542 ms on 10.0.0.133 (executor driver) (96/208) -26/04/01 06:28:16 INFO Executor: Running task 99.0 in stage 8.0 (TID 107) -26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:22 INFO Executor: Finished task 96.0 in stage 8.0 (TID 104). 6787 bytes result sent to driver -26/04/01 06:28:22 INFO TaskSetManager: Starting task 100.0 in stage 8.0 (TID 108) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:28:22 INFO TaskSetManager: Finished task 96.0 in stage 8.0 (TID 104) in 6542 ms on 10.0.0.133 (executor driver) (97/208) -26/04/01 06:28:22 INFO Executor: Running task 100.0 in stage 8.0 (TID 108) -26/04/01 06:28:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:23 INFO Executor: Finished task 98.0 in stage 8.0 (TID 106). 6787 bytes result sent to driver -26/04/01 06:28:23 INFO Executor: Finished task 97.0 in stage 8.0 (TID 105). 6787 bytes result sent to driver -26/04/01 06:28:23 INFO TaskSetManager: Starting task 101.0 in stage 8.0 (TID 109) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:28:23 INFO Executor: Running task 101.0 in stage 8.0 (TID 109) -26/04/01 06:28:23 INFO TaskSetManager: Starting task 102.0 in stage 8.0 (TID 110) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:28:23 INFO TaskSetManager: Finished task 98.0 in stage 8.0 (TID 106) in 6531 ms on 10.0.0.133 (executor driver) (98/208) -26/04/01 06:28:23 INFO Executor: Running task 102.0 in stage 8.0 (TID 110) -26/04/01 06:28:23 INFO TaskSetManager: Finished task 97.0 in stage 8.0 (TID 105) in 6534 ms on 10.0.0.133 (executor driver) (99/208) -26/04/01 06:28:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:23 INFO Executor: Finished task 99.0 in stage 8.0 (TID 107). 6787 bytes result sent to driver -26/04/01 06:28:23 INFO TaskSetManager: Starting task 103.0 in stage 8.0 (TID 111) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:28:23 INFO TaskSetManager: Finished task 99.0 in stage 8.0 (TID 107) in 6534 ms on 10.0.0.133 (executor driver) (100/208) -26/04/01 06:28:23 INFO Executor: Running task 103.0 in stage 8.0 (TID 111) -26/04/01 06:28:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:29 INFO Executor: Finished task 100.0 in stage 8.0 (TID 108). 6787 bytes result sent to driver -26/04/01 06:28:29 INFO TaskSetManager: Starting task 104.0 in stage 8.0 (TID 112) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:29 INFO Executor: Running task 104.0 in stage 8.0 (TID 112) -26/04/01 06:28:29 INFO TaskSetManager: Finished task 100.0 in stage 8.0 (TID 108) in 6518 ms on 10.0.0.133 (executor driver) (101/208) -26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:29 INFO Executor: Finished task 101.0 in stage 8.0 (TID 109). 6787 bytes result sent to driver -26/04/01 06:28:29 INFO Executor: Finished task 102.0 in stage 8.0 (TID 110). 6787 bytes result sent to driver -26/04/01 06:28:29 INFO TaskSetManager: Starting task 105.0 in stage 8.0 (TID 113) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:29 INFO TaskSetManager: Starting task 106.0 in stage 8.0 (TID 114) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:29 INFO Executor: Running task 105.0 in stage 8.0 (TID 113) -26/04/01 06:28:29 INFO TaskSetManager: Finished task 101.0 in stage 8.0 (TID 109) in 6523 ms on 10.0.0.133 (executor driver) (102/208) -26/04/01 06:28:29 INFO Executor: Running task 106.0 in stage 8.0 (TID 114) -26/04/01 06:28:29 INFO TaskSetManager: Finished task 102.0 in stage 8.0 (TID 110) in 6523 ms on 10.0.0.133 (executor driver) (103/208) -26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:29 INFO Executor: Finished task 103.0 in stage 8.0 (TID 111). 6787 bytes result sent to driver -26/04/01 06:28:29 INFO TaskSetManager: Starting task 107.0 in stage 8.0 (TID 115) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:29 INFO TaskSetManager: Finished task 103.0 in stage 8.0 (TID 111) in 6525 ms on 10.0.0.133 (executor driver) (104/208) -26/04/01 06:28:29 INFO Executor: Running task 107.0 in stage 8.0 (TID 115) -26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:35 INFO Executor: Finished task 104.0 in stage 8.0 (TID 112). 6787 bytes result sent to driver -26/04/01 06:28:35 INFO TaskSetManager: Starting task 108.0 in stage 8.0 (TID 116) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:28:35 INFO Executor: Running task 108.0 in stage 8.0 (TID 116) -26/04/01 06:28:35 INFO TaskSetManager: Finished task 104.0 in stage 8.0 (TID 112) in 6501 ms on 10.0.0.133 (executor driver) (105/208) -26/04/01 06:28:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:36 INFO Executor: Finished task 105.0 in stage 8.0 (TID 113). 6787 bytes result sent to driver -26/04/01 06:28:36 INFO TaskSetManager: Starting task 109.0 in stage 8.0 (TID 117) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:28:36 INFO Executor: Running task 109.0 in stage 8.0 (TID 117) -26/04/01 06:28:36 INFO TaskSetManager: Finished task 105.0 in stage 8.0 (TID 113) in 6503 ms on 10.0.0.133 (executor driver) (106/208) -26/04/01 06:28:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:36 INFO Executor: Finished task 106.0 in stage 8.0 (TID 114). 6787 bytes result sent to driver -26/04/01 06:28:36 INFO TaskSetManager: Starting task 110.0 in stage 8.0 (TID 118) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:28:36 INFO TaskSetManager: Finished task 106.0 in stage 8.0 (TID 114) in 6507 ms on 10.0.0.133 (executor driver) (107/208) -26/04/01 06:28:36 INFO Executor: Running task 110.0 in stage 8.0 (TID 118) -26/04/01 06:28:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:36 INFO Executor: Finished task 107.0 in stage 8.0 (TID 115). 6787 bytes result sent to driver -26/04/01 06:28:36 INFO TaskSetManager: Starting task 111.0 in stage 8.0 (TID 119) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:28:36 INFO Executor: Running task 111.0 in stage 8.0 (TID 119) -26/04/01 06:28:36 INFO TaskSetManager: Finished task 107.0 in stage 8.0 (TID 115) in 6502 ms on 10.0.0.133 (executor driver) (108/208) -26/04/01 06:28:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:42 INFO Executor: Finished task 108.0 in stage 8.0 (TID 116). 6787 bytes result sent to driver -26/04/01 06:28:42 INFO TaskSetManager: Starting task 112.0 in stage 8.0 (TID 120) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:28:42 INFO Executor: Running task 112.0 in stage 8.0 (TID 120) -26/04/01 06:28:42 INFO TaskSetManager: Finished task 108.0 in stage 8.0 (TID 116) in 6527 ms on 10.0.0.133 (executor driver) (109/208) -26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:42 INFO Executor: Finished task 109.0 in stage 8.0 (TID 117). 6787 bytes result sent to driver -26/04/01 06:28:42 INFO TaskSetManager: Starting task 113.0 in stage 8.0 (TID 121) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:28:42 INFO Executor: Running task 113.0 in stage 8.0 (TID 121) -26/04/01 06:28:42 INFO TaskSetManager: Finished task 109.0 in stage 8.0 (TID 117) in 6524 ms on 10.0.0.133 (executor driver) (110/208) -26/04/01 06:28:42 INFO Executor: Finished task 110.0 in stage 8.0 (TID 118). 6787 bytes result sent to driver -26/04/01 06:28:42 INFO TaskSetManager: Starting task 114.0 in stage 8.0 (TID 122) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:28:42 INFO TaskSetManager: Finished task 110.0 in stage 8.0 (TID 118) in 6521 ms on 10.0.0.133 (executor driver) (111/208) -26/04/01 06:28:42 INFO Executor: Running task 114.0 in stage 8.0 (TID 122) -26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:42 INFO Executor: Finished task 111.0 in stage 8.0 (TID 119). 6787 bytes result sent to driver -26/04/01 06:28:42 INFO TaskSetManager: Starting task 115.0 in stage 8.0 (TID 123) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:28:42 INFO TaskSetManager: Finished task 111.0 in stage 8.0 (TID 119) in 6527 ms on 10.0.0.133 (executor driver) (112/208) -26/04/01 06:28:42 INFO Executor: Running task 115.0 in stage 8.0 (TID 123) -26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:48 INFO Executor: Finished task 112.0 in stage 8.0 (TID 120). 6787 bytes result sent to driver -26/04/01 06:28:48 INFO TaskSetManager: Starting task 116.0 in stage 8.0 (TID 124) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:48 INFO Executor: Running task 116.0 in stage 8.0 (TID 124) -26/04/01 06:28:48 INFO TaskSetManager: Finished task 112.0 in stage 8.0 (TID 120) in 6522 ms on 10.0.0.133 (executor driver) (113/208) -26/04/01 06:28:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:49 INFO Executor: Finished task 114.0 in stage 8.0 (TID 122). 6787 bytes result sent to driver -26/04/01 06:28:49 INFO Executor: Finished task 113.0 in stage 8.0 (TID 121). 6787 bytes result sent to driver -26/04/01 06:28:49 INFO TaskSetManager: Starting task 117.0 in stage 8.0 (TID 125) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:49 INFO TaskSetManager: Finished task 114.0 in stage 8.0 (TID 122) in 6522 ms on 10.0.0.133 (executor driver) (114/208) -26/04/01 06:28:49 INFO Executor: Running task 117.0 in stage 8.0 (TID 125) -26/04/01 06:28:49 INFO TaskSetManager: Starting task 118.0 in stage 8.0 (TID 126) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:49 INFO TaskSetManager: Finished task 113.0 in stage 8.0 (TID 121) in 6523 ms on 10.0.0.133 (executor driver) (115/208) -26/04/01 06:28:49 INFO Executor: Running task 118.0 in stage 8.0 (TID 126) -26/04/01 06:28:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:49 INFO Executor: Finished task 115.0 in stage 8.0 (TID 123). 6787 bytes result sent to driver -26/04/01 06:28:49 INFO TaskSetManager: Starting task 119.0 in stage 8.0 (TID 127) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:49 INFO TaskSetManager: Finished task 115.0 in stage 8.0 (TID 123) in 6517 ms on 10.0.0.133 (executor driver) (116/208) -26/04/01 06:28:49 INFO Executor: Running task 119.0 in stage 8.0 (TID 127) -26/04/01 06:28:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:55 INFO Executor: Finished task 116.0 in stage 8.0 (TID 124). 6787 bytes result sent to driver -26/04/01 06:28:55 INFO TaskSetManager: Starting task 120.0 in stage 8.0 (TID 128) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:28:55 INFO TaskSetManager: Finished task 116.0 in stage 8.0 (TID 124) in 6540 ms on 10.0.0.133 (executor driver) (117/208) -26/04/01 06:28:55 INFO Executor: Running task 120.0 in stage 8.0 (TID 128) -26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:55 INFO Executor: Finished task 117.0 in stage 8.0 (TID 125). 6830 bytes result sent to driver -26/04/01 06:28:55 INFO TaskSetManager: Starting task 121.0 in stage 8.0 (TID 129) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:28:55 INFO Executor: Running task 121.0 in stage 8.0 (TID 129) -26/04/01 06:28:55 INFO TaskSetManager: Finished task 117.0 in stage 8.0 (TID 125) in 6535 ms on 10.0.0.133 (executor driver) (118/208) -26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:55 INFO Executor: Finished task 118.0 in stage 8.0 (TID 126). 6830 bytes result sent to driver -26/04/01 06:28:55 INFO TaskSetManager: Starting task 122.0 in stage 8.0 (TID 130) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:55 INFO Executor: Running task 122.0 in stage 8.0 (TID 130) -26/04/01 06:28:55 INFO TaskSetManager: Finished task 118.0 in stage 8.0 (TID 126) in 6537 ms on 10.0.0.133 (executor driver) (119/208) -26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:55 INFO Executor: Finished task 119.0 in stage 8.0 (TID 127). 6830 bytes result sent to driver -26/04/01 06:28:55 INFO TaskSetManager: Starting task 123.0 in stage 8.0 (TID 131) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:28:55 INFO Executor: Running task 123.0 in stage 8.0 (TID 131) -26/04/01 06:28:55 INFO TaskSetManager: Finished task 119.0 in stage 8.0 (TID 127) in 6546 ms on 10.0.0.133 (executor driver) (120/208) -26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:28:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:01 INFO Executor: Finished task 120.0 in stage 8.0 (TID 128). 6787 bytes result sent to driver -26/04/01 06:29:01 INFO TaskSetManager: Starting task 124.0 in stage 8.0 (TID 132) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:01 INFO TaskSetManager: Finished task 120.0 in stage 8.0 (TID 128) in 6516 ms on 10.0.0.133 (executor driver) (121/208) -26/04/01 06:29:01 INFO Executor: Running task 124.0 in stage 8.0 (TID 132) -26/04/01 06:29:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:02 INFO Executor: Finished task 121.0 in stage 8.0 (TID 129). 6787 bytes result sent to driver -26/04/01 06:29:02 INFO TaskSetManager: Starting task 125.0 in stage 8.0 (TID 133) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:02 INFO TaskSetManager: Finished task 121.0 in stage 8.0 (TID 129) in 6515 ms on 10.0.0.133 (executor driver) (122/208) -26/04/01 06:29:02 INFO Executor: Running task 125.0 in stage 8.0 (TID 133) -26/04/01 06:29:02 INFO Executor: Finished task 122.0 in stage 8.0 (TID 130). 6787 bytes result sent to driver -26/04/01 06:29:02 INFO TaskSetManager: Starting task 126.0 in stage 8.0 (TID 134) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:29:02 INFO TaskSetManager: Finished task 122.0 in stage 8.0 (TID 130) in 6514 ms on 10.0.0.133 (executor driver) (123/208) -26/04/01 06:29:02 INFO Executor: Running task 126.0 in stage 8.0 (TID 134) -26/04/01 06:29:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:02 INFO Executor: Finished task 123.0 in stage 8.0 (TID 131). 6787 bytes result sent to driver -26/04/01 06:29:02 INFO TaskSetManager: Starting task 127.0 in stage 8.0 (TID 135) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:29:02 INFO TaskSetManager: Finished task 123.0 in stage 8.0 (TID 131) in 6508 ms on 10.0.0.133 (executor driver) (124/208) -26/04/01 06:29:02 INFO Executor: Running task 127.0 in stage 8.0 (TID 135) -26/04/01 06:29:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:08 INFO Executor: Finished task 124.0 in stage 8.0 (TID 132). 6787 bytes result sent to driver -26/04/01 06:29:08 INFO TaskSetManager: Starting task 128.0 in stage 8.0 (TID 136) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:08 INFO TaskSetManager: Finished task 124.0 in stage 8.0 (TID 132) in 6520 ms on 10.0.0.133 (executor driver) (125/208) -26/04/01 06:29:08 INFO Executor: Running task 128.0 in stage 8.0 (TID 136) -26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:08 INFO Executor: Finished task 126.0 in stage 8.0 (TID 134). 6787 bytes result sent to driver -26/04/01 06:29:08 INFO TaskSetManager: Starting task 129.0 in stage 8.0 (TID 137) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:08 INFO TaskSetManager: Finished task 126.0 in stage 8.0 (TID 134) in 6523 ms on 10.0.0.133 (executor driver) (126/208) -26/04/01 06:29:08 INFO Executor: Running task 129.0 in stage 8.0 (TID 137) -26/04/01 06:29:08 INFO Executor: Finished task 125.0 in stage 8.0 (TID 133). 6787 bytes result sent to driver -26/04/01 06:29:08 INFO TaskSetManager: Starting task 130.0 in stage 8.0 (TID 138) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:08 INFO TaskSetManager: Finished task 125.0 in stage 8.0 (TID 133) in 6526 ms on 10.0.0.133 (executor driver) (127/208) -26/04/01 06:29:08 INFO Executor: Running task 130.0 in stage 8.0 (TID 138) -26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:08 INFO Executor: Finished task 127.0 in stage 8.0 (TID 135). 6787 bytes result sent to driver -26/04/01 06:29:08 INFO TaskSetManager: Starting task 131.0 in stage 8.0 (TID 139) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:08 INFO TaskSetManager: Finished task 127.0 in stage 8.0 (TID 135) in 6529 ms on 10.0.0.133 (executor driver) (128/208) -26/04/01 06:29:08 INFO Executor: Running task 131.0 in stage 8.0 (TID 139) -26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:14 INFO Executor: Finished task 128.0 in stage 8.0 (TID 136). 6787 bytes result sent to driver -26/04/01 06:29:14 INFO TaskSetManager: Starting task 132.0 in stage 8.0 (TID 140) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:29:14 INFO TaskSetManager: Finished task 128.0 in stage 8.0 (TID 136) in 6530 ms on 10.0.0.133 (executor driver) (129/208) -26/04/01 06:29:14 INFO Executor: Running task 132.0 in stage 8.0 (TID 140) -26/04/01 06:29:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:15 INFO Executor: Finished task 129.0 in stage 8.0 (TID 137). 6787 bytes result sent to driver -26/04/01 06:29:15 INFO Executor: Finished task 130.0 in stage 8.0 (TID 138). 6787 bytes result sent to driver -26/04/01 06:29:15 INFO TaskSetManager: Starting task 133.0 in stage 8.0 (TID 141) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:29:15 INFO Executor: Running task 133.0 in stage 8.0 (TID 141) -26/04/01 06:29:15 INFO TaskSetManager: Starting task 134.0 in stage 8.0 (TID 142) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:15 INFO TaskSetManager: Finished task 129.0 in stage 8.0 (TID 137) in 6528 ms on 10.0.0.133 (executor driver) (130/208) -26/04/01 06:29:15 INFO Executor: Running task 134.0 in stage 8.0 (TID 142) -26/04/01 06:29:15 INFO TaskSetManager: Finished task 130.0 in stage 8.0 (TID 138) in 6528 ms on 10.0.0.133 (executor driver) (131/208) -26/04/01 06:29:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:15 INFO Executor: Finished task 131.0 in stage 8.0 (TID 139). 6787 bytes result sent to driver -26/04/01 06:29:15 INFO TaskSetManager: Starting task 135.0 in stage 8.0 (TID 143) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:15 INFO TaskSetManager: Finished task 131.0 in stage 8.0 (TID 139) in 6524 ms on 10.0.0.133 (executor driver) (132/208) -26/04/01 06:29:15 INFO Executor: Running task 135.0 in stage 8.0 (TID 143) -26/04/01 06:29:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:21 INFO Executor: Finished task 132.0 in stage 8.0 (TID 140). 6787 bytes result sent to driver -26/04/01 06:29:21 INFO TaskSetManager: Starting task 136.0 in stage 8.0 (TID 144) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:21 INFO TaskSetManager: Finished task 132.0 in stage 8.0 (TID 140) in 6477 ms on 10.0.0.133 (executor driver) (133/208) -26/04/01 06:29:21 INFO Executor: Running task 136.0 in stage 8.0 (TID 144) -26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:21 INFO Executor: Finished task 133.0 in stage 8.0 (TID 141). 6787 bytes result sent to driver -26/04/01 06:29:21 INFO TaskSetManager: Starting task 137.0 in stage 8.0 (TID 145) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:21 INFO TaskSetManager: Finished task 133.0 in stage 8.0 (TID 141) in 6478 ms on 10.0.0.133 (executor driver) (134/208) -26/04/01 06:29:21 INFO Executor: Running task 137.0 in stage 8.0 (TID 145) -26/04/01 06:29:21 INFO Executor: Finished task 134.0 in stage 8.0 (TID 142). 6787 bytes result sent to driver -26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:21 INFO TaskSetManager: Starting task 138.0 in stage 8.0 (TID 146) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:29:21 INFO TaskSetManager: Finished task 134.0 in stage 8.0 (TID 142) in 6479 ms on 10.0.0.133 (executor driver) (135/208) -26/04/01 06:29:21 INFO Executor: Running task 138.0 in stage 8.0 (TID 146) -26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:21 INFO Executor: Finished task 135.0 in stage 8.0 (TID 143). 6787 bytes result sent to driver -26/04/01 06:29:21 INFO TaskSetManager: Starting task 139.0 in stage 8.0 (TID 147) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:29:21 INFO Executor: Running task 139.0 in stage 8.0 (TID 147) -26/04/01 06:29:21 INFO TaskSetManager: Finished task 135.0 in stage 8.0 (TID 143) in 6480 ms on 10.0.0.133 (executor driver) (136/208) -26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:27 INFO Executor: Finished task 136.0 in stage 8.0 (TID 144). 6787 bytes result sent to driver -26/04/01 06:29:27 INFO TaskSetManager: Starting task 140.0 in stage 8.0 (TID 148) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:27 INFO TaskSetManager: Finished task 136.0 in stage 8.0 (TID 144) in 6474 ms on 10.0.0.133 (executor driver) (137/208) -26/04/01 06:29:27 INFO Executor: Running task 140.0 in stage 8.0 (TID 148) -26/04/01 06:29:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:28 INFO Executor: Finished task 137.0 in stage 8.0 (TID 145). 6787 bytes result sent to driver -26/04/01 06:29:28 INFO TaskSetManager: Starting task 141.0 in stage 8.0 (TID 149) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:28 INFO TaskSetManager: Finished task 137.0 in stage 8.0 (TID 145) in 6473 ms on 10.0.0.133 (executor driver) (138/208) -26/04/01 06:29:28 INFO Executor: Running task 141.0 in stage 8.0 (TID 149) -26/04/01 06:29:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:28 INFO Executor: Finished task 138.0 in stage 8.0 (TID 146). 6787 bytes result sent to driver -26/04/01 06:29:28 INFO TaskSetManager: Starting task 142.0 in stage 8.0 (TID 150) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:28 INFO TaskSetManager: Finished task 138.0 in stage 8.0 (TID 146) in 6477 ms on 10.0.0.133 (executor driver) (139/208) -26/04/01 06:29:28 INFO Executor: Running task 142.0 in stage 8.0 (TID 150) -26/04/01 06:29:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:28 INFO Executor: Finished task 139.0 in stage 8.0 (TID 147). 6787 bytes result sent to driver -26/04/01 06:29:28 INFO TaskSetManager: Starting task 143.0 in stage 8.0 (TID 151) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:28 INFO TaskSetManager: Finished task 139.0 in stage 8.0 (TID 147) in 6463 ms on 10.0.0.133 (executor driver) (140/208) -26/04/01 06:29:28 INFO Executor: Running task 143.0 in stage 8.0 (TID 151) -26/04/01 06:29:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:34 INFO Executor: Finished task 140.0 in stage 8.0 (TID 148). 6787 bytes result sent to driver -26/04/01 06:29:34 INFO TaskSetManager: Starting task 144.0 in stage 8.0 (TID 152) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:29:34 INFO TaskSetManager: Finished task 140.0 in stage 8.0 (TID 148) in 6443 ms on 10.0.0.133 (executor driver) (141/208) -26/04/01 06:29:34 INFO Executor: Running task 144.0 in stage 8.0 (TID 152) -26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:34 INFO Executor: Finished task 141.0 in stage 8.0 (TID 149). 6787 bytes result sent to driver -26/04/01 06:29:34 INFO TaskSetManager: Starting task 145.0 in stage 8.0 (TID 153) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:29:34 INFO Executor: Running task 145.0 in stage 8.0 (TID 153) -26/04/01 06:29:34 INFO TaskSetManager: Finished task 141.0 in stage 8.0 (TID 149) in 6433 ms on 10.0.0.133 (executor driver) (142/208) -26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:34 INFO Executor: Finished task 142.0 in stage 8.0 (TID 150). 6787 bytes result sent to driver -26/04/01 06:29:34 INFO TaskSetManager: Starting task 146.0 in stage 8.0 (TID 154) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:29:34 INFO Executor: Running task 146.0 in stage 8.0 (TID 154) -26/04/01 06:29:34 INFO TaskSetManager: Finished task 142.0 in stage 8.0 (TID 150) in 6439 ms on 10.0.0.133 (executor driver) (143/208) -26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:34 INFO Executor: Finished task 143.0 in stage 8.0 (TID 151). 6787 bytes result sent to driver -26/04/01 06:29:34 INFO TaskSetManager: Starting task 147.0 in stage 8.0 (TID 155) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:29:34 INFO TaskSetManager: Finished task 143.0 in stage 8.0 (TID 151) in 6428 ms on 10.0.0.133 (executor driver) (144/208) -26/04/01 06:29:34 INFO Executor: Running task 147.0 in stage 8.0 (TID 155) -26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:40 INFO Executor: Finished task 144.0 in stage 8.0 (TID 152). 6787 bytes result sent to driver -26/04/01 06:29:40 INFO TaskSetManager: Starting task 148.0 in stage 8.0 (TID 156) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:29:40 INFO TaskSetManager: Finished task 144.0 in stage 8.0 (TID 152) in 6500 ms on 10.0.0.133 (executor driver) (145/208) -26/04/01 06:29:40 INFO Executor: Running task 148.0 in stage 8.0 (TID 156) -26/04/01 06:29:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:41 INFO Executor: Finished task 146.0 in stage 8.0 (TID 154). 6787 bytes result sent to driver -26/04/01 06:29:41 INFO TaskSetManager: Starting task 149.0 in stage 8.0 (TID 157) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:29:41 INFO Executor: Finished task 145.0 in stage 8.0 (TID 153). 6787 bytes result sent to driver -26/04/01 06:29:41 INFO TaskSetManager: Finished task 146.0 in stage 8.0 (TID 154) in 6502 ms on 10.0.0.133 (executor driver) (146/208) -26/04/01 06:29:41 INFO Executor: Running task 149.0 in stage 8.0 (TID 157) -26/04/01 06:29:41 INFO TaskSetManager: Starting task 150.0 in stage 8.0 (TID 158) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:29:41 INFO TaskSetManager: Finished task 145.0 in stage 8.0 (TID 153) in 6513 ms on 10.0.0.133 (executor driver) (147/208) -26/04/01 06:29:41 INFO Executor: Running task 150.0 in stage 8.0 (TID 158) -26/04/01 06:29:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:41 INFO Executor: Finished task 147.0 in stage 8.0 (TID 155). 6787 bytes result sent to driver -26/04/01 06:29:41 INFO TaskSetManager: Starting task 151.0 in stage 8.0 (TID 159) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:29:41 INFO TaskSetManager: Finished task 147.0 in stage 8.0 (TID 155) in 6523 ms on 10.0.0.133 (executor driver) (148/208) -26/04/01 06:29:41 INFO Executor: Running task 151.0 in stage 8.0 (TID 159) -26/04/01 06:29:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:47 INFO Executor: Finished task 148.0 in stage 8.0 (TID 156). 6787 bytes result sent to driver -26/04/01 06:29:47 INFO TaskSetManager: Starting task 152.0 in stage 8.0 (TID 160) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:47 INFO TaskSetManager: Finished task 148.0 in stage 8.0 (TID 156) in 6532 ms on 10.0.0.133 (executor driver) (149/208) -26/04/01 06:29:47 INFO Executor: Running task 152.0 in stage 8.0 (TID 160) -26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:47 INFO Executor: Finished task 150.0 in stage 8.0 (TID 158). 6787 bytes result sent to driver -26/04/01 06:29:47 INFO Executor: Finished task 149.0 in stage 8.0 (TID 157). 6787 bytes result sent to driver -26/04/01 06:29:47 INFO TaskSetManager: Starting task 153.0 in stage 8.0 (TID 161) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:47 INFO TaskSetManager: Starting task 154.0 in stage 8.0 (TID 162) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:47 INFO Executor: Running task 153.0 in stage 8.0 (TID 161) -26/04/01 06:29:47 INFO TaskSetManager: Finished task 150.0 in stage 8.0 (TID 158) in 6525 ms on 10.0.0.133 (executor driver) (150/208) -26/04/01 06:29:47 INFO Executor: Running task 154.0 in stage 8.0 (TID 162) -26/04/01 06:29:47 INFO TaskSetManager: Finished task 149.0 in stage 8.0 (TID 157) in 6526 ms on 10.0.0.133 (executor driver) (151/208) -26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:47 INFO Executor: Finished task 151.0 in stage 8.0 (TID 159). 6787 bytes result sent to driver -26/04/01 06:29:47 INFO TaskSetManager: Starting task 155.0 in stage 8.0 (TID 163) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:47 INFO TaskSetManager: Finished task 151.0 in stage 8.0 (TID 159) in 6528 ms on 10.0.0.133 (executor driver) (152/208) -26/04/01 06:29:47 INFO Executor: Running task 155.0 in stage 8.0 (TID 163) -26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:53 INFO Executor: Finished task 152.0 in stage 8.0 (TID 160). 6787 bytes result sent to driver -26/04/01 06:29:53 INFO TaskSetManager: Starting task 156.0 in stage 8.0 (TID 164) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:29:53 INFO TaskSetManager: Finished task 152.0 in stage 8.0 (TID 160) in 6544 ms on 10.0.0.133 (executor driver) (153/208) -26/04/01 06:29:53 INFO Executor: Running task 156.0 in stage 8.0 (TID 164) -26/04/01 06:29:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:54 INFO Executor: Finished task 153.0 in stage 8.0 (TID 161). 6787 bytes result sent to driver -26/04/01 06:29:54 INFO TaskSetManager: Starting task 157.0 in stage 8.0 (TID 165) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:29:54 INFO TaskSetManager: Finished task 153.0 in stage 8.0 (TID 161) in 6550 ms on 10.0.0.133 (executor driver) (154/208) -26/04/01 06:29:54 INFO Executor: Running task 157.0 in stage 8.0 (TID 165) -26/04/01 06:29:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:54 INFO Executor: Finished task 154.0 in stage 8.0 (TID 162). 6787 bytes result sent to driver -26/04/01 06:29:54 INFO TaskSetManager: Starting task 158.0 in stage 8.0 (TID 166) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:54 INFO TaskSetManager: Finished task 154.0 in stage 8.0 (TID 162) in 6552 ms on 10.0.0.133 (executor driver) (155/208) -26/04/01 06:29:54 INFO Executor: Running task 158.0 in stage 8.0 (TID 166) -26/04/01 06:29:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:54 INFO Executor: Finished task 155.0 in stage 8.0 (TID 163). 6787 bytes result sent to driver -26/04/01 06:29:54 INFO TaskSetManager: Starting task 159.0 in stage 8.0 (TID 167) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:29:54 INFO TaskSetManager: Finished task 155.0 in stage 8.0 (TID 163) in 6551 ms on 10.0.0.133 (executor driver) (156/208) -26/04/01 06:29:54 INFO Executor: Running task 159.0 in stage 8.0 (TID 167) -26/04/01 06:29:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:29:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:00 INFO Executor: Finished task 156.0 in stage 8.0 (TID 164). 6787 bytes result sent to driver -26/04/01 06:30:00 INFO TaskSetManager: Starting task 160.0 in stage 8.0 (TID 168) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:30:00 INFO Executor: Running task 160.0 in stage 8.0 (TID 168) -26/04/01 06:30:00 INFO TaskSetManager: Finished task 156.0 in stage 8.0 (TID 164) in 6472 ms on 10.0.0.133 (executor driver) (157/208) -26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:00 INFO Executor: Finished task 157.0 in stage 8.0 (TID 165). 6787 bytes result sent to driver -26/04/01 06:30:00 INFO TaskSetManager: Starting task 161.0 in stage 8.0 (TID 169) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:30:00 INFO Executor: Running task 161.0 in stage 8.0 (TID 169) -26/04/01 06:30:00 INFO TaskSetManager: Finished task 157.0 in stage 8.0 (TID 165) in 6469 ms on 10.0.0.133 (executor driver) (158/208) -26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:00 INFO Executor: Finished task 158.0 in stage 8.0 (TID 166). 6787 bytes result sent to driver -26/04/01 06:30:00 INFO TaskSetManager: Starting task 162.0 in stage 8.0 (TID 170) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:30:00 INFO TaskSetManager: Finished task 158.0 in stage 8.0 (TID 166) in 6471 ms on 10.0.0.133 (executor driver) (159/208) -26/04/01 06:30:00 INFO Executor: Running task 162.0 in stage 8.0 (TID 170) -26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:00 INFO Executor: Finished task 159.0 in stage 8.0 (TID 167). 6787 bytes result sent to driver -26/04/01 06:30:00 INFO TaskSetManager: Starting task 163.0 in stage 8.0 (TID 171) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:30:00 INFO TaskSetManager: Finished task 159.0 in stage 8.0 (TID 167) in 6469 ms on 10.0.0.133 (executor driver) (160/208) -26/04/01 06:30:00 INFO Executor: Running task 163.0 in stage 8.0 (TID 171) -26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:06 INFO Executor: Finished task 160.0 in stage 8.0 (TID 168). 6787 bytes result sent to driver -26/04/01 06:30:06 INFO TaskSetManager: Starting task 164.0 in stage 8.0 (TID 172) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:30:06 INFO Executor: Running task 164.0 in stage 8.0 (TID 172) -26/04/01 06:30:06 INFO TaskSetManager: Finished task 160.0 in stage 8.0 (TID 168) in 6477 ms on 10.0.0.133 (executor driver) (161/208) -26/04/01 06:30:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:07 INFO Executor: Finished task 161.0 in stage 8.0 (TID 169). 6787 bytes result sent to driver -26/04/01 06:30:07 INFO TaskSetManager: Starting task 165.0 in stage 8.0 (TID 173) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:30:07 INFO TaskSetManager: Finished task 161.0 in stage 8.0 (TID 169) in 6475 ms on 10.0.0.133 (executor driver) (162/208) -26/04/01 06:30:07 INFO Executor: Running task 165.0 in stage 8.0 (TID 173) -26/04/01 06:30:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:07 INFO Executor: Finished task 162.0 in stage 8.0 (TID 170). 6787 bytes result sent to driver -26/04/01 06:30:07 INFO TaskSetManager: Starting task 166.0 in stage 8.0 (TID 174) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:30:07 INFO Executor: Running task 166.0 in stage 8.0 (TID 174) -26/04/01 06:30:07 INFO TaskSetManager: Finished task 162.0 in stage 8.0 (TID 170) in 6494 ms on 10.0.0.133 (executor driver) (163/208) -26/04/01 06:30:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:07 INFO Executor: Finished task 163.0 in stage 8.0 (TID 171). 6787 bytes result sent to driver -26/04/01 06:30:07 INFO TaskSetManager: Starting task 167.0 in stage 8.0 (TID 175) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:30:07 INFO TaskSetManager: Finished task 163.0 in stage 8.0 (TID 171) in 6497 ms on 10.0.0.133 (executor driver) (164/208) -26/04/01 06:30:07 INFO Executor: Running task 167.0 in stage 8.0 (TID 175) -26/04/01 06:30:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:13 INFO Executor: Finished task 164.0 in stage 8.0 (TID 172). 6873 bytes result sent to driver -26/04/01 06:30:13 INFO TaskSetManager: Starting task 168.0 in stage 8.0 (TID 176) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:30:13 INFO TaskSetManager: Finished task 164.0 in stage 8.0 (TID 172) in 6502 ms on 10.0.0.133 (executor driver) (165/208) -26/04/01 06:30:13 INFO Executor: Running task 168.0 in stage 8.0 (TID 176) -26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:13 INFO Executor: Finished task 165.0 in stage 8.0 (TID 173). 6830 bytes result sent to driver -26/04/01 06:30:13 INFO TaskSetManager: Starting task 169.0 in stage 8.0 (TID 177) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:30:13 INFO TaskSetManager: Finished task 165.0 in stage 8.0 (TID 173) in 6500 ms on 10.0.0.133 (executor driver) (166/208) -26/04/01 06:30:13 INFO Executor: Running task 169.0 in stage 8.0 (TID 177) -26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:13 INFO Executor: Finished task 166.0 in stage 8.0 (TID 174). 6830 bytes result sent to driver -26/04/01 06:30:13 INFO TaskSetManager: Starting task 170.0 in stage 8.0 (TID 178) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:30:13 INFO TaskSetManager: Finished task 166.0 in stage 8.0 (TID 174) in 6478 ms on 10.0.0.133 (executor driver) (167/208) -26/04/01 06:30:13 INFO Executor: Running task 170.0 in stage 8.0 (TID 178) -26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:13 INFO Executor: Finished task 167.0 in stage 8.0 (TID 175). 6830 bytes result sent to driver -26/04/01 06:30:13 INFO TaskSetManager: Starting task 171.0 in stage 8.0 (TID 179) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:30:13 INFO Executor: Running task 171.0 in stage 8.0 (TID 179) -26/04/01 06:30:13 INFO TaskSetManager: Finished task 167.0 in stage 8.0 (TID 175) in 6481 ms on 10.0.0.133 (executor driver) (168/208) -26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:19 INFO Executor: Finished task 168.0 in stage 8.0 (TID 176). 6787 bytes result sent to driver -26/04/01 06:30:19 INFO TaskSetManager: Starting task 172.0 in stage 8.0 (TID 180) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:30:19 INFO TaskSetManager: Finished task 168.0 in stage 8.0 (TID 176) in 6486 ms on 10.0.0.133 (executor driver) (169/208) -26/04/01 06:30:19 INFO Executor: Running task 172.0 in stage 8.0 (TID 180) -26/04/01 06:30:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:20 INFO Executor: Finished task 170.0 in stage 8.0 (TID 178). 6787 bytes result sent to driver -26/04/01 06:30:20 INFO TaskSetManager: Starting task 173.0 in stage 8.0 (TID 181) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:30:20 INFO TaskSetManager: Finished task 170.0 in stage 8.0 (TID 178) in 6486 ms on 10.0.0.133 (executor driver) (170/208) -26/04/01 06:30:20 INFO Executor: Running task 173.0 in stage 8.0 (TID 181) -26/04/01 06:30:20 INFO Executor: Finished task 169.0 in stage 8.0 (TID 177). 6787 bytes result sent to driver -26/04/01 06:30:20 INFO TaskSetManager: Starting task 174.0 in stage 8.0 (TID 182) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:30:20 INFO TaskSetManager: Finished task 169.0 in stage 8.0 (TID 177) in 6489 ms on 10.0.0.133 (executor driver) (171/208) -26/04/01 06:30:20 INFO Executor: Running task 174.0 in stage 8.0 (TID 182) -26/04/01 06:30:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:20 INFO Executor: Finished task 171.0 in stage 8.0 (TID 179). 6787 bytes result sent to driver -26/04/01 06:30:20 INFO TaskSetManager: Starting task 175.0 in stage 8.0 (TID 183) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:30:20 INFO TaskSetManager: Finished task 171.0 in stage 8.0 (TID 179) in 6483 ms on 10.0.0.133 (executor driver) (172/208) -26/04/01 06:30:20 INFO Executor: Running task 175.0 in stage 8.0 (TID 183) -26/04/01 06:30:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:26 INFO Executor: Finished task 172.0 in stage 8.0 (TID 180). 6787 bytes result sent to driver -26/04/01 06:30:26 INFO TaskSetManager: Starting task 176.0 in stage 8.0 (TID 184) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:30:26 INFO TaskSetManager: Finished task 172.0 in stage 8.0 (TID 180) in 6501 ms on 10.0.0.133 (executor driver) (173/208) -26/04/01 06:30:26 INFO Executor: Running task 176.0 in stage 8.0 (TID 184) -26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:26 INFO Executor: Finished task 173.0 in stage 8.0 (TID 181). 6787 bytes result sent to driver -26/04/01 06:30:26 INFO TaskSetManager: Starting task 177.0 in stage 8.0 (TID 185) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:30:26 INFO TaskSetManager: Finished task 173.0 in stage 8.0 (TID 181) in 6503 ms on 10.0.0.133 (executor driver) (174/208) -26/04/01 06:30:26 INFO Executor: Running task 177.0 in stage 8.0 (TID 185) -26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:26 INFO Executor: Finished task 174.0 in stage 8.0 (TID 182). 6787 bytes result sent to driver -26/04/01 06:30:26 INFO TaskSetManager: Starting task 178.0 in stage 8.0 (TID 186) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:30:26 INFO Executor: Running task 178.0 in stage 8.0 (TID 186) -26/04/01 06:30:26 INFO TaskSetManager: Finished task 174.0 in stage 8.0 (TID 182) in 6522 ms on 10.0.0.133 (executor driver) (175/208) -26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:26 INFO Executor: Finished task 175.0 in stage 8.0 (TID 183). 6787 bytes result sent to driver -26/04/01 06:30:26 INFO TaskSetManager: Starting task 179.0 in stage 8.0 (TID 187) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:30:26 INFO TaskSetManager: Finished task 175.0 in stage 8.0 (TID 183) in 6523 ms on 10.0.0.133 (executor driver) (176/208) -26/04/01 06:30:26 INFO Executor: Running task 179.0 in stage 8.0 (TID 187) -26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:32 INFO Executor: Finished task 176.0 in stage 8.0 (TID 184). 6787 bytes result sent to driver -26/04/01 06:30:32 INFO TaskSetManager: Starting task 180.0 in stage 8.0 (TID 188) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:30:32 INFO Executor: Running task 180.0 in stage 8.0 (TID 188) -26/04/01 06:30:32 INFO TaskSetManager: Finished task 176.0 in stage 8.0 (TID 184) in 6525 ms on 10.0.0.133 (executor driver) (177/208) -26/04/01 06:30:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:33 INFO Executor: Finished task 177.0 in stage 8.0 (TID 185). 6787 bytes result sent to driver -26/04/01 06:30:33 INFO Executor: Finished task 178.0 in stage 8.0 (TID 186). 6787 bytes result sent to driver -26/04/01 06:30:33 INFO TaskSetManager: Starting task 181.0 in stage 8.0 (TID 189) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:30:33 INFO TaskSetManager: Starting task 182.0 in stage 8.0 (TID 190) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:30:33 INFO Executor: Running task 181.0 in stage 8.0 (TID 189) -26/04/01 06:30:33 INFO TaskSetManager: Finished task 177.0 in stage 8.0 (TID 185) in 6527 ms on 10.0.0.133 (executor driver) (178/208) -26/04/01 06:30:33 INFO Executor: Running task 182.0 in stage 8.0 (TID 190) -26/04/01 06:30:33 INFO TaskSetManager: Finished task 178.0 in stage 8.0 (TID 186) in 6508 ms on 10.0.0.133 (executor driver) (179/208) -26/04/01 06:30:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:33 INFO Executor: Finished task 179.0 in stage 8.0 (TID 187). 6787 bytes result sent to driver -26/04/01 06:30:33 INFO TaskSetManager: Starting task 183.0 in stage 8.0 (TID 191) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:30:33 INFO TaskSetManager: Finished task 179.0 in stage 8.0 (TID 187) in 6505 ms on 10.0.0.133 (executor driver) (180/208) -26/04/01 06:30:33 INFO Executor: Running task 183.0 in stage 8.0 (TID 191) -26/04/01 06:30:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:39 INFO Executor: Finished task 180.0 in stage 8.0 (TID 188). 6787 bytes result sent to driver -26/04/01 06:30:39 INFO TaskSetManager: Starting task 184.0 in stage 8.0 (TID 192) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:30:39 INFO TaskSetManager: Finished task 180.0 in stage 8.0 (TID 188) in 6512 ms on 10.0.0.133 (executor driver) (181/208) -26/04/01 06:30:39 INFO Executor: Running task 184.0 in stage 8.0 (TID 192) -26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:39 INFO Executor: Finished task 182.0 in stage 8.0 (TID 190). 6787 bytes result sent to driver -26/04/01 06:30:39 INFO TaskSetManager: Starting task 185.0 in stage 8.0 (TID 193) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:30:39 INFO TaskSetManager: Finished task 182.0 in stage 8.0 (TID 190) in 6512 ms on 10.0.0.133 (executor driver) (182/208) -26/04/01 06:30:39 INFO Executor: Running task 185.0 in stage 8.0 (TID 193) -26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:39 INFO Executor: Finished task 181.0 in stage 8.0 (TID 189). 6787 bytes result sent to driver -26/04/01 06:30:39 INFO TaskSetManager: Starting task 186.0 in stage 8.0 (TID 194) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:30:39 INFO Executor: Running task 186.0 in stage 8.0 (TID 194) -26/04/01 06:30:39 INFO TaskSetManager: Finished task 181.0 in stage 8.0 (TID 189) in 6517 ms on 10.0.0.133 (executor driver) (183/208) -26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:39 INFO Executor: Finished task 183.0 in stage 8.0 (TID 191). 6787 bytes result sent to driver -26/04/01 06:30:39 INFO TaskSetManager: Starting task 187.0 in stage 8.0 (TID 195) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:30:39 INFO Executor: Running task 187.0 in stage 8.0 (TID 195) -26/04/01 06:30:39 INFO TaskSetManager: Finished task 183.0 in stage 8.0 (TID 191) in 6513 ms on 10.0.0.133 (executor driver) (184/208) -26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:45 INFO Executor: Finished task 184.0 in stage 8.0 (TID 192). 6787 bytes result sent to driver -26/04/01 06:30:45 INFO TaskSetManager: Starting task 188.0 in stage 8.0 (TID 196) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:30:45 INFO Executor: Running task 188.0 in stage 8.0 (TID 196) -26/04/01 06:30:45 INFO TaskSetManager: Finished task 184.0 in stage 8.0 (TID 192) in 6520 ms on 10.0.0.133 (executor driver) (185/208) -26/04/01 06:30:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:46 INFO Executor: Finished task 185.0 in stage 8.0 (TID 193). 6787 bytes result sent to driver -26/04/01 06:30:46 INFO TaskSetManager: Starting task 189.0 in stage 8.0 (TID 197) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:30:46 INFO TaskSetManager: Finished task 185.0 in stage 8.0 (TID 193) in 6518 ms on 10.0.0.133 (executor driver) (186/208) -26/04/01 06:30:46 INFO Executor: Running task 189.0 in stage 8.0 (TID 197) -26/04/01 06:30:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:46 INFO Executor: Finished task 186.0 in stage 8.0 (TID 194). 6787 bytes result sent to driver -26/04/01 06:30:46 INFO TaskSetManager: Starting task 190.0 in stage 8.0 (TID 198) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:30:46 INFO Executor: Running task 190.0 in stage 8.0 (TID 198) -26/04/01 06:30:46 INFO TaskSetManager: Finished task 186.0 in stage 8.0 (TID 194) in 6514 ms on 10.0.0.133 (executor driver) (187/208) -26/04/01 06:30:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:46 INFO Executor: Finished task 187.0 in stage 8.0 (TID 195). 6787 bytes result sent to driver -26/04/01 06:30:46 INFO TaskSetManager: Starting task 191.0 in stage 8.0 (TID 199) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:30:46 INFO TaskSetManager: Finished task 187.0 in stage 8.0 (TID 195) in 6519 ms on 10.0.0.133 (executor driver) (188/208) -26/04/01 06:30:46 INFO Executor: Running task 191.0 in stage 8.0 (TID 199) -26/04/01 06:30:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:52 INFO Executor: Finished task 188.0 in stage 8.0 (TID 196). 6787 bytes result sent to driver -26/04/01 06:30:52 INFO TaskSetManager: Starting task 192.0 in stage 8.0 (TID 200) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:30:52 INFO TaskSetManager: Finished task 188.0 in stage 8.0 (TID 196) in 6523 ms on 10.0.0.133 (executor driver) (189/208) -26/04/01 06:30:52 INFO Executor: Running task 192.0 in stage 8.0 (TID 200) -26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:52 INFO Executor: Finished task 190.0 in stage 8.0 (TID 198). 6787 bytes result sent to driver -26/04/01 06:30:52 INFO Executor: Finished task 189.0 in stage 8.0 (TID 197). 6787 bytes result sent to driver -26/04/01 06:30:52 INFO TaskSetManager: Starting task 193.0 in stage 8.0 (TID 201) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:30:52 INFO Executor: Running task 193.0 in stage 8.0 (TID 201) -26/04/01 06:30:52 INFO TaskSetManager: Starting task 194.0 in stage 8.0 (TID 202) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:30:52 INFO TaskSetManager: Finished task 190.0 in stage 8.0 (TID 198) in 6525 ms on 10.0.0.133 (executor driver) (190/208) -26/04/01 06:30:52 INFO Executor: Running task 194.0 in stage 8.0 (TID 202) -26/04/01 06:30:52 INFO TaskSetManager: Finished task 189.0 in stage 8.0 (TID 197) in 6527 ms on 10.0.0.133 (executor driver) (191/208) -26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:52 INFO Executor: Finished task 191.0 in stage 8.0 (TID 199). 6787 bytes result sent to driver -26/04/01 06:30:52 INFO TaskSetManager: Starting task 195.0 in stage 8.0 (TID 203) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:30:52 INFO Executor: Running task 195.0 in stage 8.0 (TID 203) -26/04/01 06:30:52 INFO TaskSetManager: Finished task 191.0 in stage 8.0 (TID 199) in 6520 ms on 10.0.0.133 (executor driver) (192/208) -26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:56 INFO Executor: Finished task 192.0 in stage 8.0 (TID 200). 6787 bytes result sent to driver -26/04/01 06:30:56 INFO TaskSetManager: Starting task 196.0 in stage 8.0 (TID 204) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:30:56 INFO Executor: Running task 196.0 in stage 8.0 (TID 204) -26/04/01 06:30:56 INFO TaskSetManager: Finished task 192.0 in stage 8.0 (TID 200) in 4012 ms on 10.0.0.133 (executor driver) (193/208) -26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:56 INFO Executor: Finished task 193.0 in stage 8.0 (TID 201). 6787 bytes result sent to driver -26/04/01 06:30:56 INFO TaskSetManager: Starting task 197.0 in stage 8.0 (TID 205) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:30:56 INFO TaskSetManager: Finished task 193.0 in stage 8.0 (TID 201) in 4007 ms on 10.0.0.133 (executor driver) (194/208) -26/04/01 06:30:56 INFO Executor: Running task 197.0 in stage 8.0 (TID 205) -26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:56 INFO Executor: Finished task 194.0 in stage 8.0 (TID 202). 6787 bytes result sent to driver -26/04/01 06:30:56 INFO TaskSetManager: Starting task 198.0 in stage 8.0 (TID 206) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:30:56 INFO Executor: Running task 198.0 in stage 8.0 (TID 206) -26/04/01 06:30:56 INFO TaskSetManager: Finished task 194.0 in stage 8.0 (TID 202) in 4010 ms on 10.0.0.133 (executor driver) (195/208) -26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:56 INFO Executor: Finished task 195.0 in stage 8.0 (TID 203). 6787 bytes result sent to driver -26/04/01 06:30:56 INFO TaskSetManager: Starting task 199.0 in stage 8.0 (TID 207) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:30:56 INFO TaskSetManager: Finished task 195.0 in stage 8.0 (TID 203) in 4009 ms on 10.0.0.133 (executor driver) (196/208) -26/04/01 06:30:56 INFO Executor: Running task 199.0 in stage 8.0 (TID 207) -26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:30:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:00 INFO Executor: Finished task 196.0 in stage 8.0 (TID 204). 6787 bytes result sent to driver -26/04/01 06:31:00 INFO TaskSetManager: Starting task 200.0 in stage 8.0 (TID 208) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9346 bytes) -26/04/01 06:31:00 INFO Executor: Running task 200.0 in stage 8.0 (TID 208) -26/04/01 06:31:00 INFO TaskSetManager: Finished task 196.0 in stage 8.0 (TID 204) in 4012 ms on 10.0.0.133 (executor driver) (197/208) -26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:00 INFO Executor: Finished task 197.0 in stage 8.0 (TID 205). 6787 bytes result sent to driver -26/04/01 06:31:00 INFO Executor: Finished task 198.0 in stage 8.0 (TID 206). 6787 bytes result sent to driver -26/04/01 06:31:00 INFO TaskSetManager: Starting task 201.0 in stage 8.0 (TID 209) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:31:00 INFO TaskSetManager: Finished task 197.0 in stage 8.0 (TID 205) in 4017 ms on 10.0.0.133 (executor driver) (198/208) -26/04/01 06:31:00 INFO Executor: Running task 201.0 in stage 8.0 (TID 209) -26/04/01 06:31:00 INFO TaskSetManager: Starting task 202.0 in stage 8.0 (TID 210) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:31:00 INFO Executor: Running task 202.0 in stage 8.0 (TID 210) -26/04/01 06:31:00 INFO TaskSetManager: Finished task 198.0 in stage 8.0 (TID 206) in 4013 ms on 10.0.0.133 (executor driver) (199/208) -26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:00 INFO Executor: Finished task 199.0 in stage 8.0 (TID 207). 6787 bytes result sent to driver -26/04/01 06:31:00 INFO TaskSetManager: Starting task 203.0 in stage 8.0 (TID 211) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:31:00 INFO TaskSetManager: Finished task 199.0 in stage 8.0 (TID 207) in 4018 ms on 10.0.0.133 (executor driver) (200/208) -26/04/01 06:31:00 INFO Executor: Running task 203.0 in stage 8.0 (TID 211) -26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:04 INFO Executor: Finished task 200.0 in stage 8.0 (TID 208). 6787 bytes result sent to driver -26/04/01 06:31:04 INFO TaskSetManager: Starting task 204.0 in stage 8.0 (TID 212) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:31:04 INFO TaskSetManager: Finished task 200.0 in stage 8.0 (TID 208) in 4017 ms on 10.0.0.133 (executor driver) (201/208) -26/04/01 06:31:04 INFO Executor: Running task 204.0 in stage 8.0 (TID 212) -26/04/01 06:31:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:04 INFO Executor: Finished task 201.0 in stage 8.0 (TID 209). 6787 bytes result sent to driver -26/04/01 06:31:04 INFO TaskSetManager: Starting task 205.0 in stage 8.0 (TID 213) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:31:04 INFO Executor: Running task 205.0 in stage 8.0 (TID 213) -26/04/01 06:31:04 INFO TaskSetManager: Finished task 201.0 in stage 8.0 (TID 209) in 4018 ms on 10.0.0.133 (executor driver) (202/208) -26/04/01 06:31:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:04 INFO Executor: Finished task 202.0 in stage 8.0 (TID 210). 6787 bytes result sent to driver -26/04/01 06:31:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:04 INFO TaskSetManager: Starting task 206.0 in stage 8.0 (TID 214) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:31:04 INFO TaskSetManager: Finished task 202.0 in stage 8.0 (TID 210) in 4020 ms on 10.0.0.133 (executor driver) (203/208) -26/04/01 06:31:04 INFO Executor: Running task 206.0 in stage 8.0 (TID 214) -26/04/01 06:31:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:05 INFO Executor: Finished task 203.0 in stage 8.0 (TID 211). 6787 bytes result sent to driver -26/04/01 06:31:05 INFO TaskSetManager: Starting task 207.0 in stage 8.0 (TID 215) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:31:05 INFO TaskSetManager: Finished task 203.0 in stage 8.0 (TID 211) in 4020 ms on 10.0.0.133 (executor driver) (204/208) -26/04/01 06:31:05 INFO Executor: Running task 207.0 in stage 8.0 (TID 215) -26/04/01 06:31:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:08 INFO Executor: Finished task 204.0 in stage 8.0 (TID 212). 6787 bytes result sent to driver -26/04/01 06:31:08 INFO TaskSetManager: Finished task 204.0 in stage 8.0 (TID 212) in 3982 ms on 10.0.0.133 (executor driver) (205/208) -26/04/01 06:31:08 INFO Executor: Finished task 205.0 in stage 8.0 (TID 213). 6787 bytes result sent to driver -26/04/01 06:31:08 INFO TaskSetManager: Finished task 205.0 in stage 8.0 (TID 213) in 3978 ms on 10.0.0.133 (executor driver) (206/208) -26/04/01 06:31:08 INFO Executor: Finished task 206.0 in stage 8.0 (TID 214). 6787 bytes result sent to driver -26/04/01 06:31:08 INFO TaskSetManager: Finished task 206.0 in stage 8.0 (TID 214) in 3978 ms on 10.0.0.133 (executor driver) (207/208) -26/04/01 06:31:08 INFO Executor: Finished task 207.0 in stage 8.0 (TID 215). 6787 bytes result sent to driver -26/04/01 06:31:08 INFO TaskSetManager: Finished task 207.0 in stage 8.0 (TID 215) in 3957 ms on 10.0.0.133 (executor driver) (208/208) -26/04/01 06:31:08 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool -26/04/01 06:31:08 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 329.106 s -26/04/01 06:31:08 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:31:08 INFO DAGScheduler: running: Set() -26/04/01 06:31:08 INFO DAGScheduler: waiting: Set() -26/04/01 06:31:08 INFO DAGScheduler: failed: Set() -26/04/01 06:31:08 INFO ShufflePartitionsUtil: For shuffle(0), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 06:31:09 INFO BlockManagerInfo: Removed broadcast_9_piece0 on 10.0.0.133:58418 in memory (size: 10.1 KiB, free: 12.6 GiB) -26/04/01 06:31:09 INFO CodeGenerator: Code generated in 60.090958 ms -26/04/01 06:31:09 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:31:09 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:31:09 INFO DAGScheduler: Final stage: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:31:09 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 9) -26/04/01 06:31:09 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:09 INFO DAGScheduler: Submitting ResultStage 10 (MapPartitionsRDD[23] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:31:09 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 49.9 KiB, free 12.6 GiB) -26/04/01 06:31:09 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 10.4 KiB, free 12.6 GiB) -26/04/01 06:31:09 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:58418 (size: 10.4 KiB, free: 12.6 GiB) -26/04/01 06:31:09 INFO SparkContext: Created broadcast 10 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:09 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 10 (MapPartitionsRDD[23] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:31:09 INFO TaskSchedulerImpl: Adding task set 10.0 with 1 tasks resource profile 0 -26/04/01 06:31:09 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 216) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) -26/04/01 06:31:09 INFO Executor: Running task 0.0 in stage 10.0 (TID 216) -26/04/01 06:31:09 INFO ShuffleBlockFetcherIterator: Getting 208 (853.9 KiB) non-empty blocks including 208 (853.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:31:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms -26/04/01 06:31:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:09 INFO CodeGenerator: Code generated in 5.743666 ms -26/04/01 06:31:09 INFO Executor: Finished task 0.0 in stage 10.0 (TID 216). 5160 bytes result sent to driver -26/04/01 06:31:09 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 216) in 345 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:31:09 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool -26/04/01 06:31:09 INFO DAGScheduler: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.347 s -26/04/01 06:31:09 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:31:09 INFO TaskSchedulerImpl: Killing all running tasks in stage 10: Stage finished -26/04/01 06:31:09 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.351742 s -26/04/01 06:31:09 INFO DAGScheduler: Registering RDD 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 -26/04/01 06:31:09 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:31:09 INFO DAGScheduler: Final stage: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:31:09 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 11) -26/04/01 06:31:09 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:09 INFO DAGScheduler: Submitting ShuffleMapStage 12 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:31:09 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 51.7 KiB, free 12.6 GiB) -26/04/01 06:31:09 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 11.2 KiB, free 12.6 GiB) -26/04/01 06:31:09 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:58418 (size: 11.2 KiB, free: 12.6 GiB) -26/04/01 06:31:09 INFO SparkContext: Created broadcast 11 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:09 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 12 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:31:09 INFO TaskSchedulerImpl: Adding task set 12.0 with 1 tasks resource profile 0 -26/04/01 06:31:09 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 217) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9404 bytes) -26/04/01 06:31:09 INFO Executor: Running task 0.0 in stage 12.0 (TID 217) -26/04/01 06:31:09 INFO CodeGenerator: Code generated in 3.572958 ms -26/04/01 06:31:09 INFO ShuffleBlockFetcherIterator: Getting 208 (853.9 KiB) non-empty blocks including 208 (853.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:31:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:31:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:09 INFO BlockManagerInfo: Removed broadcast_10_piece0 on 10.0.0.133:58418 in memory (size: 10.4 KiB, free: 12.6 GiB) -26/04/01 06:31:09 INFO Executor: Finished task 0.0 in stage 12.0 (TID 217). 6609 bytes result sent to driver -26/04/01 06:31:09 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 217) in 311 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:31:09 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool -26/04/01 06:31:09 INFO DAGScheduler: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.314 s -26/04/01 06:31:09 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:31:09 INFO DAGScheduler: running: Set() -26/04/01 06:31:09 INFO DAGScheduler: waiting: Set() -26/04/01 06:31:09 INFO DAGScheduler: failed: Set() -26/04/01 06:31:09 INFO ShufflePartitionsUtil: For shuffle(1), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 06:31:09 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:31:09 INFO DAGScheduler: Got job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:31:09 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:31:09 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 14) -26/04/01 06:31:09 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:09 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:31:09 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 20.4 KiB, free 12.6 GiB) -26/04/01 06:31:09 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 9.0 KiB, free 12.6 GiB) -26/04/01 06:31:09 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:58418 (size: 9.0 KiB, free: 12.6 GiB) -26/04/01 06:31:09 INFO SparkContext: Created broadcast 12 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:09 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:31:09 INFO TaskSchedulerImpl: Adding task set 15.0 with 1 tasks resource profile 0 -26/04/01 06:31:09 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 218) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) -26/04/01 06:31:09 INFO Executor: Running task 0.0 in stage 15.0 (TID 218) -26/04/01 06:31:09 INFO ShuffleBlockFetcherIterator: Getting 1 (2.6 KiB) non-empty blocks including 1 (2.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:31:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:31:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:09 INFO Executor: Finished task 0.0 in stage 15.0 (TID 218). 5560 bytes result sent to driver -26/04/01 06:31:09 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 218) in 9 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:31:09 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool -26/04/01 06:31:09 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.011 s -26/04/01 06:31:09 INFO DAGScheduler: Job 11 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:31:09 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished -26/04/01 06:31:09 INFO DAGScheduler: Job 11 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.011849 s -26/04/01 06:31:09 INFO SparkContext: SparkContext is stopping with exitCode 0. -26/04/01 06:31:09 INFO CometDriverPlugin: CometDriverPlugin shutdown -26/04/01 06:31:09 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! -26/04/01 06:31:09 INFO MemoryStore: MemoryStore cleared -26/04/01 06:31:09 INFO BlockManager: BlockManager stopped -26/04/01 06:31:09 INFO BlockManagerMaster: BlockManagerMaster stopped -26/04/01 06:31:09 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! -26/04/01 06:31:09 INFO SparkContext: Successfully stopped SparkContext -26/04/01 06:31:09 INFO ShutdownHookManager: Shutdown hook called -26/04/01 06:31:09 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-3a879eef-8ce8-4940-9fe0-6c284ae0de62 -26/04/01 06:31:09 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-9d0b1c01-f2bb-4fb8-875a-59c49514dd0e -26/04/01 06:31:09 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-9d0b1c01-f2bb-4fb8-875a-59c49514dd0e/pyspark-62c23908-1d1c-43ec-8349-112409da8991 - 335.72 real 1314.97 user 5.43 sys - 697319424 maximum resident set size - 0 average shared memory size - 0 average unshared data size - 0 average unshared stack size - 68500 page reclaims - 50 page faults - 0 swaps - 0 block input operations - 0 block output operations - 1275 messages sent - 2361 messages received - 17 signals received - 35609 voluntary context switches - 165249 involuntary context switches - 20342899865196 instructions retired - 4796070096872 cycles elapsed - 579110544 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.log b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.log deleted file mode 100644 index f3cdd0b5c4..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.log +++ /dev/null @@ -1,317 +0,0 @@ -Registering table customer from /opt/tpch/sf100/customer -Registering table lineitem from /opt/tpch/sf100/lineitem -Registering table nation from /opt/tpch/sf100/nation -Registering table orders from /opt/tpch/sf100/orders -Registering table part from /opt/tpch/sf100/part -Registering table partsupp from /opt/tpch/sf100/partsupp -Registering table region from /opt/tpch/sf100/region -Registering table supplier from /opt/tpch/sf100/supplier - -============================================================ -Starting iteration 1 of 1 -============================================================ - -Running query 5 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q5.sql -Executing: -- CometBench-H query 5 derived from TPC-H query 5 under the terms of the TPC Fair Use Policy. --- TP... -== Physical Plan == -AdaptiveSparkPlan (44) -+- CometSort (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometExchange (40) - +- CometHashAggregate (39) - +- CometProject (38) - +- CometBroadcastHashJoin (37) - :- CometProject (32) - : +- CometBroadcastHashJoin (31) - : :- CometProject (27) - : : +- CometSortMergeJoin (26) - : : :- CometSort (21) - : : : +- CometExchange (20) - : : : +- CometProject (19) - : : : +- CometSortMergeJoin (18) - : : : :- CometSort (13) - : : : : +- CometExchange (12) - : : : : +- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (4) - : : : : : +- CometExchange (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometProject (7) - : : : : +- CometFilter (6) - : : : : +- CometNativeScan parquet (5) - : : : +- CometSort (17) - : : : +- CometExchange (16) - : : : +- CometFilter (15) - : : : +- CometNativeScan parquet (14) - : : +- CometSort (25) - : : +- CometExchange (24) - : : +- CometFilter (23) - : : +- CometNativeScan parquet (22) - : +- CometBroadcastExchange (30) - : +- CometFilter (29) - : +- CometNativeScan parquet (28) - +- CometBroadcastExchange (36) - +- CometProject (35) - +- CometFilter (34) - +- CometNativeScan parquet (33) - - -(1) CometNativeScan parquet -Output [2]: [c_custkey#0L, c_nationkey#3L] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/customer] -PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [c_custkey#0L, c_nationkey#3L] -Condition : (isnotnull(c_custkey#0L) AND isnotnull(c_nationkey#3L)) - -(3) CometExchange -Input [2]: [c_custkey#0L, c_nationkey#3L] -Arguments: hashpartitioning(c_custkey#0L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=412] - -(4) CometSort -Input [2]: [c_custkey#0L, c_nationkey#3L] -Arguments: [c_custkey#0L, c_nationkey#3L], [c_custkey#0L ASC NULLS FIRST] - -(5) CometNativeScan parquet -Output [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/orders] -PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1994-01-01), LessThan(o_orderdate,1995-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] -ReadSchema: struct - -(6) CometFilter -Input [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] -Condition : ((((isnotnull(o_orderdate#60) AND (o_orderdate#60 >= 1994-01-01)) AND (o_orderdate#60 < 1995-01-01)) AND isnotnull(o_custkey#57L)) AND isnotnull(o_orderkey#56L)) - -(7) CometProject -Input [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] -Arguments: [o_orderkey#56L, o_custkey#57L], [o_orderkey#56L, o_custkey#57L] - -(8) CometExchange -Input [2]: [o_orderkey#56L, o_custkey#57L] -Arguments: hashpartitioning(o_custkey#57L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=416] - -(9) CometSort -Input [2]: [o_orderkey#56L, o_custkey#57L] -Arguments: [o_orderkey#56L, o_custkey#57L], [o_custkey#57L ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [2]: [c_custkey#0L, c_nationkey#3L] -Right output [2]: [o_orderkey#56L, o_custkey#57L] -Arguments: [c_custkey#0L], [o_custkey#57L], Inner - -(11) CometProject -Input [4]: [c_custkey#0L, c_nationkey#3L, o_orderkey#56L, o_custkey#57L] -Arguments: [c_nationkey#3L, o_orderkey#56L], [c_nationkey#3L, o_orderkey#56L] - -(12) CometExchange -Input [2]: [c_nationkey#3L, o_orderkey#56L] -Arguments: hashpartitioning(o_orderkey#56L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=420] - -(13) CometSort -Input [2]: [c_nationkey#3L, o_orderkey#56L] -Arguments: [c_nationkey#3L, o_orderkey#56L], [o_orderkey#56L ASC NULLS FIRST] - -(14) CometNativeScan parquet -Output [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] -PushedFilters: [IsNotNull(l_orderkey), IsNotNull(l_suppkey)] -ReadSchema: struct - -(15) CometFilter -Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Condition : (isnotnull(l_orderkey#16L) AND isnotnull(l_suppkey#18L)) - -(16) CometExchange -Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: hashpartitioning(l_orderkey#16L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=424] - -(17) CometSort -Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22], [l_orderkey#16L ASC NULLS FIRST] - -(18) CometSortMergeJoin -Left output [2]: [c_nationkey#3L, o_orderkey#56L] -Right output [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: [o_orderkey#56L], [l_orderkey#16L], Inner - -(19) CometProject -Input [6]: [c_nationkey#3L, o_orderkey#56L, l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22], [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] - -(20) CometExchange -Input [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: hashpartitioning(l_suppkey#18L, c_nationkey#3L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=428] - -(21) CometSort -Input [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22], [l_suppkey#18L ASC NULLS FIRST, c_nationkey#3L ASC NULLS FIRST] - -(22) CometNativeScan parquet -Output [2]: [s_suppkey#108L, s_nationkey#111L] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/supplier] -PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] -ReadSchema: struct - -(23) CometFilter -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Condition : (isnotnull(s_suppkey#108L) AND isnotnull(s_nationkey#111L)) - -(24) CometExchange -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: hashpartitioning(s_suppkey#108L, s_nationkey#111L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=432] - -(25) CometSort -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: [s_suppkey#108L, s_nationkey#111L], [s_suppkey#108L ASC NULLS FIRST, s_nationkey#111L ASC NULLS FIRST] - -(26) CometSortMergeJoin -Left output [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Right output [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: [l_suppkey#18L, c_nationkey#3L], [s_suppkey#108L, s_nationkey#111L], Inner - -(27) CometProject -Input [6]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22, s_suppkey#108L, s_nationkey#111L] -Arguments: [l_extendedprice#21, l_discount#22, s_nationkey#111L], [l_extendedprice#21, l_discount#22, s_nationkey#111L] - -(28) CometNativeScan parquet -Output [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/nation] -PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] -ReadSchema: struct - -(29) CometFilter -Input [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] -Condition : ((isnotnull(n_nationkey#48L) AND isnotnull(n_regionkey#50L)) AND might_contain(Subquery subquery#128, [id=#161], xxhash64(n_regionkey#50L, 42))) - -(30) CometBroadcastExchange -Input [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] -Arguments: [n_nationkey#48L, n_name#49, n_regionkey#50L] - -(31) CometBroadcastHashJoin -Left output [3]: [l_extendedprice#21, l_discount#22, s_nationkey#111L] -Right output [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] -Arguments: [s_nationkey#111L], [n_nationkey#48L], Inner, BuildRight - -(32) CometProject -Input [6]: [l_extendedprice#21, l_discount#22, s_nationkey#111L, n_nationkey#48L, n_name#49, n_regionkey#50L] -Arguments: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L], [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L] - -(33) CometNativeScan parquet -Output [2]: [r_regionkey#102L, r_name#103] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/region] -PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AFRICA), IsNotNull(r_regionkey)] -ReadSchema: struct - -(34) CometFilter -Input [2]: [r_regionkey#102L, r_name#103] -Condition : ((isnotnull(r_name#103) AND (r_name#103 = AFRICA)) AND isnotnull(r_regionkey#102L)) - -(35) CometProject -Input [2]: [r_regionkey#102L, r_name#103] -Arguments: [r_regionkey#102L], [r_regionkey#102L] - -(36) CometBroadcastExchange -Input [1]: [r_regionkey#102L] -Arguments: [r_regionkey#102L] - -(37) CometBroadcastHashJoin -Left output [4]: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L] -Right output [1]: [r_regionkey#102L] -Arguments: [n_regionkey#50L], [r_regionkey#102L], Inner, BuildRight - -(38) CometProject -Input [5]: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L, r_regionkey#102L] -Arguments: [l_extendedprice#21, l_discount#22, n_name#49], [l_extendedprice#21, l_discount#22, n_name#49] - -(39) CometHashAggregate -Input [3]: [l_extendedprice#21, l_discount#22, n_name#49] -Keys [1]: [n_name#49] -Functions [1]: [partial_sum((l_extendedprice#21 * (1 - l_discount#22)))] - -(40) CometExchange -Input [3]: [n_name#49, sum#131, isEmpty#132] -Arguments: hashpartitioning(n_name#49, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=445] - -(41) CometHashAggregate -Input [3]: [n_name#49, sum#131, isEmpty#132] -Keys [1]: [n_name#49] -Functions [1]: [sum((l_extendedprice#21 * (1 - l_discount#22)))] - -(42) CometExchange -Input [2]: [n_name#49, revenue#122] -Arguments: rangepartitioning(revenue#122 DESC NULLS LAST, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=447] - -(43) CometSort -Input [2]: [n_name#49, revenue#122] -Arguments: [n_name#49, revenue#122], [revenue#122 DESC NULLS LAST] - -(44) AdaptiveSparkPlan -Output [2]: [n_name#49, revenue#122] -Arguments: isFinalPlan=false - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 29 Hosting Expression = Subquery subquery#128, [id=#161] -AdaptiveSparkPlan (51) -+- CometHashAggregate (50) - +- CometExchange (49) - +- CometHashAggregate (48) - +- CometProject (47) - +- CometFilter (46) - +- CometNativeScan parquet (45) - - -(45) CometNativeScan parquet -Output [2]: [r_regionkey#102L, r_name#103] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/region] -PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AFRICA), IsNotNull(r_regionkey)] -ReadSchema: struct - -(46) CometFilter -Input [2]: [r_regionkey#102L, r_name#103] -Condition : ((isnotnull(r_name#103) AND (r_name#103 = AFRICA)) AND isnotnull(r_regionkey#102L)) - -(47) CometProject -Input [2]: [r_regionkey#102L, r_name#103] -Arguments: [r_regionkey#102L], [r_regionkey#102L] - -(48) CometHashAggregate -Input [1]: [r_regionkey#102L] -Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)] - -(49) CometExchange -Input [1]: [buf#134] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=159] - -(50) CometHashAggregate -Input [1]: [buf#134] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)] - -(51) AdaptiveSparkPlan -Output [1]: [bloomFilter#127] -Arguments: isFinalPlan=false - - - -Query 5 returned 5 rows, hash=f1e68aab4aa9d0988709357fc210d775 -Query 5 took 482.04 seconds - -Iteration 1 took 482.04 seconds - -Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/comet-offheap8g-q5-tpch-1775047157157.json diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.time b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.time deleted file mode 100644 index 226129debc..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q5.time +++ /dev/null @@ -1,5639 +0,0 @@ -26/04/01 06:31:10 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) -26/04/01 06:31:10 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address -26/04/01 06:31:10 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable -26/04/01 06:31:10 INFO SparkContext: Running Spark version 3.5.8 -26/04/01 06:31:10 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 06:31:10 INFO SparkContext: Java version 17.0.17 -26/04/01 06:31:10 INFO ResourceUtils: ============================================================== -26/04/01 06:31:10 INFO ResourceUtils: No custom resources configured for spark.driver. -26/04/01 06:31:10 INFO ResourceUtils: ============================================================== -26/04/01 06:31:10 INFO SparkContext: Submitted application: comet-offheap8g-q5 benchmark derived from tpch -26/04/01 06:31:10 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 8192, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) -26/04/01 06:31:10 INFO ResourceProfile: Limiting resource is cpu -26/04/01 06:31:10 INFO ResourceProfileManager: Added ResourceProfile id: 0 -26/04/01 06:31:10 INFO SecurityManager: Changing view acls to: andy -26/04/01 06:31:10 INFO SecurityManager: Changing modify acls to: andy -26/04/01 06:31:10 INFO SecurityManager: Changing view acls groups to: -26/04/01 06:31:10 INFO SecurityManager: Changing modify acls groups to: -26/04/01 06:31:10 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY -26/04/01 06:31:10 INFO Utils: Successfully started service 'sparkDriver' on port 58440. -26/04/01 06:31:10 INFO SparkEnv: Registering MapOutputTracker -26/04/01 06:31:10 INFO SparkEnv: Registering BlockManagerMaster -26/04/01 06:31:11 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information -26/04/01 06:31:11 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up -26/04/01 06:31:11 INFO SparkEnv: Registering BlockManagerMasterHeartbeat -26/04/01 06:31:11 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-7f73f6ab-0bd7-4a05-bc84-36cd8add9577 -26/04/01 06:31:11 INFO MemoryStore: MemoryStore started with capacity 12.6 GiB -26/04/01 06:31:11 INFO SparkEnv: Registering OutputCommitCoordinator -26/04/01 06:31:11 INFO SparkContext: Added JAR file:///Users/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar at spark://10.0.0.133:58440/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775046670865 -26/04/01 06:31:11 INFO CometDriverPlugin: CometDriverPlugin init -26/04/01 06:31:11 INFO CometDriverPlugin: Setting spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions -26/04/01 06:31:11 INFO CometDriverPlugin: Comet metrics reporting is disabled. Set spark.comet.metrics.enabled=true to enable. -26/04/01 06:31:11 INFO CometDriverPlugin: Comet is running in unified memory mode and sharing off-heap memory with Spark -26/04/01 06:31:11 INFO DriverPluginContainer: Initialized driver component for plugin org.apache.spark.CometPlugin. -26/04/01 06:31:11 INFO Executor: Starting executor ID driver on host 10.0.0.133 -26/04/01 06:31:11 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 06:31:11 INFO Executor: Java version 17.0.17 -26/04/01 06:31:11 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' -26/04/01 06:31:11 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@5e9ec9dd for default. -26/04/01 06:31:11 INFO Executor: Fetching spark://10.0.0.133:58440/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775046670865 -26/04/01 06:31:11 INFO TransportClientFactory: Successfully created connection to /10.0.0.133:58440 after 8 ms (0 ms spent in bootstraps) -26/04/01 06:31:11 INFO Utils: Fetching spark://10.0.0.133:58440/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-3bd87ee3-3f8e-4785-87c2-97740eb236dc/userFiles-e2b85f9c-464e-478c-a520-f16d67759264/fetchFileTemp16161609104732802677.tmp -26/04/01 06:31:11 INFO Executor: Adding file:/private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-3bd87ee3-3f8e-4785-87c2-97740eb236dc/userFiles-e2b85f9c-464e-478c-a520-f16d67759264/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to class loader default -26/04/01 06:31:11 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 58442. -26/04/01 06:31:11 INFO NettyBlockTransferService: Server created on 10.0.0.133:58442 -26/04/01 06:31:11 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy -26/04/01 06:31:11 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 58442, None) -26/04/01 06:31:11 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:58442 with 12.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 58442, None) -26/04/01 06:31:11 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 58442, None) -26/04/01 06:31:11 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 58442, None) -26/04/01 06:31:11 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. -26/04/01 06:31:11 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. -26/04/01 06:31:11 INFO InMemoryFileIndex: It took 17 ms to list leaf files for 1 paths. -26/04/01 06:31:11 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:31:11 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:31:11 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:31:11 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:31:11 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:11 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:31:11 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:31:11 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) -26/04/01 06:31:11 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:11 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:11 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:31:11 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 -26/04/01 06:31:12 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 06:31:12 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) -26/04/01 06:31:12 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 2041 bytes result sent to driver -26/04/01 06:31:12 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 115 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:31:12 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool -26/04/01 06:31:12 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.441 s -26/04/01 06:31:12 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:31:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished -26/04/01 06:31:12 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.451211 s -26/04/01 06:31:12 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:14 INFO NativeBase: Couldn't locate log file from either COMET_CONF_DIR or comet.log.file.path. Using default log configuration with INFO log level which emits to stderr -26/04/01 06:31:14 INFO core/src/lib.rs: Comet native library version 0.15.0 initialized -26/04/01 06:31:14 INFO NativeBase: Setting system property arrow.enable_unsafe_memory_access to true -26/04/01 06:31:14 INFO NativeBase: Setting system property arrow.enable_null_check_for_get to false -26/04/01 06:31:14 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. -26/04/01 06:31:14 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:31:14 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:31:14 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:31:14 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:31:14 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:14 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:31:14 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:31:14 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) -26/04/01 06:31:14 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:14 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:14 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:31:14 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 -26/04/01 06:31:14 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 06:31:14 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) -26/04/01 06:31:14 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2344 bytes result sent to driver -26/04/01 06:31:14 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 13 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:31:14 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool -26/04/01 06:31:14 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.021 s -26/04/01 06:31:14 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:31:14 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished -26/04/01 06:31:14 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.022628 s -26/04/01 06:31:14 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. -26/04/01 06:31:14 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:14 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:31:14 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:31:14 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:31:14 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:31:14 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:14 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:31:14 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:31:14 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) -26/04/01 06:31:14 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:14 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:14 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:31:14 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 -26/04/01 06:31:14 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 06:31:14 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) -26/04/01 06:31:14 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver -26/04/01 06:31:14 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:31:14 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool -26/04/01 06:31:14 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 06:31:14 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:31:14 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished -26/04/01 06:31:14 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.013560 s -26/04/01 06:31:14 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 06:31:14 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:31:14 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:31:14 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:31:14 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:31:14 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:14 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:31:14 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:31:14 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) -26/04/01 06:31:14 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:14 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:14 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:31:14 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 -26/04/01 06:31:14 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 06:31:14 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) -26/04/01 06:31:14 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver -26/04/01 06:31:14 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:31:14 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool -26/04/01 06:31:14 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.012 s -26/04/01 06:31:14 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:31:14 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished -26/04/01 06:31:14 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.013633 s -26/04/01 06:31:14 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 06:31:14 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:31:14 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:31:14 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:31:14 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:31:14 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:14 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:31:14 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:31:14 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) -26/04/01 06:31:14 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:14 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:14 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:14 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:31:14 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 -26/04/01 06:31:14 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:14 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9299 bytes) -26/04/01 06:31:14 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) -26/04/01 06:31:14 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver -26/04/01 06:31:14 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:31:14 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool -26/04/01 06:31:14 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 06:31:14 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:31:14 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished -26/04/01 06:31:14 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.014272 s -26/04/01 06:31:14 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 06:31:14 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:31:14 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:31:14 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:31:14 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:31:14 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:14 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:15 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 -26/04/01 06:31:15 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 06:31:15 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) -26/04/01 06:31:15 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver -26/04/01 06:31:15 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool -26/04/01 06:31:15 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 06:31:15 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:31:15 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished -26/04/01 06:31:15 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.013480 s -26/04/01 06:31:15 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. -26/04/01 06:31:15 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:31:15 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:31:15 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:15 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:15 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 -26/04/01 06:31:15 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 06:31:15 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) -26/04/01 06:31:15 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver -26/04/01 06:31:15 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool -26/04/01 06:31:15 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.012 s -26/04/01 06:31:15 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:31:15 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished -26/04/01 06:31:15 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.013919 s -26/04/01 06:31:15 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 06:31:15 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 06:31:15 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 06:31:15 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) -26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:15 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 37.9 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:58442 (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:15 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 -26/04/01 06:31:15 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 06:31:15 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) -26/04/01 06:31:15 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver -26/04/01 06:31:15 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool -26/04/01 06:31:15 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.011 s -26/04/01 06:31:15 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:31:15 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished -26/04/01 06:31:15 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.012601 s -26/04/01 06:31:15 INFO FileSourceStrategy: Pushed Filters: IsNotNull(c_custkey),IsNotNull(c_nationkey) -26/04/01 06:31:15 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(c_custkey#0L),isnotnull(c_nationkey#3L) -26/04/01 06:31:15 INFO FileSourceStrategy: Pushed Filters: IsNotNull(o_orderdate),GreaterThanOrEqual(o_orderdate,1994-01-01),LessThan(o_orderdate,1995-01-01),IsNotNull(o_custkey),IsNotNull(o_orderkey) -26/04/01 06:31:15 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(o_orderdate#60),(o_orderdate#60 >= 1994-01-01),(o_orderdate#60 < 1995-01-01),isnotnull(o_custkey#57L),isnotnull(o_orderkey#56L) -26/04/01 06:31:15 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_orderkey),IsNotNull(l_suppkey) -26/04/01 06:31:15 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_orderkey#16L),isnotnull(l_suppkey#18L) -26/04/01 06:31:15 INFO FileSourceStrategy: Pushed Filters: IsNotNull(s_suppkey),IsNotNull(s_nationkey) -26/04/01 06:31:15 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(s_suppkey#108L),isnotnull(s_nationkey#111L) -26/04/01 06:31:15 INFO FileSourceStrategy: Pushed Filters: IsNotNull(n_nationkey),IsNotNull(n_regionkey) -26/04/01 06:31:15 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(n_nationkey#48L),isnotnull(n_regionkey#50L),might_contain(scalar-subquery#128 [], xxhash64(n_regionkey#50L, 42)) -26/04/01 06:31:15 INFO FileSourceStrategy: Pushed Filters: IsNotNull(r_name),EqualTo(r_name,AFRICA),IsNotNull(r_regionkey) -26/04/01 06:31:15 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(r_name#103),(r_name#103 = AFRICA),isnotnull(r_regionkey#102L) -26/04/01 06:31:15 INFO FileSourceStrategy: Pushed Filters: IsNotNull(r_name),EqualTo(r_name,AFRICA),IsNotNull(r_regionkey) -26/04/01 06:31:15 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(r_name#103),(r_name#103 = AFRICA),isnotnull(r_regionkey#102L) -26/04/01 06:31:15 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:58442 in memory (size: 37.9 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 55920126 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:58442 (size: 35.4 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:31:15 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO DAGScheduler: Registering RDD 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 -26/04/01 06:31:15 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:31:15 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:15 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 23.0 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 8.0 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:58442 (size: 8.0 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 9 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:15 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[18] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 8.0 with 1 tasks resource profile 0 -26/04/01 06:31:15 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9260 bytes) -26/04/01 06:31:15 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) -26/04/01 06:31:15 INFO BaseAllocator: Debug mode disabled. Enable with the VM option -Darrow.memory.debug.allocator=true. -26/04/01 06:31:15 INFO DefaultAllocationManagerOption: allocation manager type not specified, using netty as the default type -26/04/01 06:31:15 INFO CheckAllocator: Using DefaultAllocationManager at memory/unsafe/DefaultAllocationManagerFactory.class -26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:15 INFO core/src/execution/jni_api.rs: Comet tokio runtime: using spark.executor.cores=4 worker threads -26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:15 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 6628 bytes result sent to driver -26/04/01 06:31:15 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 269 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool -26/04/01 06:31:15 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.282 s -26/04/01 06:31:15 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:31:15 INFO DAGScheduler: running: Set() -26/04/01 06:31:15 INFO DAGScheduler: waiting: Set() -26/04/01 06:31:15 INFO DAGScheduler: failed: Set() -26/04/01 06:31:15 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:31:15 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:31:15 INFO DAGScheduler: Final stage: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 9) -26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:15 INFO DAGScheduler: Submitting ResultStage 10 (MapPartitionsRDD[22] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 17.6 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 7.5 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:58442 (size: 7.5 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 10 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:15 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 10 (MapPartitionsRDD[22] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 10.0 with 1 tasks resource profile 0 -26/04/01 06:31:15 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 9) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) -26/04/01 06:31:15 INFO Executor: Running task 0.0 in stage 10.0 (TID 9) -26/04/01 06:31:15 INFO ShuffleBlockFetcherIterator: Getting 1 (4.7 KiB) non-empty blocks including 1 (4.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:31:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:15 INFO Executor: Finished task 0.0 in stage 10.0 (TID 9). 9966 bytes result sent to driver -26/04/01 06:31:15 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 9) in 37 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool -26/04/01 06:31:15 INFO DAGScheduler: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.042 s -26/04/01 06:31:15 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:31:15 INFO TaskSchedulerImpl: Killing all running tasks in stage 10: Stage finished -26/04/01 06:31:15 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.045698 s -26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_13 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:58442 (size: 35.4 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 11 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:58442 (size: 35.4 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_13_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 12 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:31:15 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_13_piece0 in memory on 10.0.0.133:58442 (size: 35.4 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 13 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:31:15 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO DAGScheduler: Registering RDD 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 -26/04/01 06:31:15 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 16 output partitions -26/04/01 06:31:15 INFO DAGScheduler: Final stage: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:15 INFO DAGScheduler: Submitting ShuffleMapStage 11 (MapPartitionsRDD[31] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_14 stored as values in memory (estimated size 17.6 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_14_piece0 stored as bytes in memory (estimated size 7.8 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_14_piece0 in memory on 10.0.0.133:58442 (size: 7.8 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 14 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:15 INFO DAGScheduler: Submitting 16 missing tasks from ShuffleMapStage 11 (MapPartitionsRDD[31] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 11.0 with 16 tasks resource profile 0 -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_15 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:31:15 INFO TaskSetManager: Starting task 0.0 in stage 11.0 (TID 10) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:31:15 INFO DAGScheduler: Got job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:31:15 INFO DAGScheduler: Final stage: ResultStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:31:15 INFO TaskSetManager: Starting task 1.0 in stage 11.0 (TID 11) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9334 bytes) -26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:15 INFO TaskSetManager: Starting task 2.0 in stage 11.0 (TID 12) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:31:15 INFO DAGScheduler: Submitting ResultStage 12 (MapPartitionsRDD[29] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:31:15 INFO TaskSetManager: Starting task 3.0 in stage 11.0 (TID 13) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9334 bytes) -26/04/01 06:31:15 INFO Executor: Running task 0.0 in stage 11.0 (TID 10) -26/04/01 06:31:15 INFO Executor: Running task 1.0 in stage 11.0 (TID 11) -26/04/01 06:31:15 INFO Executor: Running task 2.0 in stage 11.0 (TID 12) -26/04/01 06:31:15 INFO Executor: Running task 3.0 in stage 11.0 (TID 13) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_16 stored as values in memory (estimated size 14.0 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_16_piece0 stored as bytes in memory (estimated size 5.4 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_16_piece0 in memory on 10.0.0.133:58442 (size: 5.4 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 16 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:15 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 12 (MapPartitionsRDD[29] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:31:15 INFO BlockManagerInfo: Removed broadcast_9_piece0 on 10.0.0.133:58442 in memory (size: 8.0 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 12.0 with 1 tasks resource profile 0 -26/04/01 06:31:15 INFO BlockManagerInfo: Removed broadcast_10_piece0 on 10.0.0.133:58442 in memory (size: 7.5 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_15_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_15_piece0 in memory on 10.0.0.133:58442 (size: 35.4 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 15 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:31:15 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:15 INFO DAGScheduler: Registering RDD 34 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 2 -26/04/01 06:31:15 INFO DAGScheduler: Got map stage job 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 64 output partitions -26/04/01 06:31:15 INFO DAGScheduler: Final stage: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:15 INFO DAGScheduler: Submitting ShuffleMapStage 13 (MapPartitionsRDD[34] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_17 stored as values in memory (estimated size 23.3 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_17_piece0 stored as bytes in memory (estimated size 8.1 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_17_piece0 in memory on 10.0.0.133:58442 (size: 8.1 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 17 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:15 INFO DAGScheduler: Submitting 64 missing tasks from ShuffleMapStage 13 (MapPartitionsRDD[34] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 13.0 with 64 tasks resource profile 0 -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_18 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_18_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_18_piece0 in memory on 10.0.0.133:58442 (size: 35.4 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 18 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:31:15 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO DAGScheduler: Registering RDD 37 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 3 -26/04/01 06:31:15 INFO DAGScheduler: Got map stage job 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions -26/04/01 06:31:15 INFO DAGScheduler: Final stage: ShuffleMapStage 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:15 INFO CometScanExec: Planning scan with bin packing, max size: 55920126 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 INFO DAGScheduler: Submitting ShuffleMapStage 14 (MapPartitionsRDD[37] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_20 stored as values in memory (estimated size 18.4 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_19 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_20_piece0 stored as bytes in memory (estimated size 8.0 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_20_piece0 in memory on 10.0.0.133:58442 (size: 8.0 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 20 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:15 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 14 (MapPartitionsRDD[37] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 14.0 with 208 tasks resource profile 0 -26/04/01 06:31:15 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:31:15 INFO DAGScheduler: Got job 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:31:15 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_19_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:15 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_19_piece0 in memory on 10.0.0.133:58442 (size: 35.4 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 19 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:31:15 INFO FileSourceScanExec: Planning scan with bin packing, max size: 55920126 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 06:31:15 WARN DAGScheduler: Broadcasting large task binary with size 1113.7 KiB -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_21 stored as values in memory (estimated size 1113.7 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_21_piece0 stored as bytes in memory (estimated size 30.7 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_21_piece0 in memory on 10.0.0.133:58442 (size: 30.7 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 21 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:15 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 15.0 with 1 tasks resource profile 0 -26/04/01 06:31:15 INFO DAGScheduler: Registering RDD 40 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 4 -26/04/01 06:31:15 INFO DAGScheduler: Got map stage job 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 4 output partitions -26/04/01 06:31:15 INFO DAGScheduler: Final stage: ShuffleMapStage 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:31:15 INFO DAGScheduler: Parents of final stage: List() -26/04/01 06:31:15 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:15 INFO DAGScheduler: Submitting ShuffleMapStage 16 (MapPartitionsRDD[40] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_22 stored as values in memory (estimated size 17.6 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO MemoryStore: Block broadcast_22_piece0 stored as bytes in memory (estimated size 7.8 KiB, free 12.6 GiB) -26/04/01 06:31:15 INFO BlockManagerInfo: Added broadcast_22_piece0 in memory on 10.0.0.133:58442 (size: 7.8 KiB, free: 12.6 GiB) -26/04/01 06:31:15 INFO SparkContext: Created broadcast 22 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:15 INFO DAGScheduler: Submitting 4 missing tasks from ShuffleMapStage 16 (MapPartitionsRDD[40] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3)) -26/04/01 06:31:15 INFO TaskSchedulerImpl: Adding task set 16.0 with 4 tasks resource profile 0 -26/04/01 06:31:16 INFO Executor: Finished task 1.0 in stage 11.0 (TID 11). 6649 bytes result sent to driver -26/04/01 06:31:16 INFO TaskSetManager: Starting task 4.0 in stage 11.0 (TID 14) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:31:16 INFO Executor: Running task 4.0 in stage 11.0 (TID 14) -26/04/01 06:31:16 INFO TaskSetManager: Finished task 1.0 in stage 11.0 (TID 11) in 740 ms on 10.0.0.133 (executor driver) (1/16) -26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:16 INFO Executor: Finished task 2.0 in stage 11.0 (TID 12). 6649 bytes result sent to driver -26/04/01 06:31:16 INFO Executor: Finished task 3.0 in stage 11.0 (TID 13). 6649 bytes result sent to driver -26/04/01 06:31:16 INFO TaskSetManager: Starting task 5.0 in stage 11.0 (TID 15) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:31:16 INFO TaskSetManager: Finished task 2.0 in stage 11.0 (TID 12) in 743 ms on 10.0.0.133 (executor driver) (2/16) -26/04/01 06:31:16 INFO Executor: Running task 5.0 in stage 11.0 (TID 15) -26/04/01 06:31:16 INFO TaskSetManager: Starting task 6.0 in stage 11.0 (TID 16) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:31:16 INFO Executor: Running task 6.0 in stage 11.0 (TID 16) -26/04/01 06:31:16 INFO TaskSetManager: Finished task 3.0 in stage 11.0 (TID 13) in 743 ms on 10.0.0.133 (executor driver) (3/16) -26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:16 INFO Executor: Finished task 0.0 in stage 11.0 (TID 10). 6649 bytes result sent to driver -26/04/01 06:31:16 INFO TaskSetManager: Starting task 7.0 in stage 11.0 (TID 17) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9334 bytes) -26/04/01 06:31:16 INFO Executor: Running task 7.0 in stage 11.0 (TID 17) -26/04/01 06:31:16 INFO TaskSetManager: Finished task 0.0 in stage 11.0 (TID 10) in 745 ms on 10.0.0.133 (executor driver) (4/16) -26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:17 INFO Executor: Finished task 6.0 in stage 11.0 (TID 16). 6606 bytes result sent to driver -26/04/01 06:31:17 INFO TaskSetManager: Starting task 8.0 in stage 11.0 (TID 18) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:31:17 INFO Executor: Finished task 7.0 in stage 11.0 (TID 17). 6563 bytes result sent to driver -26/04/01 06:31:17 INFO Executor: Running task 8.0 in stage 11.0 (TID 18) -26/04/01 06:31:17 INFO TaskSetManager: Starting task 9.0 in stage 11.0 (TID 19) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:31:17 INFO Executor: Finished task 5.0 in stage 11.0 (TID 15). 6606 bytes result sent to driver -26/04/01 06:31:17 INFO TaskSetManager: Finished task 6.0 in stage 11.0 (TID 16) in 743 ms on 10.0.0.133 (executor driver) (5/16) -26/04/01 06:31:17 INFO Executor: Running task 9.0 in stage 11.0 (TID 19) -26/04/01 06:31:17 INFO TaskSetManager: Starting task 10.0 in stage 11.0 (TID 20) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9334 bytes) -26/04/01 06:31:17 INFO Executor: Running task 10.0 in stage 11.0 (TID 20) -26/04/01 06:31:17 INFO TaskSetManager: Finished task 7.0 in stage 11.0 (TID 17) in 741 ms on 10.0.0.133 (executor driver) (6/16) -26/04/01 06:31:17 INFO Executor: Finished task 4.0 in stage 11.0 (TID 14). 6563 bytes result sent to driver -26/04/01 06:31:17 INFO TaskSetManager: Starting task 11.0 in stage 11.0 (TID 21) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:31:17 INFO TaskSetManager: Finished task 5.0 in stage 11.0 (TID 15) in 745 ms on 10.0.0.133 (executor driver) (7/16) -26/04/01 06:31:17 INFO Executor: Running task 11.0 in stage 11.0 (TID 21) -26/04/01 06:31:17 INFO TaskSetManager: Finished task 4.0 in stage 11.0 (TID 14) in 748 ms on 10.0.0.133 (executor driver) (8/16) -26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO Executor: Finished task 9.0 in stage 11.0 (TID 19). 6606 bytes result sent to driver -26/04/01 06:31:18 INFO Executor: Finished task 10.0 in stage 11.0 (TID 20). 6606 bytes result sent to driver -26/04/01 06:31:18 INFO TaskSetManager: Starting task 12.0 in stage 11.0 (TID 22) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9334 bytes) -26/04/01 06:31:18 INFO Executor: Finished task 11.0 in stage 11.0 (TID 21). 6563 bytes result sent to driver -26/04/01 06:31:18 INFO Executor: Running task 12.0 in stage 11.0 (TID 22) -26/04/01 06:31:18 INFO TaskSetManager: Starting task 13.0 in stage 11.0 (TID 23) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9333 bytes) -26/04/01 06:31:18 INFO TaskSetManager: Finished task 9.0 in stage 11.0 (TID 19) in 724 ms on 10.0.0.133 (executor driver) (9/16) -26/04/01 06:31:18 INFO Executor: Running task 13.0 in stage 11.0 (TID 23) -26/04/01 06:31:18 INFO TaskSetManager: Finished task 10.0 in stage 11.0 (TID 20) in 724 ms on 10.0.0.133 (executor driver) (10/16) -26/04/01 06:31:18 INFO Executor: Finished task 8.0 in stage 11.0 (TID 18). 6606 bytes result sent to driver -26/04/01 06:31:18 INFO TaskSetManager: Starting task 14.0 in stage 11.0 (TID 24) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9334 bytes) -26/04/01 06:31:18 INFO TaskSetManager: Finished task 11.0 in stage 11.0 (TID 21) in 724 ms on 10.0.0.133 (executor driver) (11/16) -26/04/01 06:31:18 INFO Executor: Running task 14.0 in stage 11.0 (TID 24) -26/04/01 06:31:18 INFO TaskSetManager: Starting task 15.0 in stage 11.0 (TID 25) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9334 bytes) -26/04/01 06:31:18 INFO Executor: Running task 15.0 in stage 11.0 (TID 25) -26/04/01 06:31:18 INFO TaskSetManager: Finished task 8.0 in stage 11.0 (TID 18) in 727 ms on 10.0.0.133 (executor driver) (12/16) -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO Executor: Finished task 13.0 in stage 11.0 (TID 23). 6606 bytes result sent to driver -26/04/01 06:31:18 INFO Executor: Finished task 12.0 in stage 11.0 (TID 22). 6606 bytes result sent to driver -26/04/01 06:31:18 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 26) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9271 bytes) -26/04/01 06:31:18 INFO Executor: Running task 0.0 in stage 12.0 (TID 26) -26/04/01 06:31:18 INFO TaskSetManager: Starting task 0.0 in stage 13.0 (TID 27) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:18 INFO TaskSetManager: Finished task 13.0 in stage 11.0 (TID 23) in 719 ms on 10.0.0.133 (executor driver) (13/16) -26/04/01 06:31:18 INFO Executor: Running task 0.0 in stage 13.0 (TID 27) -26/04/01 06:31:18 INFO TaskSetManager: Finished task 12.0 in stage 11.0 (TID 22) in 720 ms on 10.0.0.133 (executor driver) (14/16) -26/04/01 06:31:18 INFO Executor: Finished task 14.0 in stage 11.0 (TID 24). 6563 bytes result sent to driver -26/04/01 06:31:18 INFO TaskSetManager: Starting task 1.0 in stage 13.0 (TID 28) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:31:18 INFO Executor: Running task 1.0 in stage 13.0 (TID 28) -26/04/01 06:31:18 INFO TaskSetManager: Finished task 14.0 in stage 11.0 (TID 24) in 720 ms on 10.0.0.133 (executor driver) (15/16) -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO Executor: Finished task 15.0 in stage 11.0 (TID 25). 6563 bytes result sent to driver -26/04/01 06:31:18 INFO TaskSetManager: Starting task 2.0 in stage 13.0 (TID 29) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:18 INFO TaskSetManager: Finished task 15.0 in stage 11.0 (TID 25) in 721 ms on 10.0.0.133 (executor driver) (16/16) -26/04/01 06:31:18 INFO Executor: Running task 2.0 in stage 13.0 (TID 29) -26/04/01 06:31:18 INFO TaskSchedulerImpl: Removed TaskSet 11.0, whose tasks have all completed, from pool -26/04/01 06:31:18 INFO DAGScheduler: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.935 s -26/04/01 06:31:18 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:31:18 INFO DAGScheduler: running: Set(ResultStage 15, ResultStage 12, ShuffleMapStage 16, ShuffleMapStage 13, ShuffleMapStage 14) -26/04/01 06:31:18 INFO DAGScheduler: waiting: Set() -26/04/01 06:31:18 INFO DAGScheduler: failed: Set() -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO Executor: Finished task 0.0 in stage 12.0 (TID 26). 4405 bytes result sent to driver -26/04/01 06:31:18 INFO TaskSetManager: Starting task 3.0 in stage 13.0 (TID 30) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:18 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 26) in 25 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:31:18 INFO Executor: Running task 3.0 in stage 13.0 (TID 30) -26/04/01 06:31:18 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool -26/04/01 06:31:18 INFO DAGScheduler: ResultStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.955 s -26/04/01 06:31:18 INFO DAGScheduler: Job 11 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:31:18 INFO TaskSchedulerImpl: Killing all running tasks in stage 12: Stage finished -26/04/01 06:31:18 INFO DAGScheduler: Job 11 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 2.955010 s -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:18 INFO Utils: Coalesced 1 broadcast batches into 1 (1 rows) -26/04/01 06:31:18 INFO MemoryStore: Block broadcast_23 stored as values in memory (estimated size 416.0 B, free 12.6 GiB) -26/04/01 06:31:18 INFO MemoryStore: Block broadcast_23_piece0 stored as bytes in memory (estimated size 367.0 B, free 12.6 GiB) -26/04/01 06:31:18 INFO BlockManagerInfo: Added broadcast_23_piece0 in memory on 10.0.0.133:58442 (size: 367.0 B, free: 12.6 GiB) -26/04/01 06:31:18 INFO SparkContext: Created broadcast 23 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:31:20 INFO Executor: Finished task 2.0 in stage 13.0 (TID 29). 6675 bytes result sent to driver -26/04/01 06:31:20 INFO Executor: Finished task 1.0 in stage 13.0 (TID 28). 6675 bytes result sent to driver -26/04/01 06:31:20 INFO TaskSetManager: Starting task 4.0 in stage 13.0 (TID 31) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:20 INFO Executor: Running task 4.0 in stage 13.0 (TID 31) -26/04/01 06:31:20 INFO TaskSetManager: Starting task 5.0 in stage 13.0 (TID 32) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:31:20 INFO TaskSetManager: Finished task 2.0 in stage 13.0 (TID 29) in 1271 ms on 10.0.0.133 (executor driver) (1/64) -26/04/01 06:31:20 INFO Executor: Running task 5.0 in stage 13.0 (TID 32) -26/04/01 06:31:20 INFO TaskSetManager: Finished task 1.0 in stage 13.0 (TID 28) in 1273 ms on 10.0.0.133 (executor driver) (2/64) -26/04/01 06:31:20 INFO Executor: Finished task 0.0 in stage 13.0 (TID 27). 6675 bytes result sent to driver -26/04/01 06:31:20 INFO TaskSetManager: Starting task 6.0 in stage 13.0 (TID 33) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:20 INFO TaskSetManager: Finished task 0.0 in stage 13.0 (TID 27) in 1275 ms on 10.0.0.133 (executor driver) (3/64) -26/04/01 06:31:20 INFO Executor: Running task 6.0 in stage 13.0 (TID 33) -26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:20 INFO Executor: Finished task 3.0 in stage 13.0 (TID 30). 6675 bytes result sent to driver -26/04/01 06:31:20 INFO TaskSetManager: Starting task 7.0 in stage 13.0 (TID 34) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:20 INFO TaskSetManager: Finished task 3.0 in stage 13.0 (TID 30) in 1270 ms on 10.0.0.133 (executor driver) (4/64) -26/04/01 06:31:20 INFO Executor: Running task 7.0 in stage 13.0 (TID 34) -26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:21 INFO Executor: Finished task 6.0 in stage 13.0 (TID 33). 6675 bytes result sent to driver -26/04/01 06:31:21 INFO TaskSetManager: Starting task 8.0 in stage 13.0 (TID 35) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:31:21 INFO Executor: Finished task 4.0 in stage 13.0 (TID 31). 6675 bytes result sent to driver -26/04/01 06:31:21 INFO Executor: Running task 8.0 in stage 13.0 (TID 35) -26/04/01 06:31:21 INFO TaskSetManager: Starting task 9.0 in stage 13.0 (TID 36) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:21 INFO TaskSetManager: Finished task 6.0 in stage 13.0 (TID 33) in 1265 ms on 10.0.0.133 (executor driver) (5/64) -26/04/01 06:31:21 INFO TaskSetManager: Finished task 4.0 in stage 13.0 (TID 31) in 1265 ms on 10.0.0.133 (executor driver) (6/64) -26/04/01 06:31:21 INFO Executor: Running task 9.0 in stage 13.0 (TID 36) -26/04/01 06:31:21 INFO Executor: Finished task 5.0 in stage 13.0 (TID 32). 6761 bytes result sent to driver -26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:21 INFO TaskSetManager: Starting task 10.0 in stage 13.0 (TID 37) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:31:21 INFO TaskSetManager: Finished task 5.0 in stage 13.0 (TID 32) in 1269 ms on 10.0.0.133 (executor driver) (7/64) -26/04/01 06:31:21 INFO Executor: Running task 10.0 in stage 13.0 (TID 37) -26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:21 INFO Executor: Finished task 7.0 in stage 13.0 (TID 34). 6718 bytes result sent to driver -26/04/01 06:31:21 INFO TaskSetManager: Starting task 11.0 in stage 13.0 (TID 38) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:21 INFO TaskSetManager: Finished task 7.0 in stage 13.0 (TID 34) in 1271 ms on 10.0.0.133 (executor driver) (8/64) -26/04/01 06:31:21 INFO Executor: Running task 11.0 in stage 13.0 (TID 38) -26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:22 INFO Executor: Finished task 8.0 in stage 13.0 (TID 35). 6718 bytes result sent to driver -26/04/01 06:31:22 INFO Executor: Finished task 9.0 in stage 13.0 (TID 36). 6718 bytes result sent to driver -26/04/01 06:31:22 INFO TaskSetManager: Starting task 12.0 in stage 13.0 (TID 39) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:22 INFO Executor: Running task 12.0 in stage 13.0 (TID 39) -26/04/01 06:31:22 INFO TaskSetManager: Starting task 13.0 in stage 13.0 (TID 40) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:22 INFO TaskSetManager: Finished task 8.0 in stage 13.0 (TID 35) in 1265 ms on 10.0.0.133 (executor driver) (9/64) -26/04/01 06:31:22 INFO Executor: Running task 13.0 in stage 13.0 (TID 40) -26/04/01 06:31:22 INFO TaskSetManager: Finished task 9.0 in stage 13.0 (TID 36) in 1265 ms on 10.0.0.133 (executor driver) (10/64) -26/04/01 06:31:22 INFO Executor: Finished task 10.0 in stage 13.0 (TID 37). 6675 bytes result sent to driver -26/04/01 06:31:22 INFO TaskSetManager: Starting task 14.0 in stage 13.0 (TID 41) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:22 INFO TaskSetManager: Finished task 10.0 in stage 13.0 (TID 37) in 1262 ms on 10.0.0.133 (executor driver) (11/64) -26/04/01 06:31:22 INFO Executor: Running task 14.0 in stage 13.0 (TID 41) -26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:22 INFO Executor: Finished task 11.0 in stage 13.0 (TID 38). 6675 bytes result sent to driver -26/04/01 06:31:22 INFO TaskSetManager: Starting task 15.0 in stage 13.0 (TID 42) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:22 INFO TaskSetManager: Finished task 11.0 in stage 13.0 (TID 38) in 1266 ms on 10.0.0.133 (executor driver) (12/64) -26/04/01 06:31:22 INFO Executor: Running task 15.0 in stage 13.0 (TID 42) -26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:23 INFO Executor: Finished task 13.0 in stage 13.0 (TID 40). 6675 bytes result sent to driver -26/04/01 06:31:23 INFO TaskSetManager: Starting task 16.0 in stage 13.0 (TID 43) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:31:23 INFO Executor: Running task 16.0 in stage 13.0 (TID 43) -26/04/01 06:31:23 INFO TaskSetManager: Finished task 13.0 in stage 13.0 (TID 40) in 1270 ms on 10.0.0.133 (executor driver) (13/64) -26/04/01 06:31:23 INFO Executor: Finished task 14.0 in stage 13.0 (TID 41). 6675 bytes result sent to driver -26/04/01 06:31:23 INFO TaskSetManager: Starting task 17.0 in stage 13.0 (TID 44) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:23 INFO Executor: Finished task 12.0 in stage 13.0 (TID 39). 6675 bytes result sent to driver -26/04/01 06:31:23 INFO TaskSetManager: Finished task 14.0 in stage 13.0 (TID 41) in 1270 ms on 10.0.0.133 (executor driver) (14/64) -26/04/01 06:31:23 INFO Executor: Running task 17.0 in stage 13.0 (TID 44) -26/04/01 06:31:23 INFO TaskSetManager: Starting task 18.0 in stage 13.0 (TID 45) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:31:23 INFO Executor: Running task 18.0 in stage 13.0 (TID 45) -26/04/01 06:31:23 INFO TaskSetManager: Finished task 12.0 in stage 13.0 (TID 39) in 1271 ms on 10.0.0.133 (executor driver) (15/64) -26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:23 INFO Executor: Finished task 15.0 in stage 13.0 (TID 42). 6675 bytes result sent to driver -26/04/01 06:31:23 INFO TaskSetManager: Starting task 19.0 in stage 13.0 (TID 46) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:23 INFO TaskSetManager: Finished task 15.0 in stage 13.0 (TID 42) in 1269 ms on 10.0.0.133 (executor driver) (16/64) -26/04/01 06:31:23 INFO Executor: Running task 19.0 in stage 13.0 (TID 46) -26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:25 INFO Executor: Finished task 17.0 in stage 13.0 (TID 44). 6675 bytes result sent to driver -26/04/01 06:31:25 INFO Executor: Finished task 16.0 in stage 13.0 (TID 43). 6675 bytes result sent to driver -26/04/01 06:31:25 INFO TaskSetManager: Starting task 20.0 in stage 13.0 (TID 47) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:25 INFO TaskSetManager: Starting task 21.0 in stage 13.0 (TID 48) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:31:25 INFO Executor: Running task 20.0 in stage 13.0 (TID 47) -26/04/01 06:31:25 INFO TaskSetManager: Finished task 16.0 in stage 13.0 (TID 43) in 1270 ms on 10.0.0.133 (executor driver) (17/64) -26/04/01 06:31:25 INFO Executor: Running task 21.0 in stage 13.0 (TID 48) -26/04/01 06:31:25 INFO TaskSetManager: Finished task 17.0 in stage 13.0 (TID 44) in 1268 ms on 10.0.0.133 (executor driver) (18/64) -26/04/01 06:31:25 INFO Executor: Finished task 18.0 in stage 13.0 (TID 45). 6675 bytes result sent to driver -26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:25 INFO TaskSetManager: Starting task 22.0 in stage 13.0 (TID 49) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:25 INFO TaskSetManager: Finished task 18.0 in stage 13.0 (TID 45) in 1270 ms on 10.0.0.133 (executor driver) (19/64) -26/04/01 06:31:25 INFO Executor: Running task 22.0 in stage 13.0 (TID 49) -26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:25 INFO Executor: Finished task 19.0 in stage 13.0 (TID 46). 6675 bytes result sent to driver -26/04/01 06:31:25 INFO TaskSetManager: Starting task 23.0 in stage 13.0 (TID 50) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:25 INFO Executor: Running task 23.0 in stage 13.0 (TID 50) -26/04/01 06:31:25 INFO TaskSetManager: Finished task 19.0 in stage 13.0 (TID 46) in 1268 ms on 10.0.0.133 (executor driver) (20/64) -26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:26 INFO Executor: Finished task 22.0 in stage 13.0 (TID 49). 6675 bytes result sent to driver -26/04/01 06:31:26 INFO TaskSetManager: Starting task 24.0 in stage 13.0 (TID 51) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:31:26 INFO TaskSetManager: Finished task 22.0 in stage 13.0 (TID 49) in 1259 ms on 10.0.0.133 (executor driver) (21/64) -26/04/01 06:31:26 INFO Executor: Running task 24.0 in stage 13.0 (TID 51) -26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:26 INFO Executor: Finished task 21.0 in stage 13.0 (TID 48). 6675 bytes result sent to driver -26/04/01 06:31:26 INFO TaskSetManager: Starting task 25.0 in stage 13.0 (TID 52) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:26 INFO Executor: Running task 25.0 in stage 13.0 (TID 52) -26/04/01 06:31:26 INFO TaskSetManager: Finished task 21.0 in stage 13.0 (TID 48) in 1264 ms on 10.0.0.133 (executor driver) (22/64) -26/04/01 06:31:26 INFO Executor: Finished task 20.0 in stage 13.0 (TID 47). 6675 bytes result sent to driver -26/04/01 06:31:26 INFO TaskSetManager: Starting task 26.0 in stage 13.0 (TID 53) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:26 INFO TaskSetManager: Finished task 20.0 in stage 13.0 (TID 47) in 1265 ms on 10.0.0.133 (executor driver) (23/64) -26/04/01 06:31:26 INFO Executor: Running task 26.0 in stage 13.0 (TID 53) -26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:26 INFO Executor: Finished task 23.0 in stage 13.0 (TID 50). 6675 bytes result sent to driver -26/04/01 06:31:26 INFO TaskSetManager: Starting task 27.0 in stage 13.0 (TID 54) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:26 INFO Executor: Running task 27.0 in stage 13.0 (TID 54) -26/04/01 06:31:26 INFO TaskSetManager: Finished task 23.0 in stage 13.0 (TID 50) in 1266 ms on 10.0.0.133 (executor driver) (24/64) -26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:27 INFO Executor: Finished task 24.0 in stage 13.0 (TID 51). 6675 bytes result sent to driver -26/04/01 06:31:27 INFO Executor: Finished task 25.0 in stage 13.0 (TID 52). 6675 bytes result sent to driver -26/04/01 06:31:27 INFO Executor: Finished task 26.0 in stage 13.0 (TID 53). 6675 bytes result sent to driver -26/04/01 06:31:27 INFO TaskSetManager: Starting task 28.0 in stage 13.0 (TID 55) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:27 INFO Executor: Running task 28.0 in stage 13.0 (TID 55) -26/04/01 06:31:27 INFO TaskSetManager: Starting task 29.0 in stage 13.0 (TID 56) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:27 INFO TaskSetManager: Finished task 24.0 in stage 13.0 (TID 51) in 1271 ms on 10.0.0.133 (executor driver) (25/64) -26/04/01 06:31:27 INFO TaskSetManager: Finished task 25.0 in stage 13.0 (TID 52) in 1268 ms on 10.0.0.133 (executor driver) (26/64) -26/04/01 06:31:27 INFO Executor: Running task 29.0 in stage 13.0 (TID 56) -26/04/01 06:31:27 INFO TaskSetManager: Starting task 30.0 in stage 13.0 (TID 57) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9264 bytes) -26/04/01 06:31:27 INFO TaskSetManager: Finished task 26.0 in stage 13.0 (TID 53) in 1267 ms on 10.0.0.133 (executor driver) (27/64) -26/04/01 06:31:27 INFO Executor: Running task 30.0 in stage 13.0 (TID 57) -26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:27 INFO Executor: Finished task 27.0 in stage 13.0 (TID 54). 6675 bytes result sent to driver -26/04/01 06:31:27 INFO TaskSetManager: Starting task 31.0 in stage 13.0 (TID 58) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9263 bytes) -26/04/01 06:31:27 INFO TaskSetManager: Finished task 27.0 in stage 13.0 (TID 54) in 1271 ms on 10.0.0.133 (executor driver) (28/64) -26/04/01 06:31:27 INFO Executor: Running task 31.0 in stage 13.0 (TID 58) -26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:28 INFO Executor: Finished task 30.0 in stage 13.0 (TID 57). 6675 bytes result sent to driver -26/04/01 06:31:28 INFO TaskSetManager: Starting task 32.0 in stage 13.0 (TID 59) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:28 INFO TaskSetManager: Finished task 30.0 in stage 13.0 (TID 57) in 1274 ms on 10.0.0.133 (executor driver) (29/64) -26/04/01 06:31:28 INFO Executor: Running task 32.0 in stage 13.0 (TID 59) -26/04/01 06:31:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:28 INFO Executor: Finished task 28.0 in stage 13.0 (TID 55). 6675 bytes result sent to driver -26/04/01 06:31:28 INFO TaskSetManager: Starting task 33.0 in stage 13.0 (TID 60) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:28 INFO Executor: Running task 33.0 in stage 13.0 (TID 60) -26/04/01 06:31:28 INFO TaskSetManager: Finished task 28.0 in stage 13.0 (TID 55) in 1277 ms on 10.0.0.133 (executor driver) (30/64) -26/04/01 06:31:28 INFO Executor: Finished task 29.0 in stage 13.0 (TID 56). 6675 bytes result sent to driver -26/04/01 06:31:28 INFO TaskSetManager: Starting task 34.0 in stage 13.0 (TID 61) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:28 INFO Executor: Running task 34.0 in stage 13.0 (TID 61) -26/04/01 06:31:28 INFO TaskSetManager: Finished task 29.0 in stage 13.0 (TID 56) in 1276 ms on 10.0.0.133 (executor driver) (31/64) -26/04/01 06:31:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:29 INFO Executor: Finished task 31.0 in stage 13.0 (TID 58). 6675 bytes result sent to driver -26/04/01 06:31:29 INFO TaskSetManager: Starting task 35.0 in stage 13.0 (TID 62) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:31:29 INFO TaskSetManager: Finished task 31.0 in stage 13.0 (TID 58) in 1276 ms on 10.0.0.133 (executor driver) (32/64) -26/04/01 06:31:29 INFO Executor: Running task 35.0 in stage 13.0 (TID 62) -26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:29 INFO Executor: Finished task 33.0 in stage 13.0 (TID 60). 6675 bytes result sent to driver -26/04/01 06:31:29 INFO Executor: Finished task 32.0 in stage 13.0 (TID 59). 6675 bytes result sent to driver -26/04/01 06:31:29 INFO TaskSetManager: Starting task 36.0 in stage 13.0 (TID 63) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:29 INFO Executor: Running task 36.0 in stage 13.0 (TID 63) -26/04/01 06:31:29 INFO TaskSetManager: Starting task 37.0 in stage 13.0 (TID 64) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:29 INFO TaskSetManager: Finished task 33.0 in stage 13.0 (TID 60) in 903 ms on 10.0.0.133 (executor driver) (33/64) -26/04/01 06:31:29 INFO Executor: Running task 37.0 in stage 13.0 (TID 64) -26/04/01 06:31:29 INFO TaskSetManager: Finished task 32.0 in stage 13.0 (TID 59) in 905 ms on 10.0.0.133 (executor driver) (34/64) -26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:29 INFO Executor: Finished task 34.0 in stage 13.0 (TID 61). 6675 bytes result sent to driver -26/04/01 06:31:29 INFO TaskSetManager: Starting task 38.0 in stage 13.0 (TID 65) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:29 INFO Executor: Running task 38.0 in stage 13.0 (TID 65) -26/04/01 06:31:29 INFO TaskSetManager: Finished task 34.0 in stage 13.0 (TID 61) in 905 ms on 10.0.0.133 (executor driver) (35/64) -26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:29 INFO Executor: Finished task 35.0 in stage 13.0 (TID 62). 6675 bytes result sent to driver -26/04/01 06:31:29 INFO TaskSetManager: Starting task 39.0 in stage 13.0 (TID 66) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:29 INFO TaskSetManager: Finished task 35.0 in stage 13.0 (TID 62) in 905 ms on 10.0.0.133 (executor driver) (36/64) -26/04/01 06:31:29 INFO Executor: Running task 39.0 in stage 13.0 (TID 66) -26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:30 INFO Executor: Finished task 36.0 in stage 13.0 (TID 63). 6675 bytes result sent to driver -26/04/01 06:31:30 INFO TaskSetManager: Starting task 40.0 in stage 13.0 (TID 67) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:31:30 INFO TaskSetManager: Finished task 36.0 in stage 13.0 (TID 63) in 900 ms on 10.0.0.133 (executor driver) (37/64) -26/04/01 06:31:30 INFO Executor: Running task 40.0 in stage 13.0 (TID 67) -26/04/01 06:31:30 INFO Executor: Finished task 37.0 in stage 13.0 (TID 64). 6675 bytes result sent to driver -26/04/01 06:31:30 INFO TaskSetManager: Starting task 41.0 in stage 13.0 (TID 68) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:30 INFO TaskSetManager: Finished task 37.0 in stage 13.0 (TID 64) in 901 ms on 10.0.0.133 (executor driver) (38/64) -26/04/01 06:31:30 INFO Executor: Running task 41.0 in stage 13.0 (TID 68) -26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:30 INFO Executor: Finished task 38.0 in stage 13.0 (TID 65). 6675 bytes result sent to driver -26/04/01 06:31:30 INFO TaskSetManager: Starting task 42.0 in stage 13.0 (TID 69) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:30 INFO TaskSetManager: Finished task 38.0 in stage 13.0 (TID 65) in 906 ms on 10.0.0.133 (executor driver) (39/64) -26/04/01 06:31:30 INFO Executor: Running task 42.0 in stage 13.0 (TID 69) -26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:30 INFO Executor: Finished task 39.0 in stage 13.0 (TID 66). 6675 bytes result sent to driver -26/04/01 06:31:30 INFO TaskSetManager: Starting task 43.0 in stage 13.0 (TID 70) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:31:30 INFO TaskSetManager: Finished task 39.0 in stage 13.0 (TID 66) in 907 ms on 10.0.0.133 (executor driver) (40/64) -26/04/01 06:31:30 INFO Executor: Running task 43.0 in stage 13.0 (TID 70) -26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:31 INFO Executor: Finished task 40.0 in stage 13.0 (TID 67). 6675 bytes result sent to driver -26/04/01 06:31:31 INFO TaskSetManager: Starting task 44.0 in stage 13.0 (TID 71) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:31:31 INFO Executor: Finished task 41.0 in stage 13.0 (TID 68). 6675 bytes result sent to driver -26/04/01 06:31:31 INFO TaskSetManager: Finished task 40.0 in stage 13.0 (TID 67) in 906 ms on 10.0.0.133 (executor driver) (41/64) -26/04/01 06:31:31 INFO Executor: Running task 44.0 in stage 13.0 (TID 71) -26/04/01 06:31:31 INFO TaskSetManager: Starting task 45.0 in stage 13.0 (TID 72) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:31 INFO TaskSetManager: Finished task 41.0 in stage 13.0 (TID 68) in 905 ms on 10.0.0.133 (executor driver) (42/64) -26/04/01 06:31:31 INFO Executor: Running task 45.0 in stage 13.0 (TID 72) -26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:31 INFO Executor: Finished task 42.0 in stage 13.0 (TID 69). 6675 bytes result sent to driver -26/04/01 06:31:31 INFO TaskSetManager: Starting task 46.0 in stage 13.0 (TID 73) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:31 INFO TaskSetManager: Finished task 42.0 in stage 13.0 (TID 69) in 905 ms on 10.0.0.133 (executor driver) (43/64) -26/04/01 06:31:31 INFO Executor: Running task 46.0 in stage 13.0 (TID 73) -26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:31 INFO Executor: Finished task 43.0 in stage 13.0 (TID 70). 6675 bytes result sent to driver -26/04/01 06:31:31 INFO TaskSetManager: Starting task 47.0 in stage 13.0 (TID 74) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:31:31 INFO TaskSetManager: Finished task 43.0 in stage 13.0 (TID 70) in 908 ms on 10.0.0.133 (executor driver) (44/64) -26/04/01 06:31:31 INFO Executor: Running task 47.0 in stage 13.0 (TID 74) -26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:32 INFO Executor: Finished task 45.0 in stage 13.0 (TID 72). 6675 bytes result sent to driver -26/04/01 06:31:32 INFO TaskSetManager: Starting task 48.0 in stage 13.0 (TID 75) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:32 INFO TaskSetManager: Finished task 45.0 in stage 13.0 (TID 72) in 906 ms on 10.0.0.133 (executor driver) (45/64) -26/04/01 06:31:32 INFO Executor: Running task 48.0 in stage 13.0 (TID 75) -26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:32 INFO Executor: Finished task 46.0 in stage 13.0 (TID 73). 6675 bytes result sent to driver -26/04/01 06:31:32 INFO Executor: Finished task 44.0 in stage 13.0 (TID 71). 6675 bytes result sent to driver -26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:32 INFO TaskSetManager: Starting task 49.0 in stage 13.0 (TID 76) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:32 INFO TaskSetManager: Finished task 46.0 in stage 13.0 (TID 73) in 903 ms on 10.0.0.133 (executor driver) (46/64) -26/04/01 06:31:32 INFO Executor: Running task 49.0 in stage 13.0 (TID 76) -26/04/01 06:31:32 INFO TaskSetManager: Starting task 50.0 in stage 13.0 (TID 77) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:32 INFO TaskSetManager: Finished task 44.0 in stage 13.0 (TID 71) in 909 ms on 10.0.0.133 (executor driver) (47/64) -26/04/01 06:31:32 INFO Executor: Running task 50.0 in stage 13.0 (TID 77) -26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:32 INFO Executor: Finished task 47.0 in stage 13.0 (TID 74). 6675 bytes result sent to driver -26/04/01 06:31:32 INFO TaskSetManager: Starting task 51.0 in stage 13.0 (TID 78) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:32 INFO TaskSetManager: Finished task 47.0 in stage 13.0 (TID 74) in 906 ms on 10.0.0.133 (executor driver) (48/64) -26/04/01 06:31:32 INFO Executor: Running task 51.0 in stage 13.0 (TID 78) -26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:33 INFO Executor: Finished task 48.0 in stage 13.0 (TID 75). 6675 bytes result sent to driver -26/04/01 06:31:33 INFO TaskSetManager: Starting task 52.0 in stage 13.0 (TID 79) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:33 INFO Executor: Running task 52.0 in stage 13.0 (TID 79) -26/04/01 06:31:33 INFO TaskSetManager: Finished task 48.0 in stage 13.0 (TID 75) in 914 ms on 10.0.0.133 (executor driver) (49/64) -26/04/01 06:31:33 INFO Executor: Finished task 49.0 in stage 13.0 (TID 76). 6675 bytes result sent to driver -26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:33 INFO TaskSetManager: Starting task 53.0 in stage 13.0 (TID 80) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:33 INFO TaskSetManager: Finished task 49.0 in stage 13.0 (TID 76) in 912 ms on 10.0.0.133 (executor driver) (50/64) -26/04/01 06:31:33 INFO Executor: Finished task 50.0 in stage 13.0 (TID 77). 6675 bytes result sent to driver -26/04/01 06:31:33 INFO Executor: Running task 53.0 in stage 13.0 (TID 80) -26/04/01 06:31:33 INFO TaskSetManager: Starting task 54.0 in stage 13.0 (TID 81) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:31:33 INFO TaskSetManager: Finished task 50.0 in stage 13.0 (TID 77) in 913 ms on 10.0.0.133 (executor driver) (51/64) -26/04/01 06:31:33 INFO Executor: Running task 54.0 in stage 13.0 (TID 81) -26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:33 INFO Executor: Finished task 51.0 in stage 13.0 (TID 78). 6675 bytes result sent to driver -26/04/01 06:31:33 INFO TaskSetManager: Starting task 55.0 in stage 13.0 (TID 82) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:33 INFO TaskSetManager: Finished task 51.0 in stage 13.0 (TID 78) in 915 ms on 10.0.0.133 (executor driver) (52/64) -26/04/01 06:31:33 INFO Executor: Running task 55.0 in stage 13.0 (TID 82) -26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:34 INFO Executor: Finished task 52.0 in stage 13.0 (TID 79). 6675 bytes result sent to driver -26/04/01 06:31:34 INFO TaskSetManager: Starting task 56.0 in stage 13.0 (TID 83) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:31:34 INFO TaskSetManager: Finished task 52.0 in stage 13.0 (TID 79) in 909 ms on 10.0.0.133 (executor driver) (53/64) -26/04/01 06:31:34 INFO Executor: Running task 56.0 in stage 13.0 (TID 83) -26/04/01 06:31:34 INFO Executor: Finished task 54.0 in stage 13.0 (TID 81). 6675 bytes result sent to driver -26/04/01 06:31:34 INFO TaskSetManager: Starting task 57.0 in stage 13.0 (TID 84) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:34 INFO Executor: Finished task 53.0 in stage 13.0 (TID 80). 6675 bytes result sent to driver -26/04/01 06:31:34 INFO TaskSetManager: Finished task 54.0 in stage 13.0 (TID 81) in 907 ms on 10.0.0.133 (executor driver) (54/64) -26/04/01 06:31:34 INFO Executor: Running task 57.0 in stage 13.0 (TID 84) -26/04/01 06:31:34 INFO TaskSetManager: Starting task 58.0 in stage 13.0 (TID 85) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:34 INFO TaskSetManager: Finished task 53.0 in stage 13.0 (TID 80) in 908 ms on 10.0.0.133 (executor driver) (55/64) -26/04/01 06:31:34 INFO Executor: Running task 58.0 in stage 13.0 (TID 85) -26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:34 INFO Executor: Finished task 55.0 in stage 13.0 (TID 82). 6675 bytes result sent to driver -26/04/01 06:31:34 INFO TaskSetManager: Starting task 59.0 in stage 13.0 (TID 86) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:34 INFO TaskSetManager: Finished task 55.0 in stage 13.0 (TID 82) in 905 ms on 10.0.0.133 (executor driver) (56/64) -26/04/01 06:31:34 INFO Executor: Running task 59.0 in stage 13.0 (TID 86) -26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:35 INFO Executor: Finished task 56.0 in stage 13.0 (TID 83). 6675 bytes result sent to driver -26/04/01 06:31:35 INFO TaskSetManager: Starting task 60.0 in stage 13.0 (TID 87) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:35 INFO Executor: Running task 60.0 in stage 13.0 (TID 87) -26/04/01 06:31:35 INFO TaskSetManager: Finished task 56.0 in stage 13.0 (TID 83) in 907 ms on 10.0.0.133 (executor driver) (57/64) -26/04/01 06:31:35 INFO Executor: Finished task 58.0 in stage 13.0 (TID 85). 6675 bytes result sent to driver -26/04/01 06:31:35 INFO TaskSetManager: Starting task 61.0 in stage 13.0 (TID 88) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:35 INFO TaskSetManager: Finished task 58.0 in stage 13.0 (TID 85) in 906 ms on 10.0.0.133 (executor driver) (58/64) -26/04/01 06:31:35 INFO Executor: Running task 61.0 in stage 13.0 (TID 88) -26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:35 INFO Executor: Finished task 57.0 in stage 13.0 (TID 84). 6675 bytes result sent to driver -26/04/01 06:31:35 INFO TaskSetManager: Starting task 62.0 in stage 13.0 (TID 89) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:31:35 INFO Executor: Running task 62.0 in stage 13.0 (TID 89) -26/04/01 06:31:35 INFO TaskSetManager: Finished task 57.0 in stage 13.0 (TID 84) in 908 ms on 10.0.0.133 (executor driver) (59/64) -26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:35 INFO Executor: Finished task 59.0 in stage 13.0 (TID 86). 6675 bytes result sent to driver -26/04/01 06:31:35 INFO TaskSetManager: Starting task 63.0 in stage 13.0 (TID 90) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:31:35 INFO TaskSetManager: Finished task 59.0 in stage 13.0 (TID 86) in 907 ms on 10.0.0.133 (executor driver) (60/64) -26/04/01 06:31:35 INFO Executor: Running task 63.0 in stage 13.0 (TID 90) -26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:36 INFO Executor: Finished task 62.0 in stage 13.0 (TID 89). 6675 bytes result sent to driver -26/04/01 06:31:36 INFO Executor: Finished task 61.0 in stage 13.0 (TID 88). 6675 bytes result sent to driver -26/04/01 06:31:36 INFO TaskSetManager: Starting task 0.0 in stage 14.0 (TID 91) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:36 INFO Executor: Running task 0.0 in stage 14.0 (TID 91) -26/04/01 06:31:36 INFO TaskSetManager: Starting task 1.0 in stage 14.0 (TID 92) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:31:36 INFO Executor: Running task 1.0 in stage 14.0 (TID 92) -26/04/01 06:31:36 INFO TaskSetManager: Finished task 62.0 in stage 13.0 (TID 89) in 901 ms on 10.0.0.133 (executor driver) (61/64) -26/04/01 06:31:36 INFO TaskSetManager: Finished task 61.0 in stage 13.0 (TID 88) in 902 ms on 10.0.0.133 (executor driver) (62/64) -26/04/01 06:31:36 INFO Executor: Finished task 60.0 in stage 13.0 (TID 87). 6675 bytes result sent to driver -26/04/01 06:31:36 INFO TaskSetManager: Starting task 2.0 in stage 14.0 (TID 93) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:31:36 INFO TaskSetManager: Finished task 60.0 in stage 13.0 (TID 87) in 903 ms on 10.0.0.133 (executor driver) (63/64) -26/04/01 06:31:36 INFO Executor: Running task 2.0 in stage 14.0 (TID 93) -26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:36 INFO Executor: Finished task 63.0 in stage 13.0 (TID 90). 6675 bytes result sent to driver -26/04/01 06:31:36 INFO TaskSetManager: Starting task 3.0 in stage 14.0 (TID 94) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:31:36 INFO Executor: Running task 3.0 in stage 14.0 (TID 94) -26/04/01 06:31:36 INFO TaskSetManager: Finished task 63.0 in stage 13.0 (TID 90) in 906 ms on 10.0.0.133 (executor driver) (64/64) -26/04/01 06:31:36 INFO TaskSchedulerImpl: Removed TaskSet 13.0, whose tasks have all completed, from pool -26/04/01 06:31:36 INFO DAGScheduler: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 20.348 s -26/04/01 06:31:36 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:31:36 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 16, ShuffleMapStage 14) -26/04/01 06:31:36 INFO DAGScheduler: waiting: Set() -26/04/01 06:31:36 INFO DAGScheduler: failed: Set() -26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:36 INFO ShufflePartitionsUtil: For shuffle(1, 2), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 06:31:36 INFO DAGScheduler: Registering RDD 44 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 5 -26/04/01 06:31:36 INFO DAGScheduler: Got map stage job 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 6 output partitions -26/04/01 06:31:36 INFO DAGScheduler: Final stage: ShuffleMapStage 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:31:36 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 17, ShuffleMapStage 18) -26/04/01 06:31:36 INFO DAGScheduler: Missing parents: List() -26/04/01 06:31:36 INFO DAGScheduler: Submitting ShuffleMapStage 19 (MapPartitionsRDD[44] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:31:36 INFO MemoryStore: Block broadcast_24 stored as values in memory (estimated size 22.7 KiB, free 12.6 GiB) -26/04/01 06:31:36 INFO MemoryStore: Block broadcast_24_piece0 stored as bytes in memory (estimated size 9.5 KiB, free 12.6 GiB) -26/04/01 06:31:36 INFO BlockManagerInfo: Added broadcast_24_piece0 in memory on 10.0.0.133:58442 (size: 9.5 KiB, free: 12.6 GiB) -26/04/01 06:31:36 INFO SparkContext: Created broadcast 24 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:31:36 INFO DAGScheduler: Submitting 6 missing tasks from ShuffleMapStage 19 (MapPartitionsRDD[44] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5)) -26/04/01 06:31:36 INFO TaskSchedulerImpl: Adding task set 19.0 with 6 tasks resource profile 0 -26/04/01 06:31:40 INFO Executor: Finished task 1.0 in stage 14.0 (TID 92). 6563 bytes result sent to driver -26/04/01 06:31:40 INFO Executor: Finished task 0.0 in stage 14.0 (TID 91). 6563 bytes result sent to driver -26/04/01 06:31:40 INFO TaskSetManager: Starting task 4.0 in stage 14.0 (TID 95) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:31:40 INFO Executor: Finished task 2.0 in stage 14.0 (TID 93). 6563 bytes result sent to driver -26/04/01 06:31:40 INFO Executor: Running task 4.0 in stage 14.0 (TID 95) -26/04/01 06:31:40 INFO TaskSetManager: Starting task 5.0 in stage 14.0 (TID 96) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:31:40 INFO TaskSetManager: Finished task 1.0 in stage 14.0 (TID 92) in 4141 ms on 10.0.0.133 (executor driver) (1/208) -26/04/01 06:31:40 INFO Executor: Running task 5.0 in stage 14.0 (TID 96) -26/04/01 06:31:40 INFO TaskSetManager: Starting task 6.0 in stage 14.0 (TID 97) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:40 INFO TaskSetManager: Finished task 2.0 in stage 14.0 (TID 93) in 4141 ms on 10.0.0.133 (executor driver) (2/208) -26/04/01 06:31:40 INFO Executor: Running task 6.0 in stage 14.0 (TID 97) -26/04/01 06:31:40 INFO TaskSetManager: Finished task 0.0 in stage 14.0 (TID 91) in 4143 ms on 10.0.0.133 (executor driver) (3/208) -26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:40 INFO Executor: Finished task 3.0 in stage 14.0 (TID 94). 6563 bytes result sent to driver -26/04/01 06:31:40 INFO TaskSetManager: Starting task 7.0 in stage 14.0 (TID 98) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:31:40 INFO TaskSetManager: Finished task 3.0 in stage 14.0 (TID 94) in 4144 ms on 10.0.0.133 (executor driver) (4/208) -26/04/01 06:31:40 INFO Executor: Running task 7.0 in stage 14.0 (TID 98) -26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:44 INFO Executor: Finished task 4.0 in stage 14.0 (TID 95). 6563 bytes result sent to driver -26/04/01 06:31:44 INFO TaskSetManager: Starting task 8.0 in stage 14.0 (TID 99) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:31:44 INFO Executor: Running task 8.0 in stage 14.0 (TID 99) -26/04/01 06:31:44 INFO TaskSetManager: Finished task 4.0 in stage 14.0 (TID 95) in 4138 ms on 10.0.0.133 (executor driver) (5/208) -26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:44 INFO Executor: Finished task 6.0 in stage 14.0 (TID 97). 6563 bytes result sent to driver -26/04/01 06:31:44 INFO TaskSetManager: Starting task 9.0 in stage 14.0 (TID 100) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:31:44 INFO Executor: Running task 9.0 in stage 14.0 (TID 100) -26/04/01 06:31:44 INFO TaskSetManager: Finished task 6.0 in stage 14.0 (TID 97) in 4140 ms on 10.0.0.133 (executor driver) (6/208) -26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:44 INFO Executor: Finished task 5.0 in stage 14.0 (TID 96). 6563 bytes result sent to driver -26/04/01 06:31:44 INFO TaskSetManager: Starting task 10.0 in stage 14.0 (TID 101) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:31:44 INFO TaskSetManager: Finished task 5.0 in stage 14.0 (TID 96) in 4143 ms on 10.0.0.133 (executor driver) (7/208) -26/04/01 06:31:44 INFO Executor: Running task 10.0 in stage 14.0 (TID 101) -26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:44 INFO Executor: Finished task 7.0 in stage 14.0 (TID 98). 6563 bytes result sent to driver -26/04/01 06:31:44 INFO TaskSetManager: Starting task 11.0 in stage 14.0 (TID 102) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:31:44 INFO TaskSetManager: Finished task 7.0 in stage 14.0 (TID 98) in 4140 ms on 10.0.0.133 (executor driver) (8/208) -26/04/01 06:31:44 INFO Executor: Running task 11.0 in stage 14.0 (TID 102) -26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:48 INFO Executor: Finished task 9.0 in stage 14.0 (TID 100). 6563 bytes result sent to driver -26/04/01 06:31:48 INFO TaskSetManager: Starting task 12.0 in stage 14.0 (TID 103) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:31:48 INFO Executor: Running task 12.0 in stage 14.0 (TID 103) -26/04/01 06:31:48 INFO TaskSetManager: Finished task 9.0 in stage 14.0 (TID 100) in 4138 ms on 10.0.0.133 (executor driver) (9/208) -26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:48 INFO Executor: Finished task 8.0 in stage 14.0 (TID 99). 6563 bytes result sent to driver -26/04/01 06:31:48 INFO TaskSetManager: Starting task 13.0 in stage 14.0 (TID 104) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:31:48 INFO TaskSetManager: Finished task 8.0 in stage 14.0 (TID 99) in 4143 ms on 10.0.0.133 (executor driver) (10/208) -26/04/01 06:31:48 INFO Executor: Running task 13.0 in stage 14.0 (TID 104) -26/04/01 06:31:48 INFO Executor: Finished task 10.0 in stage 14.0 (TID 101). 6563 bytes result sent to driver -26/04/01 06:31:48 INFO TaskSetManager: Starting task 14.0 in stage 14.0 (TID 105) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:31:48 INFO Executor: Running task 14.0 in stage 14.0 (TID 105) -26/04/01 06:31:48 INFO TaskSetManager: Finished task 10.0 in stage 14.0 (TID 101) in 4139 ms on 10.0.0.133 (executor driver) (11/208) -26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:48 INFO Executor: Finished task 11.0 in stage 14.0 (TID 102). 6563 bytes result sent to driver -26/04/01 06:31:48 INFO TaskSetManager: Starting task 15.0 in stage 14.0 (TID 106) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:31:48 INFO Executor: Running task 15.0 in stage 14.0 (TID 106) -26/04/01 06:31:48 INFO TaskSetManager: Finished task 11.0 in stage 14.0 (TID 102) in 4152 ms on 10.0.0.133 (executor driver) (12/208) -26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:52 INFO Executor: Finished task 12.0 in stage 14.0 (TID 103). 6563 bytes result sent to driver -26/04/01 06:31:52 INFO Executor: Finished task 13.0 in stage 14.0 (TID 104). 6563 bytes result sent to driver -26/04/01 06:31:52 INFO TaskSetManager: Starting task 16.0 in stage 14.0 (TID 107) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:31:52 INFO TaskSetManager: Starting task 17.0 in stage 14.0 (TID 108) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:31:52 INFO Executor: Running task 16.0 in stage 14.0 (TID 107) -26/04/01 06:31:52 INFO TaskSetManager: Finished task 12.0 in stage 14.0 (TID 103) in 4152 ms on 10.0.0.133 (executor driver) (13/208) -26/04/01 06:31:52 INFO Executor: Running task 17.0 in stage 14.0 (TID 108) -26/04/01 06:31:52 INFO TaskSetManager: Finished task 13.0 in stage 14.0 (TID 104) in 4149 ms on 10.0.0.133 (executor driver) (14/208) -26/04/01 06:31:52 INFO Executor: Finished task 14.0 in stage 14.0 (TID 105). 6563 bytes result sent to driver -26/04/01 06:31:52 INFO TaskSetManager: Starting task 18.0 in stage 14.0 (TID 109) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:31:52 INFO TaskSetManager: Finished task 14.0 in stage 14.0 (TID 105) in 4149 ms on 10.0.0.133 (executor driver) (15/208) -26/04/01 06:31:52 INFO Executor: Running task 18.0 in stage 14.0 (TID 109) -26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:52 INFO Executor: Finished task 15.0 in stage 14.0 (TID 106). 6563 bytes result sent to driver -26/04/01 06:31:52 INFO TaskSetManager: Starting task 19.0 in stage 14.0 (TID 110) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:31:52 INFO TaskSetManager: Finished task 15.0 in stage 14.0 (TID 106) in 4147 ms on 10.0.0.133 (executor driver) (16/208) -26/04/01 06:31:52 INFO Executor: Running task 19.0 in stage 14.0 (TID 110) -26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:56 INFO Executor: Finished task 17.0 in stage 14.0 (TID 108). 6563 bytes result sent to driver -26/04/01 06:31:56 INFO Executor: Finished task 18.0 in stage 14.0 (TID 109). 6563 bytes result sent to driver -26/04/01 06:31:56 INFO TaskSetManager: Starting task 20.0 in stage 14.0 (TID 111) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:31:56 INFO Executor: Finished task 16.0 in stage 14.0 (TID 107). 6563 bytes result sent to driver -26/04/01 06:31:56 INFO Executor: Running task 20.0 in stage 14.0 (TID 111) -26/04/01 06:31:56 INFO TaskSetManager: Starting task 21.0 in stage 14.0 (TID 112) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:31:56 INFO TaskSetManager: Finished task 17.0 in stage 14.0 (TID 108) in 4143 ms on 10.0.0.133 (executor driver) (17/208) -26/04/01 06:31:56 INFO Executor: Running task 21.0 in stage 14.0 (TID 112) -26/04/01 06:31:56 INFO TaskSetManager: Finished task 18.0 in stage 14.0 (TID 109) in 4142 ms on 10.0.0.133 (executor driver) (18/208) -26/04/01 06:31:56 INFO TaskSetManager: Starting task 22.0 in stage 14.0 (TID 113) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:31:56 INFO TaskSetManager: Finished task 16.0 in stage 14.0 (TID 107) in 4143 ms on 10.0.0.133 (executor driver) (19/208) -26/04/01 06:31:56 INFO Executor: Running task 22.0 in stage 14.0 (TID 113) -26/04/01 06:31:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:56 INFO Executor: Finished task 19.0 in stage 14.0 (TID 110). 6563 bytes result sent to driver -26/04/01 06:31:56 INFO TaskSetManager: Starting task 23.0 in stage 14.0 (TID 114) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:31:56 INFO TaskSetManager: Finished task 19.0 in stage 14.0 (TID 110) in 4136 ms on 10.0.0.133 (executor driver) (20/208) -26/04/01 06:31:56 INFO Executor: Running task 23.0 in stage 14.0 (TID 114) -26/04/01 06:31:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:31:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:01 INFO Executor: Finished task 21.0 in stage 14.0 (TID 112). 6563 bytes result sent to driver -26/04/01 06:32:01 INFO TaskSetManager: Starting task 24.0 in stage 14.0 (TID 115) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:32:01 INFO TaskSetManager: Finished task 21.0 in stage 14.0 (TID 112) in 4125 ms on 10.0.0.133 (executor driver) (21/208) -26/04/01 06:32:01 INFO Executor: Running task 24.0 in stage 14.0 (TID 115) -26/04/01 06:32:01 INFO Executor: Finished task 20.0 in stage 14.0 (TID 111). 6563 bytes result sent to driver -26/04/01 06:32:01 INFO TaskSetManager: Starting task 25.0 in stage 14.0 (TID 116) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:01 INFO TaskSetManager: Finished task 20.0 in stage 14.0 (TID 111) in 4126 ms on 10.0.0.133 (executor driver) (22/208) -26/04/01 06:32:01 INFO Executor: Running task 25.0 in stage 14.0 (TID 116) -26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:01 INFO Executor: Finished task 22.0 in stage 14.0 (TID 113). 6563 bytes result sent to driver -26/04/01 06:32:01 INFO TaskSetManager: Starting task 26.0 in stage 14.0 (TID 117) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:01 INFO TaskSetManager: Finished task 22.0 in stage 14.0 (TID 113) in 4129 ms on 10.0.0.133 (executor driver) (23/208) -26/04/01 06:32:01 INFO Executor: Running task 26.0 in stage 14.0 (TID 117) -26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:01 INFO Executor: Finished task 23.0 in stage 14.0 (TID 114). 6563 bytes result sent to driver -26/04/01 06:32:01 INFO TaskSetManager: Starting task 27.0 in stage 14.0 (TID 118) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:01 INFO TaskSetManager: Finished task 23.0 in stage 14.0 (TID 114) in 4123 ms on 10.0.0.133 (executor driver) (24/208) -26/04/01 06:32:01 INFO Executor: Running task 27.0 in stage 14.0 (TID 118) -26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:01 INFO BlockManagerInfo: Removed broadcast_17_piece0 on 10.0.0.133:58442 in memory (size: 8.1 KiB, free: 12.6 GiB) -26/04/01 06:32:05 INFO Executor: Finished task 24.0 in stage 14.0 (TID 115). 6606 bytes result sent to driver -26/04/01 06:32:05 INFO TaskSetManager: Starting task 28.0 in stage 14.0 (TID 119) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:05 INFO TaskSetManager: Finished task 24.0 in stage 14.0 (TID 115) in 4070 ms on 10.0.0.133 (executor driver) (25/208) -26/04/01 06:32:05 INFO Executor: Running task 28.0 in stage 14.0 (TID 119) -26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:05 INFO Executor: Finished task 25.0 in stage 14.0 (TID 116). 6606 bytes result sent to driver -26/04/01 06:32:05 INFO TaskSetManager: Starting task 29.0 in stage 14.0 (TID 120) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:05 INFO Executor: Running task 29.0 in stage 14.0 (TID 120) -26/04/01 06:32:05 INFO TaskSetManager: Finished task 25.0 in stage 14.0 (TID 116) in 4078 ms on 10.0.0.133 (executor driver) (26/208) -26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:05 INFO Executor: Finished task 26.0 in stage 14.0 (TID 117). 6606 bytes result sent to driver -26/04/01 06:32:05 INFO TaskSetManager: Starting task 30.0 in stage 14.0 (TID 121) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:32:05 INFO TaskSetManager: Finished task 26.0 in stage 14.0 (TID 117) in 4079 ms on 10.0.0.133 (executor driver) (27/208) -26/04/01 06:32:05 INFO Executor: Running task 30.0 in stage 14.0 (TID 121) -26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:05 INFO Executor: Finished task 27.0 in stage 14.0 (TID 118). 6606 bytes result sent to driver -26/04/01 06:32:05 INFO TaskSetManager: Starting task 31.0 in stage 14.0 (TID 122) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:05 INFO TaskSetManager: Finished task 27.0 in stage 14.0 (TID 118) in 4078 ms on 10.0.0.133 (executor driver) (28/208) -26/04/01 06:32:05 INFO Executor: Running task 31.0 in stage 14.0 (TID 122) -26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:09 INFO Executor: Finished task 28.0 in stage 14.0 (TID 119). 6563 bytes result sent to driver -26/04/01 06:32:09 INFO TaskSetManager: Starting task 32.0 in stage 14.0 (TID 123) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:09 INFO TaskSetManager: Finished task 28.0 in stage 14.0 (TID 119) in 4079 ms on 10.0.0.133 (executor driver) (29/208) -26/04/01 06:32:09 INFO Executor: Running task 32.0 in stage 14.0 (TID 123) -26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:09 INFO Executor: Finished task 29.0 in stage 14.0 (TID 120). 6563 bytes result sent to driver -26/04/01 06:32:09 INFO TaskSetManager: Starting task 33.0 in stage 14.0 (TID 124) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:09 INFO TaskSetManager: Finished task 29.0 in stage 14.0 (TID 120) in 4085 ms on 10.0.0.133 (executor driver) (30/208) -26/04/01 06:32:09 INFO Executor: Running task 33.0 in stage 14.0 (TID 124) -26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:09 INFO Executor: Finished task 30.0 in stage 14.0 (TID 121). 6563 bytes result sent to driver -26/04/01 06:32:09 INFO TaskSetManager: Starting task 34.0 in stage 14.0 (TID 125) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:09 INFO TaskSetManager: Finished task 30.0 in stage 14.0 (TID 121) in 4083 ms on 10.0.0.133 (executor driver) (31/208) -26/04/01 06:32:09 INFO Executor: Running task 34.0 in stage 14.0 (TID 125) -26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:09 INFO Executor: Finished task 31.0 in stage 14.0 (TID 122). 6563 bytes result sent to driver -26/04/01 06:32:09 INFO TaskSetManager: Starting task 35.0 in stage 14.0 (TID 126) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:09 INFO TaskSetManager: Finished task 31.0 in stage 14.0 (TID 122) in 4089 ms on 10.0.0.133 (executor driver) (32/208) -26/04/01 06:32:09 INFO Executor: Running task 35.0 in stage 14.0 (TID 126) -26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:13 INFO Executor: Finished task 32.0 in stage 14.0 (TID 123). 6563 bytes result sent to driver -26/04/01 06:32:13 INFO Executor: Finished task 33.0 in stage 14.0 (TID 124). 6563 bytes result sent to driver -26/04/01 06:32:13 INFO TaskSetManager: Starting task 36.0 in stage 14.0 (TID 127) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:32:13 INFO TaskSetManager: Finished task 32.0 in stage 14.0 (TID 123) in 4077 ms on 10.0.0.133 (executor driver) (33/208) -26/04/01 06:32:13 INFO Executor: Running task 36.0 in stage 14.0 (TID 127) -26/04/01 06:32:13 INFO TaskSetManager: Starting task 37.0 in stage 14.0 (TID 128) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:13 INFO TaskSetManager: Finished task 33.0 in stage 14.0 (TID 124) in 4063 ms on 10.0.0.133 (executor driver) (34/208) -26/04/01 06:32:13 INFO Executor: Running task 37.0 in stage 14.0 (TID 128) -26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:13 INFO Executor: Finished task 34.0 in stage 14.0 (TID 125). 6563 bytes result sent to driver -26/04/01 06:32:13 INFO TaskSetManager: Starting task 38.0 in stage 14.0 (TID 129) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:13 INFO Executor: Running task 38.0 in stage 14.0 (TID 129) -26/04/01 06:32:13 INFO TaskSetManager: Finished task 34.0 in stage 14.0 (TID 125) in 4075 ms on 10.0.0.133 (executor driver) (35/208) -26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:13 INFO Executor: Finished task 35.0 in stage 14.0 (TID 126). 6563 bytes result sent to driver -26/04/01 06:32:13 INFO TaskSetManager: Starting task 39.0 in stage 14.0 (TID 130) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:13 INFO Executor: Running task 39.0 in stage 14.0 (TID 130) -26/04/01 06:32:13 INFO TaskSetManager: Finished task 35.0 in stage 14.0 (TID 126) in 4072 ms on 10.0.0.133 (executor driver) (36/208) -26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:17 INFO Executor: Finished task 36.0 in stage 14.0 (TID 127). 6563 bytes result sent to driver -26/04/01 06:32:17 INFO TaskSetManager: Starting task 40.0 in stage 14.0 (TID 131) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:17 INFO Executor: Finished task 37.0 in stage 14.0 (TID 128). 6563 bytes result sent to driver -26/04/01 06:32:17 INFO Executor: Running task 40.0 in stage 14.0 (TID 131) -26/04/01 06:32:17 INFO TaskSetManager: Starting task 41.0 in stage 14.0 (TID 132) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:17 INFO TaskSetManager: Finished task 36.0 in stage 14.0 (TID 127) in 4131 ms on 10.0.0.133 (executor driver) (37/208) -26/04/01 06:32:17 INFO TaskSetManager: Finished task 37.0 in stage 14.0 (TID 128) in 4130 ms on 10.0.0.133 (executor driver) (38/208) -26/04/01 06:32:17 INFO Executor: Running task 41.0 in stage 14.0 (TID 132) -26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:17 INFO Executor: Finished task 38.0 in stage 14.0 (TID 129). 6563 bytes result sent to driver -26/04/01 06:32:17 INFO TaskSetManager: Starting task 42.0 in stage 14.0 (TID 133) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:32:17 INFO Executor: Running task 42.0 in stage 14.0 (TID 133) -26/04/01 06:32:17 INFO TaskSetManager: Finished task 38.0 in stage 14.0 (TID 129) in 4119 ms on 10.0.0.133 (executor driver) (39/208) -26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:17 INFO Executor: Finished task 39.0 in stage 14.0 (TID 130). 6563 bytes result sent to driver -26/04/01 06:32:17 INFO TaskSetManager: Starting task 43.0 in stage 14.0 (TID 134) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:32:17 INFO TaskSetManager: Finished task 39.0 in stage 14.0 (TID 130) in 4134 ms on 10.0.0.133 (executor driver) (40/208) -26/04/01 06:32:17 INFO Executor: Running task 43.0 in stage 14.0 (TID 134) -26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:21 INFO Executor: Finished task 42.0 in stage 14.0 (TID 133). 6563 bytes result sent to driver -26/04/01 06:32:21 INFO TaskSetManager: Starting task 44.0 in stage 14.0 (TID 135) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:21 INFO TaskSetManager: Finished task 42.0 in stage 14.0 (TID 133) in 4075 ms on 10.0.0.133 (executor driver) (41/208) -26/04/01 06:32:21 INFO Executor: Running task 44.0 in stage 14.0 (TID 135) -26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:21 INFO Executor: Finished task 43.0 in stage 14.0 (TID 134). 6563 bytes result sent to driver -26/04/01 06:32:21 INFO TaskSetManager: Starting task 45.0 in stage 14.0 (TID 136) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:21 INFO TaskSetManager: Finished task 43.0 in stage 14.0 (TID 134) in 4075 ms on 10.0.0.133 (executor driver) (42/208) -26/04/01 06:32:21 INFO Executor: Running task 45.0 in stage 14.0 (TID 136) -26/04/01 06:32:21 INFO Executor: Finished task 40.0 in stage 14.0 (TID 131). 6563 bytes result sent to driver -26/04/01 06:32:21 INFO TaskSetManager: Starting task 46.0 in stage 14.0 (TID 137) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:21 INFO Executor: Running task 46.0 in stage 14.0 (TID 137) -26/04/01 06:32:21 INFO TaskSetManager: Finished task 40.0 in stage 14.0 (TID 131) in 4136 ms on 10.0.0.133 (executor driver) (43/208) -26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:21 INFO Executor: Finished task 41.0 in stage 14.0 (TID 132). 6563 bytes result sent to driver -26/04/01 06:32:21 INFO TaskSetManager: Starting task 47.0 in stage 14.0 (TID 138) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:21 INFO TaskSetManager: Finished task 41.0 in stage 14.0 (TID 132) in 4153 ms on 10.0.0.133 (executor driver) (44/208) -26/04/01 06:32:21 INFO Executor: Running task 47.0 in stage 14.0 (TID 138) -26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:25 INFO Executor: Finished task 45.0 in stage 14.0 (TID 136). 6563 bytes result sent to driver -26/04/01 06:32:25 INFO TaskSetManager: Starting task 48.0 in stage 14.0 (TID 139) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:32:25 INFO TaskSetManager: Finished task 45.0 in stage 14.0 (TID 136) in 3930 ms on 10.0.0.133 (executor driver) (45/208) -26/04/01 06:32:25 INFO Executor: Running task 48.0 in stage 14.0 (TID 139) -26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:25 INFO Executor: Finished task 46.0 in stage 14.0 (TID 137). 6563 bytes result sent to driver -26/04/01 06:32:25 INFO TaskSetManager: Starting task 49.0 in stage 14.0 (TID 140) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:25 INFO TaskSetManager: Finished task 46.0 in stage 14.0 (TID 137) in 4082 ms on 10.0.0.133 (executor driver) (46/208) -26/04/01 06:32:25 INFO Executor: Running task 49.0 in stage 14.0 (TID 140) -26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:25 INFO Executor: Finished task 47.0 in stage 14.0 (TID 138). 6563 bytes result sent to driver -26/04/01 06:32:25 INFO TaskSetManager: Starting task 50.0 in stage 14.0 (TID 141) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:25 INFO Executor: Running task 50.0 in stage 14.0 (TID 141) -26/04/01 06:32:25 INFO TaskSetManager: Finished task 47.0 in stage 14.0 (TID 138) in 4067 ms on 10.0.0.133 (executor driver) (47/208) -26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:25 INFO Executor: Finished task 44.0 in stage 14.0 (TID 135). 6563 bytes result sent to driver -26/04/01 06:32:25 INFO TaskSetManager: Starting task 51.0 in stage 14.0 (TID 142) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:25 INFO TaskSetManager: Finished task 44.0 in stage 14.0 (TID 135) in 4221 ms on 10.0.0.133 (executor driver) (48/208) -26/04/01 06:32:25 INFO Executor: Running task 51.0 in stage 14.0 (TID 142) -26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:29 INFO Executor: Finished task 48.0 in stage 14.0 (TID 139). 6563 bytes result sent to driver -26/04/01 06:32:29 INFO TaskSetManager: Starting task 52.0 in stage 14.0 (TID 143) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:29 INFO Executor: Running task 52.0 in stage 14.0 (TID 143) -26/04/01 06:32:29 INFO TaskSetManager: Finished task 48.0 in stage 14.0 (TID 139) in 4132 ms on 10.0.0.133 (executor driver) (49/208) -26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:29 INFO Executor: Finished task 49.0 in stage 14.0 (TID 140). 6563 bytes result sent to driver -26/04/01 06:32:29 INFO TaskSetManager: Starting task 53.0 in stage 14.0 (TID 144) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:29 INFO TaskSetManager: Finished task 49.0 in stage 14.0 (TID 140) in 4121 ms on 10.0.0.133 (executor driver) (50/208) -26/04/01 06:32:29 INFO Executor: Running task 53.0 in stage 14.0 (TID 144) -26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:29 INFO Executor: Finished task 50.0 in stage 14.0 (TID 141). 6563 bytes result sent to driver -26/04/01 06:32:29 INFO TaskSetManager: Starting task 54.0 in stage 14.0 (TID 145) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:32:29 INFO TaskSetManager: Finished task 50.0 in stage 14.0 (TID 141) in 4121 ms on 10.0.0.133 (executor driver) (51/208) -26/04/01 06:32:29 INFO Executor: Running task 54.0 in stage 14.0 (TID 145) -26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:29 INFO Executor: Finished task 51.0 in stage 14.0 (TID 142). 6563 bytes result sent to driver -26/04/01 06:32:29 INFO TaskSetManager: Starting task 55.0 in stage 14.0 (TID 146) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:29 INFO TaskSetManager: Finished task 51.0 in stage 14.0 (TID 142) in 4132 ms on 10.0.0.133 (executor driver) (52/208) -26/04/01 06:32:29 INFO Executor: Running task 55.0 in stage 14.0 (TID 146) -26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:33 INFO Executor: Finished task 52.0 in stage 14.0 (TID 143). 6563 bytes result sent to driver -26/04/01 06:32:33 INFO TaskSetManager: Starting task 56.0 in stage 14.0 (TID 147) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:33 INFO TaskSetManager: Finished task 52.0 in stage 14.0 (TID 143) in 4148 ms on 10.0.0.133 (executor driver) (53/208) -26/04/01 06:32:33 INFO Executor: Running task 56.0 in stage 14.0 (TID 147) -26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:33 INFO Executor: Finished task 53.0 in stage 14.0 (TID 144). 6563 bytes result sent to driver -26/04/01 06:32:33 INFO TaskSetManager: Starting task 57.0 in stage 14.0 (TID 148) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:33 INFO TaskSetManager: Finished task 53.0 in stage 14.0 (TID 144) in 4129 ms on 10.0.0.133 (executor driver) (54/208) -26/04/01 06:32:33 INFO Executor: Running task 57.0 in stage 14.0 (TID 148) -26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:33 INFO Executor: Finished task 54.0 in stage 14.0 (TID 145). 6563 bytes result sent to driver -26/04/01 06:32:33 INFO TaskSetManager: Starting task 58.0 in stage 14.0 (TID 149) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:33 INFO Executor: Running task 58.0 in stage 14.0 (TID 149) -26/04/01 06:32:33 INFO TaskSetManager: Finished task 54.0 in stage 14.0 (TID 145) in 4130 ms on 10.0.0.133 (executor driver) (55/208) -26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:33 INFO Executor: Finished task 55.0 in stage 14.0 (TID 146). 6563 bytes result sent to driver -26/04/01 06:32:33 INFO TaskSetManager: Starting task 59.0 in stage 14.0 (TID 150) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:33 INFO TaskSetManager: Finished task 55.0 in stage 14.0 (TID 146) in 4129 ms on 10.0.0.133 (executor driver) (56/208) -26/04/01 06:32:33 INFO Executor: Running task 59.0 in stage 14.0 (TID 150) -26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:37 INFO Executor: Finished task 56.0 in stage 14.0 (TID 147). 6563 bytes result sent to driver -26/04/01 06:32:37 INFO TaskSetManager: Starting task 60.0 in stage 14.0 (TID 151) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:32:37 INFO TaskSetManager: Finished task 56.0 in stage 14.0 (TID 147) in 4129 ms on 10.0.0.133 (executor driver) (57/208) -26/04/01 06:32:37 INFO Executor: Running task 60.0 in stage 14.0 (TID 151) -26/04/01 06:32:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:38 INFO Executor: Finished task 58.0 in stage 14.0 (TID 149). 6563 bytes result sent to driver -26/04/01 06:32:38 INFO Executor: Finished task 57.0 in stage 14.0 (TID 148). 6563 bytes result sent to driver -26/04/01 06:32:38 INFO TaskSetManager: Starting task 61.0 in stage 14.0 (TID 152) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:38 INFO Executor: Running task 61.0 in stage 14.0 (TID 152) -26/04/01 06:32:38 INFO TaskSetManager: Starting task 62.0 in stage 14.0 (TID 153) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:38 INFO TaskSetManager: Finished task 58.0 in stage 14.0 (TID 149) in 4135 ms on 10.0.0.133 (executor driver) (58/208) -26/04/01 06:32:38 INFO Executor: Running task 62.0 in stage 14.0 (TID 153) -26/04/01 06:32:38 INFO TaskSetManager: Finished task 57.0 in stage 14.0 (TID 148) in 4137 ms on 10.0.0.133 (executor driver) (59/208) -26/04/01 06:32:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:38 INFO Executor: Finished task 59.0 in stage 14.0 (TID 150). 6563 bytes result sent to driver -26/04/01 06:32:38 INFO TaskSetManager: Starting task 63.0 in stage 14.0 (TID 154) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:38 INFO TaskSetManager: Finished task 59.0 in stage 14.0 (TID 150) in 4134 ms on 10.0.0.133 (executor driver) (60/208) -26/04/01 06:32:38 INFO Executor: Running task 63.0 in stage 14.0 (TID 154) -26/04/01 06:32:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:42 INFO Executor: Finished task 60.0 in stage 14.0 (TID 151). 6563 bytes result sent to driver -26/04/01 06:32:42 INFO TaskSetManager: Starting task 64.0 in stage 14.0 (TID 155) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:42 INFO TaskSetManager: Finished task 60.0 in stage 14.0 (TID 151) in 4139 ms on 10.0.0.133 (executor driver) (61/208) -26/04/01 06:32:42 INFO Executor: Running task 64.0 in stage 14.0 (TID 155) -26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:42 INFO Executor: Finished task 62.0 in stage 14.0 (TID 153). 6563 bytes result sent to driver -26/04/01 06:32:42 INFO TaskSetManager: Starting task 65.0 in stage 14.0 (TID 156) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:42 INFO TaskSetManager: Finished task 62.0 in stage 14.0 (TID 153) in 4141 ms on 10.0.0.133 (executor driver) (62/208) -26/04/01 06:32:42 INFO Executor: Running task 65.0 in stage 14.0 (TID 156) -26/04/01 06:32:42 INFO Executor: Finished task 61.0 in stage 14.0 (TID 152). 6563 bytes result sent to driver -26/04/01 06:32:42 INFO TaskSetManager: Starting task 66.0 in stage 14.0 (TID 157) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:32:42 INFO TaskSetManager: Finished task 61.0 in stage 14.0 (TID 152) in 4142 ms on 10.0.0.133 (executor driver) (63/208) -26/04/01 06:32:42 INFO Executor: Running task 66.0 in stage 14.0 (TID 157) -26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:42 INFO Executor: Finished task 63.0 in stage 14.0 (TID 154). 6563 bytes result sent to driver -26/04/01 06:32:42 INFO TaskSetManager: Starting task 67.0 in stage 14.0 (TID 158) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:32:42 INFO TaskSetManager: Finished task 63.0 in stage 14.0 (TID 154) in 4143 ms on 10.0.0.133 (executor driver) (64/208) -26/04/01 06:32:42 INFO Executor: Running task 67.0 in stage 14.0 (TID 158) -26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:46 INFO Executor: Finished task 64.0 in stage 14.0 (TID 155). 6606 bytes result sent to driver -26/04/01 06:32:46 INFO TaskSetManager: Starting task 68.0 in stage 14.0 (TID 159) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:46 INFO TaskSetManager: Finished task 64.0 in stage 14.0 (TID 155) in 4137 ms on 10.0.0.133 (executor driver) (65/208) -26/04/01 06:32:46 INFO Executor: Running task 68.0 in stage 14.0 (TID 159) -26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:46 INFO Executor: Finished task 66.0 in stage 14.0 (TID 157). 6606 bytes result sent to driver -26/04/01 06:32:46 INFO TaskSetManager: Starting task 69.0 in stage 14.0 (TID 160) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:46 INFO TaskSetManager: Finished task 66.0 in stage 14.0 (TID 157) in 4075 ms on 10.0.0.133 (executor driver) (66/208) -26/04/01 06:32:46 INFO Executor: Running task 69.0 in stage 14.0 (TID 160) -26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:46 INFO Executor: Finished task 65.0 in stage 14.0 (TID 156). 6606 bytes result sent to driver -26/04/01 06:32:46 INFO TaskSetManager: Starting task 70.0 in stage 14.0 (TID 161) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:46 INFO TaskSetManager: Finished task 65.0 in stage 14.0 (TID 156) in 4142 ms on 10.0.0.133 (executor driver) (67/208) -26/04/01 06:32:46 INFO Executor: Running task 70.0 in stage 14.0 (TID 161) -26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:46 INFO Executor: Finished task 67.0 in stage 14.0 (TID 158). 6606 bytes result sent to driver -26/04/01 06:32:46 INFO TaskSetManager: Starting task 71.0 in stage 14.0 (TID 162) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:46 INFO TaskSetManager: Finished task 67.0 in stage 14.0 (TID 158) in 4086 ms on 10.0.0.133 (executor driver) (68/208) -26/04/01 06:32:46 INFO Executor: Running task 71.0 in stage 14.0 (TID 162) -26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:50 INFO Executor: Finished task 68.0 in stage 14.0 (TID 159). 6563 bytes result sent to driver -26/04/01 06:32:50 INFO TaskSetManager: Starting task 72.0 in stage 14.0 (TID 163) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:32:50 INFO TaskSetManager: Finished task 68.0 in stage 14.0 (TID 159) in 4093 ms on 10.0.0.133 (executor driver) (69/208) -26/04/01 06:32:50 INFO Executor: Running task 72.0 in stage 14.0 (TID 163) -26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:50 INFO Executor: Finished task 69.0 in stage 14.0 (TID 160). 6563 bytes result sent to driver -26/04/01 06:32:50 INFO TaskSetManager: Starting task 73.0 in stage 14.0 (TID 164) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:32:50 INFO TaskSetManager: Finished task 69.0 in stage 14.0 (TID 160) in 4078 ms on 10.0.0.133 (executor driver) (70/208) -26/04/01 06:32:50 INFO Executor: Running task 73.0 in stage 14.0 (TID 164) -26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:50 INFO Executor: Finished task 70.0 in stage 14.0 (TID 161). 6563 bytes result sent to driver -26/04/01 06:32:50 INFO TaskSetManager: Starting task 74.0 in stage 14.0 (TID 165) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:50 INFO Executor: Running task 74.0 in stage 14.0 (TID 165) -26/04/01 06:32:50 INFO TaskSetManager: Finished task 70.0 in stage 14.0 (TID 161) in 4087 ms on 10.0.0.133 (executor driver) (71/208) -26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:50 INFO Executor: Finished task 71.0 in stage 14.0 (TID 162). 6563 bytes result sent to driver -26/04/01 06:32:50 INFO TaskSetManager: Starting task 75.0 in stage 14.0 (TID 166) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:50 INFO TaskSetManager: Finished task 71.0 in stage 14.0 (TID 162) in 4080 ms on 10.0.0.133 (executor driver) (72/208) -26/04/01 06:32:50 INFO Executor: Running task 75.0 in stage 14.0 (TID 166) -26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:54 INFO Executor: Finished task 72.0 in stage 14.0 (TID 163). 6563 bytes result sent to driver -26/04/01 06:32:54 INFO TaskSetManager: Starting task 76.0 in stage 14.0 (TID 167) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:54 INFO TaskSetManager: Finished task 72.0 in stage 14.0 (TID 163) in 4111 ms on 10.0.0.133 (executor driver) (73/208) -26/04/01 06:32:54 INFO Executor: Running task 76.0 in stage 14.0 (TID 167) -26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:54 INFO Executor: Finished task 73.0 in stage 14.0 (TID 164). 6563 bytes result sent to driver -26/04/01 06:32:54 INFO TaskSetManager: Starting task 77.0 in stage 14.0 (TID 168) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:54 INFO TaskSetManager: Finished task 73.0 in stage 14.0 (TID 164) in 4123 ms on 10.0.0.133 (executor driver) (74/208) -26/04/01 06:32:54 INFO Executor: Running task 77.0 in stage 14.0 (TID 168) -26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:54 INFO Executor: Finished task 74.0 in stage 14.0 (TID 165). 6563 bytes result sent to driver -26/04/01 06:32:54 INFO TaskSetManager: Starting task 78.0 in stage 14.0 (TID 169) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:32:54 INFO TaskSetManager: Finished task 74.0 in stage 14.0 (TID 165) in 4123 ms on 10.0.0.133 (executor driver) (75/208) -26/04/01 06:32:54 INFO Executor: Running task 78.0 in stage 14.0 (TID 169) -26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:54 INFO Executor: Finished task 75.0 in stage 14.0 (TID 166). 6563 bytes result sent to driver -26/04/01 06:32:54 INFO TaskSetManager: Starting task 79.0 in stage 14.0 (TID 170) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:32:54 INFO TaskSetManager: Finished task 75.0 in stage 14.0 (TID 166) in 4121 ms on 10.0.0.133 (executor driver) (76/208) -26/04/01 06:32:54 INFO Executor: Running task 79.0 in stage 14.0 (TID 170) -26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:58 INFO Executor: Finished task 76.0 in stage 14.0 (TID 167). 6563 bytes result sent to driver -26/04/01 06:32:58 INFO TaskSetManager: Starting task 80.0 in stage 14.0 (TID 171) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:58 INFO TaskSetManager: Finished task 76.0 in stage 14.0 (TID 167) in 4134 ms on 10.0.0.133 (executor driver) (77/208) -26/04/01 06:32:58 INFO Executor: Running task 80.0 in stage 14.0 (TID 171) -26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:58 INFO Executor: Finished task 77.0 in stage 14.0 (TID 168). 6563 bytes result sent to driver -26/04/01 06:32:58 INFO TaskSetManager: Starting task 81.0 in stage 14.0 (TID 172) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:58 INFO TaskSetManager: Finished task 77.0 in stage 14.0 (TID 168) in 4120 ms on 10.0.0.133 (executor driver) (78/208) -26/04/01 06:32:58 INFO Executor: Running task 81.0 in stage 14.0 (TID 172) -26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:58 INFO Executor: Finished task 78.0 in stage 14.0 (TID 169). 6563 bytes result sent to driver -26/04/01 06:32:58 INFO TaskSetManager: Starting task 82.0 in stage 14.0 (TID 173) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:58 INFO TaskSetManager: Finished task 78.0 in stage 14.0 (TID 169) in 4125 ms on 10.0.0.133 (executor driver) (79/208) -26/04/01 06:32:58 INFO Executor: Running task 82.0 in stage 14.0 (TID 173) -26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:58 INFO Executor: Finished task 79.0 in stage 14.0 (TID 170). 6563 bytes result sent to driver -26/04/01 06:32:58 INFO TaskSetManager: Starting task 83.0 in stage 14.0 (TID 174) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:32:58 INFO TaskSetManager: Finished task 79.0 in stage 14.0 (TID 170) in 4123 ms on 10.0.0.133 (executor driver) (80/208) -26/04/01 06:32:58 INFO Executor: Running task 83.0 in stage 14.0 (TID 174) -26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:32:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:02 INFO Executor: Finished task 80.0 in stage 14.0 (TID 171). 6563 bytes result sent to driver -26/04/01 06:33:02 INFO TaskSetManager: Starting task 84.0 in stage 14.0 (TID 175) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:33:02 INFO TaskSetManager: Finished task 80.0 in stage 14.0 (TID 171) in 4134 ms on 10.0.0.133 (executor driver) (81/208) -26/04/01 06:33:02 INFO Executor: Running task 84.0 in stage 14.0 (TID 175) -26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:02 INFO Executor: Finished task 81.0 in stage 14.0 (TID 172). 6563 bytes result sent to driver -26/04/01 06:33:02 INFO TaskSetManager: Starting task 85.0 in stage 14.0 (TID 176) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:02 INFO TaskSetManager: Finished task 81.0 in stage 14.0 (TID 172) in 4123 ms on 10.0.0.133 (executor driver) (82/208) -26/04/01 06:33:02 INFO Executor: Running task 85.0 in stage 14.0 (TID 176) -26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:02 INFO Executor: Finished task 82.0 in stage 14.0 (TID 173). 6563 bytes result sent to driver -26/04/01 06:33:02 INFO TaskSetManager: Starting task 86.0 in stage 14.0 (TID 177) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:02 INFO TaskSetManager: Finished task 82.0 in stage 14.0 (TID 173) in 4131 ms on 10.0.0.133 (executor driver) (83/208) -26/04/01 06:33:02 INFO Executor: Running task 86.0 in stage 14.0 (TID 177) -26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:02 INFO Executor: Finished task 83.0 in stage 14.0 (TID 174). 6563 bytes result sent to driver -26/04/01 06:33:02 INFO TaskSetManager: Starting task 87.0 in stage 14.0 (TID 178) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:02 INFO Executor: Running task 87.0 in stage 14.0 (TID 178) -26/04/01 06:33:02 INFO TaskSetManager: Finished task 83.0 in stage 14.0 (TID 174) in 4133 ms on 10.0.0.133 (executor driver) (84/208) -26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:06 INFO Executor: Finished task 84.0 in stage 14.0 (TID 175). 6563 bytes result sent to driver -26/04/01 06:33:06 INFO TaskSetManager: Starting task 88.0 in stage 14.0 (TID 179) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:06 INFO TaskSetManager: Finished task 84.0 in stage 14.0 (TID 175) in 4133 ms on 10.0.0.133 (executor driver) (85/208) -26/04/01 06:33:06 INFO Executor: Running task 88.0 in stage 14.0 (TID 179) -26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:06 INFO Executor: Finished task 85.0 in stage 14.0 (TID 176). 6563 bytes result sent to driver -26/04/01 06:33:06 INFO TaskSetManager: Starting task 89.0 in stage 14.0 (TID 180) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:06 INFO Executor: Running task 89.0 in stage 14.0 (TID 180) -26/04/01 06:33:06 INFO TaskSetManager: Finished task 85.0 in stage 14.0 (TID 176) in 4141 ms on 10.0.0.133 (executor driver) (86/208) -26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:06 INFO Executor: Finished task 86.0 in stage 14.0 (TID 177). 6563 bytes result sent to driver -26/04/01 06:33:06 INFO TaskSetManager: Starting task 90.0 in stage 14.0 (TID 181) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:33:06 INFO TaskSetManager: Finished task 86.0 in stage 14.0 (TID 177) in 4141 ms on 10.0.0.133 (executor driver) (87/208) -26/04/01 06:33:06 INFO Executor: Running task 90.0 in stage 14.0 (TID 181) -26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:06 INFO Executor: Finished task 87.0 in stage 14.0 (TID 178). 6563 bytes result sent to driver -26/04/01 06:33:06 INFO TaskSetManager: Starting task 91.0 in stage 14.0 (TID 182) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:06 INFO TaskSetManager: Finished task 87.0 in stage 14.0 (TID 178) in 4133 ms on 10.0.0.133 (executor driver) (88/208) -26/04/01 06:33:06 INFO Executor: Running task 91.0 in stage 14.0 (TID 182) -26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:10 INFO Executor: Finished task 91.0 in stage 14.0 (TID 182). 6563 bytes result sent to driver -26/04/01 06:33:10 INFO TaskSetManager: Starting task 92.0 in stage 14.0 (TID 183) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:10 INFO TaskSetManager: Finished task 91.0 in stage 14.0 (TID 182) in 3982 ms on 10.0.0.133 (executor driver) (89/208) -26/04/01 06:33:10 INFO Executor: Running task 92.0 in stage 14.0 (TID 183) -26/04/01 06:33:10 INFO Executor: Finished task 88.0 in stage 14.0 (TID 179). 6563 bytes result sent to driver -26/04/01 06:33:10 INFO TaskSetManager: Starting task 93.0 in stage 14.0 (TID 184) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:10 INFO TaskSetManager: Finished task 88.0 in stage 14.0 (TID 179) in 4146 ms on 10.0.0.133 (executor driver) (90/208) -26/04/01 06:33:10 INFO Executor: Running task 93.0 in stage 14.0 (TID 184) -26/04/01 06:33:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:10 INFO Executor: Finished task 89.0 in stage 14.0 (TID 180). 6563 bytes result sent to driver -26/04/01 06:33:10 INFO TaskSetManager: Starting task 94.0 in stage 14.0 (TID 185) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:10 INFO TaskSetManager: Finished task 89.0 in stage 14.0 (TID 180) in 4138 ms on 10.0.0.133 (executor driver) (91/208) -26/04/01 06:33:10 INFO Executor: Running task 94.0 in stage 14.0 (TID 185) -26/04/01 06:33:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:11 INFO Executor: Finished task 90.0 in stage 14.0 (TID 181). 6563 bytes result sent to driver -26/04/01 06:33:11 INFO TaskSetManager: Starting task 95.0 in stage 14.0 (TID 186) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:11 INFO TaskSetManager: Finished task 90.0 in stage 14.0 (TID 181) in 4288 ms on 10.0.0.133 (executor driver) (92/208) -26/04/01 06:33:11 INFO Executor: Running task 95.0 in stage 14.0 (TID 186) -26/04/01 06:33:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:15 INFO Executor: Finished task 93.0 in stage 14.0 (TID 184). 6563 bytes result sent to driver -26/04/01 06:33:15 INFO Executor: Finished task 92.0 in stage 14.0 (TID 183). 6563 bytes result sent to driver -26/04/01 06:33:15 INFO TaskSetManager: Starting task 96.0 in stage 14.0 (TID 187) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:33:15 INFO TaskSetManager: Finished task 93.0 in stage 14.0 (TID 184) in 4144 ms on 10.0.0.133 (executor driver) (93/208) -26/04/01 06:33:15 INFO Executor: Running task 96.0 in stage 14.0 (TID 187) -26/04/01 06:33:15 INFO TaskSetManager: Starting task 97.0 in stage 14.0 (TID 188) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:33:15 INFO TaskSetManager: Finished task 92.0 in stage 14.0 (TID 183) in 4145 ms on 10.0.0.133 (executor driver) (94/208) -26/04/01 06:33:15 INFO Executor: Running task 97.0 in stage 14.0 (TID 188) -26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:15 INFO Executor: Finished task 94.0 in stage 14.0 (TID 185). 6563 bytes result sent to driver -26/04/01 06:33:15 INFO TaskSetManager: Starting task 98.0 in stage 14.0 (TID 189) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:15 INFO TaskSetManager: Finished task 94.0 in stage 14.0 (TID 185) in 4139 ms on 10.0.0.133 (executor driver) (95/208) -26/04/01 06:33:15 INFO Executor: Running task 98.0 in stage 14.0 (TID 189) -26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:15 INFO Executor: Finished task 95.0 in stage 14.0 (TID 186). 6563 bytes result sent to driver -26/04/01 06:33:15 INFO TaskSetManager: Starting task 99.0 in stage 14.0 (TID 190) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:15 INFO Executor: Running task 99.0 in stage 14.0 (TID 190) -26/04/01 06:33:15 INFO TaskSetManager: Finished task 95.0 in stage 14.0 (TID 186) in 4143 ms on 10.0.0.133 (executor driver) (96/208) -26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:19 INFO Executor: Finished task 97.0 in stage 14.0 (TID 188). 6563 bytes result sent to driver -26/04/01 06:33:19 INFO TaskSetManager: Starting task 100.0 in stage 14.0 (TID 191) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:19 INFO TaskSetManager: Finished task 97.0 in stage 14.0 (TID 188) in 4131 ms on 10.0.0.133 (executor driver) (97/208) -26/04/01 06:33:19 INFO Executor: Running task 100.0 in stage 14.0 (TID 191) -26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:19 INFO Executor: Finished task 96.0 in stage 14.0 (TID 187). 6563 bytes result sent to driver -26/04/01 06:33:19 INFO TaskSetManager: Starting task 101.0 in stage 14.0 (TID 192) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:19 INFO TaskSetManager: Finished task 96.0 in stage 14.0 (TID 187) in 4140 ms on 10.0.0.133 (executor driver) (98/208) -26/04/01 06:33:19 INFO Executor: Running task 101.0 in stage 14.0 (TID 192) -26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:19 INFO Executor: Finished task 98.0 in stage 14.0 (TID 189). 6563 bytes result sent to driver -26/04/01 06:33:19 INFO TaskSetManager: Starting task 102.0 in stage 14.0 (TID 193) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:33:19 INFO Executor: Running task 102.0 in stage 14.0 (TID 193) -26/04/01 06:33:19 INFO TaskSetManager: Finished task 98.0 in stage 14.0 (TID 189) in 4140 ms on 10.0.0.133 (executor driver) (99/208) -26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:19 INFO Executor: Finished task 99.0 in stage 14.0 (TID 190). 6563 bytes result sent to driver -26/04/01 06:33:19 INFO TaskSetManager: Starting task 103.0 in stage 14.0 (TID 194) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:19 INFO TaskSetManager: Finished task 99.0 in stage 14.0 (TID 190) in 4118 ms on 10.0.0.133 (executor driver) (100/208) -26/04/01 06:33:19 INFO Executor: Running task 103.0 in stage 14.0 (TID 194) -26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:23 INFO Executor: Finished task 100.0 in stage 14.0 (TID 191). 6563 bytes result sent to driver -26/04/01 06:33:23 INFO Executor: Finished task 101.0 in stage 14.0 (TID 192). 6563 bytes result sent to driver -26/04/01 06:33:23 INFO TaskSetManager: Starting task 104.0 in stage 14.0 (TID 195) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:23 INFO TaskSetManager: Finished task 100.0 in stage 14.0 (TID 191) in 4147 ms on 10.0.0.133 (executor driver) (101/208) -26/04/01 06:33:23 INFO Executor: Running task 104.0 in stage 14.0 (TID 195) -26/04/01 06:33:23 INFO TaskSetManager: Starting task 105.0 in stage 14.0 (TID 196) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:23 INFO TaskSetManager: Finished task 101.0 in stage 14.0 (TID 192) in 4137 ms on 10.0.0.133 (executor driver) (102/208) -26/04/01 06:33:23 INFO Executor: Running task 105.0 in stage 14.0 (TID 196) -26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:23 INFO Executor: Finished task 102.0 in stage 14.0 (TID 193). 6563 bytes result sent to driver -26/04/01 06:33:23 INFO TaskSetManager: Starting task 106.0 in stage 14.0 (TID 197) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:23 INFO TaskSetManager: Finished task 102.0 in stage 14.0 (TID 193) in 4137 ms on 10.0.0.133 (executor driver) (103/208) -26/04/01 06:33:23 INFO Executor: Running task 106.0 in stage 14.0 (TID 197) -26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:23 INFO Executor: Finished task 103.0 in stage 14.0 (TID 194). 6563 bytes result sent to driver -26/04/01 06:33:23 INFO TaskSetManager: Starting task 107.0 in stage 14.0 (TID 198) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:23 INFO TaskSetManager: Finished task 103.0 in stage 14.0 (TID 194) in 4145 ms on 10.0.0.133 (executor driver) (104/208) -26/04/01 06:33:23 INFO Executor: Running task 107.0 in stage 14.0 (TID 198) -26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:27 INFO Executor: Finished task 105.0 in stage 14.0 (TID 196). 6606 bytes result sent to driver -26/04/01 06:33:27 INFO TaskSetManager: Starting task 108.0 in stage 14.0 (TID 199) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:33:27 INFO TaskSetManager: Finished task 105.0 in stage 14.0 (TID 196) in 4139 ms on 10.0.0.133 (executor driver) (105/208) -26/04/01 06:33:27 INFO Executor: Running task 108.0 in stage 14.0 (TID 199) -26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:27 INFO Executor: Finished task 104.0 in stage 14.0 (TID 195). 6606 bytes result sent to driver -26/04/01 06:33:27 INFO TaskSetManager: Starting task 109.0 in stage 14.0 (TID 200) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:33:27 INFO Executor: Running task 109.0 in stage 14.0 (TID 200) -26/04/01 06:33:27 INFO TaskSetManager: Finished task 104.0 in stage 14.0 (TID 195) in 4146 ms on 10.0.0.133 (executor driver) (106/208) -26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:27 INFO Executor: Finished task 106.0 in stage 14.0 (TID 197). 6606 bytes result sent to driver -26/04/01 06:33:27 INFO TaskSetManager: Starting task 110.0 in stage 14.0 (TID 201) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:27 INFO TaskSetManager: Finished task 106.0 in stage 14.0 (TID 197) in 4135 ms on 10.0.0.133 (executor driver) (107/208) -26/04/01 06:33:27 INFO Executor: Running task 110.0 in stage 14.0 (TID 201) -26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:27 INFO Executor: Finished task 107.0 in stage 14.0 (TID 198). 6606 bytes result sent to driver -26/04/01 06:33:27 INFO TaskSetManager: Starting task 111.0 in stage 14.0 (TID 202) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:27 INFO Executor: Running task 111.0 in stage 14.0 (TID 202) -26/04/01 06:33:27 INFO TaskSetManager: Finished task 107.0 in stage 14.0 (TID 198) in 4138 ms on 10.0.0.133 (executor driver) (108/208) -26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:31 INFO Executor: Finished task 109.0 in stage 14.0 (TID 200). 6563 bytes result sent to driver -26/04/01 06:33:31 INFO TaskSetManager: Starting task 112.0 in stage 14.0 (TID 203) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:31 INFO Executor: Running task 112.0 in stage 14.0 (TID 203) -26/04/01 06:33:31 INFO TaskSetManager: Finished task 109.0 in stage 14.0 (TID 200) in 4132 ms on 10.0.0.133 (executor driver) (109/208) -26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:31 INFO Executor: Finished task 108.0 in stage 14.0 (TID 199). 6563 bytes result sent to driver -26/04/01 06:33:31 INFO TaskSetManager: Starting task 113.0 in stage 14.0 (TID 204) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:31 INFO Executor: Running task 113.0 in stage 14.0 (TID 204) -26/04/01 06:33:31 INFO TaskSetManager: Finished task 108.0 in stage 14.0 (TID 199) in 4144 ms on 10.0.0.133 (executor driver) (110/208) -26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:31 INFO Executor: Finished task 110.0 in stage 14.0 (TID 201). 6563 bytes result sent to driver -26/04/01 06:33:31 INFO TaskSetManager: Starting task 114.0 in stage 14.0 (TID 205) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:33:31 INFO TaskSetManager: Finished task 110.0 in stage 14.0 (TID 201) in 4141 ms on 10.0.0.133 (executor driver) (111/208) -26/04/01 06:33:31 INFO Executor: Running task 114.0 in stage 14.0 (TID 205) -26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:31 INFO Executor: Finished task 111.0 in stage 14.0 (TID 202). 6563 bytes result sent to driver -26/04/01 06:33:31 INFO TaskSetManager: Starting task 115.0 in stage 14.0 (TID 206) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:31 INFO TaskSetManager: Finished task 111.0 in stage 14.0 (TID 202) in 4139 ms on 10.0.0.133 (executor driver) (112/208) -26/04/01 06:33:31 INFO Executor: Running task 115.0 in stage 14.0 (TID 206) -26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:35 INFO Executor: Finished task 112.0 in stage 14.0 (TID 203). 6563 bytes result sent to driver -26/04/01 06:33:35 INFO TaskSetManager: Starting task 116.0 in stage 14.0 (TID 207) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:35 INFO TaskSetManager: Finished task 112.0 in stage 14.0 (TID 203) in 4142 ms on 10.0.0.133 (executor driver) (113/208) -26/04/01 06:33:35 INFO Executor: Running task 116.0 in stage 14.0 (TID 207) -26/04/01 06:33:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:35 INFO Executor: Finished task 113.0 in stage 14.0 (TID 204). 6563 bytes result sent to driver -26/04/01 06:33:35 INFO TaskSetManager: Starting task 117.0 in stage 14.0 (TID 208) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:35 INFO Executor: Running task 117.0 in stage 14.0 (TID 208) -26/04/01 06:33:35 INFO TaskSetManager: Finished task 113.0 in stage 14.0 (TID 204) in 4136 ms on 10.0.0.133 (executor driver) (114/208) -26/04/01 06:33:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:35 INFO Executor: Finished task 114.0 in stage 14.0 (TID 205). 6563 bytes result sent to driver -26/04/01 06:33:35 INFO TaskSetManager: Starting task 118.0 in stage 14.0 (TID 209) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:35 INFO TaskSetManager: Finished task 114.0 in stage 14.0 (TID 205) in 4129 ms on 10.0.0.133 (executor driver) (115/208) -26/04/01 06:33:35 INFO Executor: Running task 118.0 in stage 14.0 (TID 209) -26/04/01 06:33:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:36 INFO Executor: Finished task 115.0 in stage 14.0 (TID 206). 6563 bytes result sent to driver -26/04/01 06:33:36 INFO TaskSetManager: Starting task 119.0 in stage 14.0 (TID 210) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:36 INFO TaskSetManager: Finished task 115.0 in stage 14.0 (TID 206) in 4130 ms on 10.0.0.133 (executor driver) (116/208) -26/04/01 06:33:36 INFO Executor: Running task 119.0 in stage 14.0 (TID 210) -26/04/01 06:33:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:39 INFO Executor: Finished task 117.0 in stage 14.0 (TID 208). 6563 bytes result sent to driver -26/04/01 06:33:39 INFO Executor: Finished task 116.0 in stage 14.0 (TID 207). 6563 bytes result sent to driver -26/04/01 06:33:39 INFO TaskSetManager: Starting task 120.0 in stage 14.0 (TID 211) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:33:39 INFO TaskSetManager: Starting task 121.0 in stage 14.0 (TID 212) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:39 INFO Executor: Running task 120.0 in stage 14.0 (TID 211) -26/04/01 06:33:39 INFO TaskSetManager: Finished task 117.0 in stage 14.0 (TID 208) in 4130 ms on 10.0.0.133 (executor driver) (117/208) -26/04/01 06:33:39 INFO Executor: Running task 121.0 in stage 14.0 (TID 212) -26/04/01 06:33:39 INFO TaskSetManager: Finished task 116.0 in stage 14.0 (TID 207) in 4132 ms on 10.0.0.133 (executor driver) (118/208) -26/04/01 06:33:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:39 INFO Executor: Finished task 118.0 in stage 14.0 (TID 209). 6563 bytes result sent to driver -26/04/01 06:33:39 INFO TaskSetManager: Starting task 122.0 in stage 14.0 (TID 213) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:39 INFO TaskSetManager: Finished task 118.0 in stage 14.0 (TID 209) in 4129 ms on 10.0.0.133 (executor driver) (119/208) -26/04/01 06:33:39 INFO Executor: Running task 122.0 in stage 14.0 (TID 213) -26/04/01 06:33:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:40 INFO Executor: Finished task 119.0 in stage 14.0 (TID 210). 6563 bytes result sent to driver -26/04/01 06:33:40 INFO TaskSetManager: Starting task 123.0 in stage 14.0 (TID 214) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:40 INFO TaskSetManager: Finished task 119.0 in stage 14.0 (TID 210) in 4132 ms on 10.0.0.133 (executor driver) (120/208) -26/04/01 06:33:40 INFO Executor: Running task 123.0 in stage 14.0 (TID 214) -26/04/01 06:33:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:44 INFO Executor: Finished task 120.0 in stage 14.0 (TID 211). 6563 bytes result sent to driver -26/04/01 06:33:44 INFO TaskSetManager: Starting task 124.0 in stage 14.0 (TID 215) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:44 INFO TaskSetManager: Finished task 120.0 in stage 14.0 (TID 211) in 4115 ms on 10.0.0.133 (executor driver) (121/208) -26/04/01 06:33:44 INFO Executor: Running task 124.0 in stage 14.0 (TID 215) -26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:44 INFO Executor: Finished task 121.0 in stage 14.0 (TID 212). 6563 bytes result sent to driver -26/04/01 06:33:44 INFO TaskSetManager: Starting task 125.0 in stage 14.0 (TID 216) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:44 INFO TaskSetManager: Finished task 121.0 in stage 14.0 (TID 212) in 4132 ms on 10.0.0.133 (executor driver) (122/208) -26/04/01 06:33:44 INFO Executor: Running task 125.0 in stage 14.0 (TID 216) -26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:44 INFO Executor: Finished task 122.0 in stage 14.0 (TID 213). 6563 bytes result sent to driver -26/04/01 06:33:44 INFO TaskSetManager: Starting task 126.0 in stage 14.0 (TID 217) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:33:44 INFO TaskSetManager: Finished task 122.0 in stage 14.0 (TID 213) in 4132 ms on 10.0.0.133 (executor driver) (123/208) -26/04/01 06:33:44 INFO Executor: Running task 126.0 in stage 14.0 (TID 217) -26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:44 INFO Executor: Finished task 123.0 in stage 14.0 (TID 214). 6563 bytes result sent to driver -26/04/01 06:33:44 INFO TaskSetManager: Starting task 127.0 in stage 14.0 (TID 218) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:44 INFO TaskSetManager: Finished task 123.0 in stage 14.0 (TID 214) in 4129 ms on 10.0.0.133 (executor driver) (124/208) -26/04/01 06:33:44 INFO Executor: Running task 127.0 in stage 14.0 (TID 218) -26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:48 INFO Executor: Finished task 124.0 in stage 14.0 (TID 215). 6563 bytes result sent to driver -26/04/01 06:33:48 INFO TaskSetManager: Starting task 128.0 in stage 14.0 (TID 219) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:48 INFO TaskSetManager: Finished task 124.0 in stage 14.0 (TID 215) in 4118 ms on 10.0.0.133 (executor driver) (125/208) -26/04/01 06:33:48 INFO Executor: Running task 128.0 in stage 14.0 (TID 219) -26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:48 INFO Executor: Finished task 125.0 in stage 14.0 (TID 216). 6563 bytes result sent to driver -26/04/01 06:33:48 INFO TaskSetManager: Starting task 129.0 in stage 14.0 (TID 220) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:48 INFO TaskSetManager: Finished task 125.0 in stage 14.0 (TID 216) in 4128 ms on 10.0.0.133 (executor driver) (126/208) -26/04/01 06:33:48 INFO Executor: Running task 129.0 in stage 14.0 (TID 220) -26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:48 INFO Executor: Finished task 126.0 in stage 14.0 (TID 217). 6563 bytes result sent to driver -26/04/01 06:33:48 INFO TaskSetManager: Starting task 130.0 in stage 14.0 (TID 221) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:48 INFO TaskSetManager: Finished task 126.0 in stage 14.0 (TID 217) in 4120 ms on 10.0.0.133 (executor driver) (127/208) -26/04/01 06:33:48 INFO Executor: Running task 130.0 in stage 14.0 (TID 221) -26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:48 INFO Executor: Finished task 127.0 in stage 14.0 (TID 218). 6563 bytes result sent to driver -26/04/01 06:33:48 INFO TaskSetManager: Starting task 131.0 in stage 14.0 (TID 222) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:48 INFO TaskSetManager: Finished task 127.0 in stage 14.0 (TID 218) in 4117 ms on 10.0.0.133 (executor driver) (128/208) -26/04/01 06:33:48 INFO Executor: Running task 131.0 in stage 14.0 (TID 222) -26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:52 INFO Executor: Finished task 128.0 in stage 14.0 (TID 219). 6563 bytes result sent to driver -26/04/01 06:33:52 INFO TaskSetManager: Starting task 132.0 in stage 14.0 (TID 223) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:33:52 INFO TaskSetManager: Finished task 128.0 in stage 14.0 (TID 219) in 4131 ms on 10.0.0.133 (executor driver) (129/208) -26/04/01 06:33:52 INFO Executor: Running task 132.0 in stage 14.0 (TID 223) -26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:52 INFO Executor: Finished task 129.0 in stage 14.0 (TID 220). 6563 bytes result sent to driver -26/04/01 06:33:52 INFO TaskSetManager: Starting task 133.0 in stage 14.0 (TID 224) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:52 INFO TaskSetManager: Finished task 129.0 in stage 14.0 (TID 220) in 4136 ms on 10.0.0.133 (executor driver) (130/208) -26/04/01 06:33:52 INFO Executor: Running task 133.0 in stage 14.0 (TID 224) -26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:52 INFO Executor: Finished task 130.0 in stage 14.0 (TID 221). 6563 bytes result sent to driver -26/04/01 06:33:52 INFO TaskSetManager: Starting task 134.0 in stage 14.0 (TID 225) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:52 INFO TaskSetManager: Finished task 130.0 in stage 14.0 (TID 221) in 4133 ms on 10.0.0.133 (executor driver) (131/208) -26/04/01 06:33:52 INFO Executor: Running task 134.0 in stage 14.0 (TID 225) -26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:52 INFO Executor: Finished task 131.0 in stage 14.0 (TID 222). 6563 bytes result sent to driver -26/04/01 06:33:52 INFO TaskSetManager: Starting task 135.0 in stage 14.0 (TID 226) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:52 INFO TaskSetManager: Finished task 131.0 in stage 14.0 (TID 222) in 4140 ms on 10.0.0.133 (executor driver) (132/208) -26/04/01 06:33:52 INFO Executor: Running task 135.0 in stage 14.0 (TID 226) -26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:56 INFO Executor: Finished task 132.0 in stage 14.0 (TID 223). 6563 bytes result sent to driver -26/04/01 06:33:56 INFO TaskSetManager: Starting task 136.0 in stage 14.0 (TID 227) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:56 INFO TaskSetManager: Finished task 132.0 in stage 14.0 (TID 223) in 4083 ms on 10.0.0.133 (executor driver) (133/208) -26/04/01 06:33:56 INFO Executor: Running task 136.0 in stage 14.0 (TID 227) -26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:56 INFO Executor: Finished task 133.0 in stage 14.0 (TID 224). 6563 bytes result sent to driver -26/04/01 06:33:56 INFO TaskSetManager: Starting task 137.0 in stage 14.0 (TID 228) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:33:56 INFO TaskSetManager: Finished task 133.0 in stage 14.0 (TID 224) in 4076 ms on 10.0.0.133 (executor driver) (134/208) -26/04/01 06:33:56 INFO Executor: Running task 137.0 in stage 14.0 (TID 228) -26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:56 INFO Executor: Finished task 134.0 in stage 14.0 (TID 225). 6563 bytes result sent to driver -26/04/01 06:33:56 INFO TaskSetManager: Starting task 138.0 in stage 14.0 (TID 229) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:33:56 INFO TaskSetManager: Finished task 134.0 in stage 14.0 (TID 225) in 4084 ms on 10.0.0.133 (executor driver) (135/208) -26/04/01 06:33:56 INFO Executor: Running task 138.0 in stage 14.0 (TID 229) -26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:56 INFO Executor: Finished task 135.0 in stage 14.0 (TID 226). 6563 bytes result sent to driver -26/04/01 06:33:56 INFO TaskSetManager: Starting task 139.0 in stage 14.0 (TID 230) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:33:56 INFO Executor: Running task 139.0 in stage 14.0 (TID 230) -26/04/01 06:33:56 INFO TaskSetManager: Finished task 135.0 in stage 14.0 (TID 226) in 4079 ms on 10.0.0.133 (executor driver) (136/208) -26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:33:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:00 INFO Executor: Finished task 136.0 in stage 14.0 (TID 227). 6563 bytes result sent to driver -26/04/01 06:34:00 INFO TaskSetManager: Starting task 140.0 in stage 14.0 (TID 231) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:00 INFO TaskSetManager: Finished task 136.0 in stage 14.0 (TID 227) in 4091 ms on 10.0.0.133 (executor driver) (137/208) -26/04/01 06:34:00 INFO Executor: Running task 140.0 in stage 14.0 (TID 231) -26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:00 INFO Executor: Finished task 137.0 in stage 14.0 (TID 228). 6563 bytes result sent to driver -26/04/01 06:34:00 INFO TaskSetManager: Starting task 141.0 in stage 14.0 (TID 232) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:00 INFO TaskSetManager: Finished task 137.0 in stage 14.0 (TID 228) in 4080 ms on 10.0.0.133 (executor driver) (138/208) -26/04/01 06:34:00 INFO Executor: Running task 141.0 in stage 14.0 (TID 232) -26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:00 INFO Executor: Finished task 138.0 in stage 14.0 (TID 229). 6563 bytes result sent to driver -26/04/01 06:34:00 INFO TaskSetManager: Starting task 142.0 in stage 14.0 (TID 233) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:00 INFO TaskSetManager: Finished task 138.0 in stage 14.0 (TID 229) in 4083 ms on 10.0.0.133 (executor driver) (139/208) -26/04/01 06:34:00 INFO Executor: Running task 142.0 in stage 14.0 (TID 233) -26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:00 INFO Executor: Finished task 139.0 in stage 14.0 (TID 230). 6563 bytes result sent to driver -26/04/01 06:34:00 INFO TaskSetManager: Starting task 143.0 in stage 14.0 (TID 234) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:00 INFO TaskSetManager: Finished task 139.0 in stage 14.0 (TID 230) in 4081 ms on 10.0.0.133 (executor driver) (140/208) -26/04/01 06:34:00 INFO Executor: Running task 143.0 in stage 14.0 (TID 234) -26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:04 INFO Executor: Finished task 140.0 in stage 14.0 (TID 231). 6563 bytes result sent to driver -26/04/01 06:34:04 INFO TaskSetManager: Starting task 144.0 in stage 14.0 (TID 235) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:34:04 INFO TaskSetManager: Finished task 140.0 in stage 14.0 (TID 231) in 4079 ms on 10.0.0.133 (executor driver) (141/208) -26/04/01 06:34:04 INFO Executor: Running task 144.0 in stage 14.0 (TID 235) -26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:04 INFO Executor: Finished task 141.0 in stage 14.0 (TID 232). 6563 bytes result sent to driver -26/04/01 06:34:04 INFO TaskSetManager: Starting task 145.0 in stage 14.0 (TID 236) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:34:04 INFO Executor: Running task 145.0 in stage 14.0 (TID 236) -26/04/01 06:34:04 INFO TaskSetManager: Finished task 141.0 in stage 14.0 (TID 232) in 4078 ms on 10.0.0.133 (executor driver) (142/208) -26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:04 INFO Executor: Finished task 142.0 in stage 14.0 (TID 233). 6563 bytes result sent to driver -26/04/01 06:34:04 INFO TaskSetManager: Starting task 146.0 in stage 14.0 (TID 237) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:04 INFO Executor: Running task 146.0 in stage 14.0 (TID 237) -26/04/01 06:34:04 INFO TaskSetManager: Finished task 142.0 in stage 14.0 (TID 233) in 4077 ms on 10.0.0.133 (executor driver) (143/208) -26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:04 INFO Executor: Finished task 143.0 in stage 14.0 (TID 234). 6563 bytes result sent to driver -26/04/01 06:34:04 INFO TaskSetManager: Starting task 147.0 in stage 14.0 (TID 238) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:04 INFO TaskSetManager: Finished task 143.0 in stage 14.0 (TID 234) in 4081 ms on 10.0.0.133 (executor driver) (144/208) -26/04/01 06:34:04 INFO Executor: Running task 147.0 in stage 14.0 (TID 238) -26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:08 INFO Executor: Finished task 144.0 in stage 14.0 (TID 235). 6606 bytes result sent to driver -26/04/01 06:34:08 INFO TaskSetManager: Starting task 148.0 in stage 14.0 (TID 239) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:08 INFO Executor: Running task 148.0 in stage 14.0 (TID 239) -26/04/01 06:34:08 INFO TaskSetManager: Finished task 144.0 in stage 14.0 (TID 235) in 4139 ms on 10.0.0.133 (executor driver) (145/208) -26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:08 INFO Executor: Finished task 145.0 in stage 14.0 (TID 236). 6606 bytes result sent to driver -26/04/01 06:34:08 INFO TaskSetManager: Starting task 149.0 in stage 14.0 (TID 240) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:08 INFO Executor: Running task 149.0 in stage 14.0 (TID 240) -26/04/01 06:34:08 INFO TaskSetManager: Finished task 145.0 in stage 14.0 (TID 236) in 4129 ms on 10.0.0.133 (executor driver) (146/208) -26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:08 INFO Executor: Finished task 146.0 in stage 14.0 (TID 237). 6606 bytes result sent to driver -26/04/01 06:34:08 INFO TaskSetManager: Starting task 150.0 in stage 14.0 (TID 241) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:34:08 INFO TaskSetManager: Finished task 146.0 in stage 14.0 (TID 237) in 4120 ms on 10.0.0.133 (executor driver) (147/208) -26/04/01 06:34:08 INFO Executor: Running task 150.0 in stage 14.0 (TID 241) -26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:08 INFO Executor: Finished task 147.0 in stage 14.0 (TID 238). 6606 bytes result sent to driver -26/04/01 06:34:08 INFO TaskSetManager: Starting task 151.0 in stage 14.0 (TID 242) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:08 INFO Executor: Running task 151.0 in stage 14.0 (TID 242) -26/04/01 06:34:08 INFO TaskSetManager: Finished task 147.0 in stage 14.0 (TID 238) in 4133 ms on 10.0.0.133 (executor driver) (148/208) -26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:12 INFO Executor: Finished task 148.0 in stage 14.0 (TID 239). 6563 bytes result sent to driver -26/04/01 06:34:12 INFO Executor: Finished task 149.0 in stage 14.0 (TID 240). 6563 bytes result sent to driver -26/04/01 06:34:12 INFO TaskSetManager: Starting task 152.0 in stage 14.0 (TID 243) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:12 INFO TaskSetManager: Starting task 153.0 in stage 14.0 (TID 244) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:12 INFO Executor: Running task 152.0 in stage 14.0 (TID 243) -26/04/01 06:34:12 INFO TaskSetManager: Finished task 148.0 in stage 14.0 (TID 239) in 4158 ms on 10.0.0.133 (executor driver) (149/208) -26/04/01 06:34:12 INFO Executor: Running task 153.0 in stage 14.0 (TID 244) -26/04/01 06:34:12 INFO TaskSetManager: Finished task 149.0 in stage 14.0 (TID 240) in 4157 ms on 10.0.0.133 (executor driver) (150/208) -26/04/01 06:34:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:12 INFO Executor: Finished task 150.0 in stage 14.0 (TID 241). 6563 bytes result sent to driver -26/04/01 06:34:12 INFO TaskSetManager: Starting task 154.0 in stage 14.0 (TID 245) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:12 INFO TaskSetManager: Finished task 150.0 in stage 14.0 (TID 241) in 4151 ms on 10.0.0.133 (executor driver) (151/208) -26/04/01 06:34:12 INFO Executor: Running task 154.0 in stage 14.0 (TID 245) -26/04/01 06:34:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:13 INFO Executor: Finished task 151.0 in stage 14.0 (TID 242). 6563 bytes result sent to driver -26/04/01 06:34:13 INFO TaskSetManager: Starting task 155.0 in stage 14.0 (TID 246) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:13 INFO TaskSetManager: Finished task 151.0 in stage 14.0 (TID 242) in 4146 ms on 10.0.0.133 (executor driver) (152/208) -26/04/01 06:34:13 INFO Executor: Running task 155.0 in stage 14.0 (TID 246) -26/04/01 06:34:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:16 INFO Executor: Finished task 152.0 in stage 14.0 (TID 243). 6563 bytes result sent to driver -26/04/01 06:34:16 INFO TaskSetManager: Starting task 156.0 in stage 14.0 (TID 247) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:34:16 INFO TaskSetManager: Finished task 152.0 in stage 14.0 (TID 243) in 4134 ms on 10.0.0.133 (executor driver) (153/208) -26/04/01 06:34:16 INFO Executor: Running task 156.0 in stage 14.0 (TID 247) -26/04/01 06:34:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:16 INFO Executor: Finished task 153.0 in stage 14.0 (TID 244). 6563 bytes result sent to driver -26/04/01 06:34:16 INFO TaskSetManager: Starting task 157.0 in stage 14.0 (TID 248) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:16 INFO TaskSetManager: Finished task 153.0 in stage 14.0 (TID 244) in 4150 ms on 10.0.0.133 (executor driver) (154/208) -26/04/01 06:34:16 INFO Executor: Running task 157.0 in stage 14.0 (TID 248) -26/04/01 06:34:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:16 INFO Executor: Finished task 154.0 in stage 14.0 (TID 245). 6563 bytes result sent to driver -26/04/01 06:34:16 INFO TaskSetManager: Starting task 158.0 in stage 14.0 (TID 249) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:16 INFO TaskSetManager: Finished task 154.0 in stage 14.0 (TID 245) in 4141 ms on 10.0.0.133 (executor driver) (155/208) -26/04/01 06:34:16 INFO Executor: Running task 158.0 in stage 14.0 (TID 249) -26/04/01 06:34:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:17 INFO Executor: Finished task 155.0 in stage 14.0 (TID 246). 6563 bytes result sent to driver -26/04/01 06:34:17 INFO TaskSetManager: Starting task 159.0 in stage 14.0 (TID 250) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:17 INFO TaskSetManager: Finished task 155.0 in stage 14.0 (TID 246) in 4134 ms on 10.0.0.133 (executor driver) (156/208) -26/04/01 06:34:17 INFO Executor: Running task 159.0 in stage 14.0 (TID 250) -26/04/01 06:34:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:21 INFO Executor: Finished task 156.0 in stage 14.0 (TID 247). 6563 bytes result sent to driver -26/04/01 06:34:21 INFO TaskSetManager: Starting task 160.0 in stage 14.0 (TID 251) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:21 INFO TaskSetManager: Finished task 156.0 in stage 14.0 (TID 247) in 4081 ms on 10.0.0.133 (executor driver) (157/208) -26/04/01 06:34:21 INFO Executor: Running task 160.0 in stage 14.0 (TID 251) -26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:21 INFO Executor: Finished task 157.0 in stage 14.0 (TID 248). 6563 bytes result sent to driver -26/04/01 06:34:21 INFO Executor: Finished task 158.0 in stage 14.0 (TID 249). 6563 bytes result sent to driver -26/04/01 06:34:21 INFO TaskSetManager: Starting task 161.0 in stage 14.0 (TID 252) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:21 INFO Executor: Running task 161.0 in stage 14.0 (TID 252) -26/04/01 06:34:21 INFO TaskSetManager: Starting task 162.0 in stage 14.0 (TID 253) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:34:21 INFO Executor: Running task 162.0 in stage 14.0 (TID 253) -26/04/01 06:34:21 INFO TaskSetManager: Finished task 157.0 in stage 14.0 (TID 248) in 4085 ms on 10.0.0.133 (executor driver) (158/208) -26/04/01 06:34:21 INFO TaskSetManager: Finished task 158.0 in stage 14.0 (TID 249) in 4080 ms on 10.0.0.133 (executor driver) (159/208) -26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:21 INFO Executor: Finished task 159.0 in stage 14.0 (TID 250). 6563 bytes result sent to driver -26/04/01 06:34:21 INFO TaskSetManager: Starting task 163.0 in stage 14.0 (TID 254) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:34:21 INFO TaskSetManager: Finished task 159.0 in stage 14.0 (TID 250) in 4082 ms on 10.0.0.133 (executor driver) (160/208) -26/04/01 06:34:21 INFO Executor: Running task 163.0 in stage 14.0 (TID 254) -26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:25 INFO Executor: Finished task 160.0 in stage 14.0 (TID 251). 6563 bytes result sent to driver -26/04/01 06:34:25 INFO TaskSetManager: Starting task 164.0 in stage 14.0 (TID 255) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:25 INFO TaskSetManager: Finished task 160.0 in stage 14.0 (TID 251) in 4089 ms on 10.0.0.133 (executor driver) (161/208) -26/04/01 06:34:25 INFO Executor: Running task 164.0 in stage 14.0 (TID 255) -26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:25 INFO Executor: Finished task 161.0 in stage 14.0 (TID 252). 6563 bytes result sent to driver -26/04/01 06:34:25 INFO TaskSetManager: Starting task 165.0 in stage 14.0 (TID 256) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:25 INFO TaskSetManager: Finished task 161.0 in stage 14.0 (TID 252) in 4078 ms on 10.0.0.133 (executor driver) (162/208) -26/04/01 06:34:25 INFO Executor: Running task 165.0 in stage 14.0 (TID 256) -26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:25 INFO Executor: Finished task 162.0 in stage 14.0 (TID 253). 6563 bytes result sent to driver -26/04/01 06:34:25 INFO TaskSetManager: Starting task 166.0 in stage 14.0 (TID 257) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:25 INFO Executor: Running task 166.0 in stage 14.0 (TID 257) -26/04/01 06:34:25 INFO TaskSetManager: Finished task 162.0 in stage 14.0 (TID 253) in 4129 ms on 10.0.0.133 (executor driver) (163/208) -26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:25 INFO Executor: Finished task 163.0 in stage 14.0 (TID 254). 6563 bytes result sent to driver -26/04/01 06:34:25 INFO TaskSetManager: Starting task 167.0 in stage 14.0 (TID 258) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:25 INFO TaskSetManager: Finished task 163.0 in stage 14.0 (TID 254) in 4147 ms on 10.0.0.133 (executor driver) (164/208) -26/04/01 06:34:25 INFO Executor: Running task 167.0 in stage 14.0 (TID 258) -26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:29 INFO Executor: Finished task 164.0 in stage 14.0 (TID 255). 6563 bytes result sent to driver -26/04/01 06:34:29 INFO TaskSetManager: Starting task 168.0 in stage 14.0 (TID 259) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:34:29 INFO Executor: Running task 168.0 in stage 14.0 (TID 259) -26/04/01 06:34:29 INFO TaskSetManager: Finished task 164.0 in stage 14.0 (TID 255) in 4135 ms on 10.0.0.133 (executor driver) (165/208) -26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:29 INFO Executor: Finished task 165.0 in stage 14.0 (TID 256). 6563 bytes result sent to driver -26/04/01 06:34:29 INFO TaskSetManager: Starting task 169.0 in stage 14.0 (TID 260) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:29 INFO TaskSetManager: Finished task 165.0 in stage 14.0 (TID 256) in 4130 ms on 10.0.0.133 (executor driver) (166/208) -26/04/01 06:34:29 INFO Executor: Running task 169.0 in stage 14.0 (TID 260) -26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:29 INFO Executor: Finished task 166.0 in stage 14.0 (TID 257). 6563 bytes result sent to driver -26/04/01 06:34:29 INFO TaskSetManager: Starting task 170.0 in stage 14.0 (TID 261) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:29 INFO TaskSetManager: Finished task 166.0 in stage 14.0 (TID 257) in 4117 ms on 10.0.0.133 (executor driver) (167/208) -26/04/01 06:34:29 INFO Executor: Running task 170.0 in stage 14.0 (TID 261) -26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:29 INFO Executor: Finished task 167.0 in stage 14.0 (TID 258). 6563 bytes result sent to driver -26/04/01 06:34:29 INFO TaskSetManager: Starting task 171.0 in stage 14.0 (TID 262) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:29 INFO Executor: Running task 171.0 in stage 14.0 (TID 262) -26/04/01 06:34:29 INFO TaskSetManager: Finished task 167.0 in stage 14.0 (TID 258) in 4130 ms on 10.0.0.133 (executor driver) (168/208) -26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:33 INFO Executor: Finished task 169.0 in stage 14.0 (TID 260). 6563 bytes result sent to driver -26/04/01 06:34:33 INFO TaskSetManager: Starting task 172.0 in stage 14.0 (TID 263) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:33 INFO Executor: Running task 172.0 in stage 14.0 (TID 263) -26/04/01 06:34:33 INFO TaskSetManager: Finished task 169.0 in stage 14.0 (TID 260) in 4086 ms on 10.0.0.133 (executor driver) (169/208) -26/04/01 06:34:33 INFO Executor: Finished task 168.0 in stage 14.0 (TID 259). 6563 bytes result sent to driver -26/04/01 06:34:33 INFO TaskSetManager: Starting task 173.0 in stage 14.0 (TID 264) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:33 INFO Executor: Running task 173.0 in stage 14.0 (TID 264) -26/04/01 06:34:33 INFO TaskSetManager: Finished task 168.0 in stage 14.0 (TID 259) in 4090 ms on 10.0.0.133 (executor driver) (170/208) -26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:33 INFO Executor: Finished task 170.0 in stage 14.0 (TID 261). 6563 bytes result sent to driver -26/04/01 06:34:33 INFO TaskSetManager: Starting task 174.0 in stage 14.0 (TID 265) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9268 bytes) -26/04/01 06:34:33 INFO TaskSetManager: Finished task 170.0 in stage 14.0 (TID 261) in 4080 ms on 10.0.0.133 (executor driver) (171/208) -26/04/01 06:34:33 INFO Executor: Running task 174.0 in stage 14.0 (TID 265) -26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:33 INFO Executor: Finished task 171.0 in stage 14.0 (TID 262). 6563 bytes result sent to driver -26/04/01 06:34:33 INFO TaskSetManager: Starting task 175.0 in stage 14.0 (TID 266) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9273 bytes) -26/04/01 06:34:33 INFO Executor: Running task 175.0 in stage 14.0 (TID 266) -26/04/01 06:34:33 INFO TaskSetManager: Finished task 171.0 in stage 14.0 (TID 262) in 4081 ms on 10.0.0.133 (executor driver) (172/208) -26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:37 INFO Executor: Finished task 172.0 in stage 14.0 (TID 263). 6563 bytes result sent to driver -26/04/01 06:34:37 INFO TaskSetManager: Starting task 176.0 in stage 14.0 (TID 267) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:37 INFO TaskSetManager: Finished task 172.0 in stage 14.0 (TID 263) in 4071 ms on 10.0.0.133 (executor driver) (173/208) -26/04/01 06:34:37 INFO Executor: Running task 176.0 in stage 14.0 (TID 267) -26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:37 INFO Executor: Finished task 173.0 in stage 14.0 (TID 264). 6563 bytes result sent to driver -26/04/01 06:34:37 INFO TaskSetManager: Starting task 177.0 in stage 14.0 (TID 268) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:37 INFO Executor: Running task 177.0 in stage 14.0 (TID 268) -26/04/01 06:34:37 INFO TaskSetManager: Finished task 173.0 in stage 14.0 (TID 264) in 4079 ms on 10.0.0.133 (executor driver) (174/208) -26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:37 INFO Executor: Finished task 174.0 in stage 14.0 (TID 265). 6563 bytes result sent to driver -26/04/01 06:34:37 INFO TaskSetManager: Starting task 178.0 in stage 14.0 (TID 269) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:37 INFO Executor: Running task 178.0 in stage 14.0 (TID 269) -26/04/01 06:34:37 INFO TaskSetManager: Finished task 174.0 in stage 14.0 (TID 265) in 4135 ms on 10.0.0.133 (executor driver) (175/208) -26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:37 INFO Executor: Finished task 175.0 in stage 14.0 (TID 266). 6563 bytes result sent to driver -26/04/01 06:34:37 INFO TaskSetManager: Starting task 179.0 in stage 14.0 (TID 270) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:37 INFO Executor: Running task 179.0 in stage 14.0 (TID 270) -26/04/01 06:34:37 INFO TaskSetManager: Finished task 175.0 in stage 14.0 (TID 266) in 4134 ms on 10.0.0.133 (executor driver) (176/208) -26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:41 INFO Executor: Finished task 176.0 in stage 14.0 (TID 267). 6563 bytes result sent to driver -26/04/01 06:34:41 INFO Executor: Finished task 177.0 in stage 14.0 (TID 268). 6563 bytes result sent to driver -26/04/01 06:34:41 INFO TaskSetManager: Starting task 180.0 in stage 14.0 (TID 271) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:34:41 INFO Executor: Running task 180.0 in stage 14.0 (TID 271) -26/04/01 06:34:41 INFO TaskSetManager: Starting task 181.0 in stage 14.0 (TID 272) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:41 INFO TaskSetManager: Finished task 176.0 in stage 14.0 (TID 267) in 4146 ms on 10.0.0.133 (executor driver) (177/208) -26/04/01 06:34:41 INFO Executor: Running task 181.0 in stage 14.0 (TID 272) -26/04/01 06:34:41 INFO TaskSetManager: Finished task 177.0 in stage 14.0 (TID 268) in 4137 ms on 10.0.0.133 (executor driver) (178/208) -26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:41 INFO Executor: Finished task 178.0 in stage 14.0 (TID 269). 6563 bytes result sent to driver -26/04/01 06:34:41 INFO TaskSetManager: Starting task 182.0 in stage 14.0 (TID 273) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:41 INFO TaskSetManager: Finished task 178.0 in stage 14.0 (TID 269) in 4151 ms on 10.0.0.133 (executor driver) (179/208) -26/04/01 06:34:41 INFO Executor: Running task 182.0 in stage 14.0 (TID 273) -26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:41 INFO Executor: Finished task 179.0 in stage 14.0 (TID 270). 6563 bytes result sent to driver -26/04/01 06:34:41 INFO TaskSetManager: Starting task 183.0 in stage 14.0 (TID 274) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:41 INFO TaskSetManager: Finished task 179.0 in stage 14.0 (TID 270) in 4136 ms on 10.0.0.133 (executor driver) (180/208) -26/04/01 06:34:41 INFO Executor: Running task 183.0 in stage 14.0 (TID 274) -26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:45 INFO Executor: Finished task 180.0 in stage 14.0 (TID 271). 6563 bytes result sent to driver -26/04/01 06:34:45 INFO TaskSetManager: Starting task 184.0 in stage 14.0 (TID 275) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:45 INFO TaskSetManager: Finished task 180.0 in stage 14.0 (TID 271) in 4130 ms on 10.0.0.133 (executor driver) (181/208) -26/04/01 06:34:45 INFO Executor: Running task 184.0 in stage 14.0 (TID 275) -26/04/01 06:34:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:45 INFO Executor: Finished task 181.0 in stage 14.0 (TID 272). 6563 bytes result sent to driver -26/04/01 06:34:45 INFO TaskSetManager: Starting task 185.0 in stage 14.0 (TID 276) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:45 INFO TaskSetManager: Finished task 181.0 in stage 14.0 (TID 272) in 4140 ms on 10.0.0.133 (executor driver) (182/208) -26/04/01 06:34:45 INFO Executor: Running task 185.0 in stage 14.0 (TID 276) -26/04/01 06:34:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:45 INFO Executor: Finished task 182.0 in stage 14.0 (TID 273). 6563 bytes result sent to driver -26/04/01 06:34:45 INFO TaskSetManager: Starting task 186.0 in stage 14.0 (TID 277) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:34:45 INFO TaskSetManager: Finished task 182.0 in stage 14.0 (TID 273) in 4145 ms on 10.0.0.133 (executor driver) (183/208) -26/04/01 06:34:45 INFO Executor: Running task 186.0 in stage 14.0 (TID 277) -26/04/01 06:34:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:46 INFO Executor: Finished task 183.0 in stage 14.0 (TID 274). 6563 bytes result sent to driver -26/04/01 06:34:46 INFO TaskSetManager: Starting task 187.0 in stage 14.0 (TID 278) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9274 bytes) -26/04/01 06:34:46 INFO TaskSetManager: Finished task 183.0 in stage 14.0 (TID 274) in 4128 ms on 10.0.0.133 (executor driver) (184/208) -26/04/01 06:34:46 INFO Executor: Running task 187.0 in stage 14.0 (TID 278) -26/04/01 06:34:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:49 INFO Executor: Finished task 184.0 in stage 14.0 (TID 275). 6606 bytes result sent to driver -26/04/01 06:34:49 INFO TaskSetManager: Starting task 188.0 in stage 14.0 (TID 279) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:49 INFO TaskSetManager: Finished task 184.0 in stage 14.0 (TID 275) in 4139 ms on 10.0.0.133 (executor driver) (185/208) -26/04/01 06:34:49 INFO Executor: Finished task 185.0 in stage 14.0 (TID 276). 6606 bytes result sent to driver -26/04/01 06:34:49 INFO Executor: Running task 188.0 in stage 14.0 (TID 279) -26/04/01 06:34:49 INFO TaskSetManager: Starting task 189.0 in stage 14.0 (TID 280) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:49 INFO TaskSetManager: Finished task 185.0 in stage 14.0 (TID 276) in 4128 ms on 10.0.0.133 (executor driver) (186/208) -26/04/01 06:34:49 INFO Executor: Running task 189.0 in stage 14.0 (TID 280) -26/04/01 06:34:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:49 INFO Executor: Finished task 186.0 in stage 14.0 (TID 277). 6606 bytes result sent to driver -26/04/01 06:34:49 INFO TaskSetManager: Starting task 190.0 in stage 14.0 (TID 281) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:49 INFO Executor: Running task 190.0 in stage 14.0 (TID 281) -26/04/01 06:34:49 INFO TaskSetManager: Finished task 186.0 in stage 14.0 (TID 277) in 4120 ms on 10.0.0.133 (executor driver) (187/208) -26/04/01 06:34:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:50 INFO Executor: Finished task 187.0 in stage 14.0 (TID 278). 6606 bytes result sent to driver -26/04/01 06:34:50 INFO TaskSetManager: Starting task 191.0 in stage 14.0 (TID 282) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9275 bytes) -26/04/01 06:34:50 INFO TaskSetManager: Finished task 187.0 in stage 14.0 (TID 278) in 4148 ms on 10.0.0.133 (executor driver) (188/208) -26/04/01 06:34:50 INFO Executor: Running task 191.0 in stage 14.0 (TID 282) -26/04/01 06:34:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:53 INFO Executor: Finished task 189.0 in stage 14.0 (TID 280). 6563 bytes result sent to driver -26/04/01 06:34:53 INFO TaskSetManager: Starting task 192.0 in stage 14.0 (TID 283) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:34:53 INFO TaskSetManager: Finished task 189.0 in stage 14.0 (TID 280) in 4124 ms on 10.0.0.133 (executor driver) (189/208) -26/04/01 06:34:53 INFO Executor: Running task 192.0 in stage 14.0 (TID 283) -26/04/01 06:34:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:53 INFO Executor: Finished task 188.0 in stage 14.0 (TID 279). 6563 bytes result sent to driver -26/04/01 06:34:53 INFO TaskSetManager: Starting task 193.0 in stage 14.0 (TID 284) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:34:53 INFO Executor: Running task 193.0 in stage 14.0 (TID 284) -26/04/01 06:34:53 INFO TaskSetManager: Finished task 188.0 in stage 14.0 (TID 279) in 4128 ms on 10.0.0.133 (executor driver) (190/208) -26/04/01 06:34:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:54 INFO Executor: Finished task 190.0 in stage 14.0 (TID 281). 6563 bytes result sent to driver -26/04/01 06:34:54 INFO TaskSetManager: Starting task 194.0 in stage 14.0 (TID 285) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:34:54 INFO TaskSetManager: Finished task 190.0 in stage 14.0 (TID 281) in 4131 ms on 10.0.0.133 (executor driver) (191/208) -26/04/01 06:34:54 INFO Executor: Running task 194.0 in stage 14.0 (TID 285) -26/04/01 06:34:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:54 INFO Executor: Finished task 191.0 in stage 14.0 (TID 282). 6563 bytes result sent to driver -26/04/01 06:34:54 INFO TaskSetManager: Starting task 195.0 in stage 14.0 (TID 286) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:34:54 INFO Executor: Running task 195.0 in stage 14.0 (TID 286) -26/04/01 06:34:54 INFO TaskSetManager: Finished task 191.0 in stage 14.0 (TID 282) in 4129 ms on 10.0.0.133 (executor driver) (192/208) -26/04/01 06:34:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:56 INFO Executor: Finished task 193.0 in stage 14.0 (TID 284). 6563 bytes result sent to driver -26/04/01 06:34:56 INFO TaskSetManager: Starting task 196.0 in stage 14.0 (TID 287) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:34:56 INFO TaskSetManager: Finished task 193.0 in stage 14.0 (TID 284) in 2583 ms on 10.0.0.133 (executor driver) (193/208) -26/04/01 06:34:56 INFO Executor: Running task 196.0 in stage 14.0 (TID 287) -26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:56 INFO Executor: Finished task 192.0 in stage 14.0 (TID 283). 6563 bytes result sent to driver -26/04/01 06:34:56 INFO TaskSetManager: Starting task 197.0 in stage 14.0 (TID 288) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:34:56 INFO Executor: Running task 197.0 in stage 14.0 (TID 288) -26/04/01 06:34:56 INFO TaskSetManager: Finished task 192.0 in stage 14.0 (TID 283) in 2597 ms on 10.0.0.133 (executor driver) (194/208) -26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:56 INFO Executor: Finished task 194.0 in stage 14.0 (TID 285). 6563 bytes result sent to driver -26/04/01 06:34:56 INFO TaskSetManager: Starting task 198.0 in stage 14.0 (TID 289) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:34:56 INFO TaskSetManager: Finished task 194.0 in stage 14.0 (TID 285) in 2581 ms on 10.0.0.133 (executor driver) (195/208) -26/04/01 06:34:56 INFO Executor: Running task 198.0 in stage 14.0 (TID 289) -26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:56 INFO Executor: Finished task 195.0 in stage 14.0 (TID 286). 6563 bytes result sent to driver -26/04/01 06:34:56 INFO TaskSetManager: Starting task 199.0 in stage 14.0 (TID 290) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:34:56 INFO TaskSetManager: Finished task 195.0 in stage 14.0 (TID 286) in 2590 ms on 10.0.0.133 (executor driver) (196/208) -26/04/01 06:34:56 INFO Executor: Running task 199.0 in stage 14.0 (TID 290) -26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:59 INFO Executor: Finished task 196.0 in stage 14.0 (TID 287). 6563 bytes result sent to driver -26/04/01 06:34:59 INFO TaskSetManager: Starting task 200.0 in stage 14.0 (TID 291) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9346 bytes) -26/04/01 06:34:59 INFO TaskSetManager: Finished task 196.0 in stage 14.0 (TID 287) in 2570 ms on 10.0.0.133 (executor driver) (197/208) -26/04/01 06:34:59 INFO Executor: Running task 200.0 in stage 14.0 (TID 291) -26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:59 INFO Executor: Finished task 197.0 in stage 14.0 (TID 288). 6563 bytes result sent to driver -26/04/01 06:34:59 INFO TaskSetManager: Starting task 201.0 in stage 14.0 (TID 292) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:34:59 INFO Executor: Running task 201.0 in stage 14.0 (TID 292) -26/04/01 06:34:59 INFO TaskSetManager: Finished task 197.0 in stage 14.0 (TID 288) in 2567 ms on 10.0.0.133 (executor driver) (198/208) -26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:59 INFO Executor: Finished task 198.0 in stage 14.0 (TID 289). 6563 bytes result sent to driver -26/04/01 06:34:59 INFO TaskSetManager: Starting task 202.0 in stage 14.0 (TID 293) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:34:59 INFO TaskSetManager: Finished task 198.0 in stage 14.0 (TID 289) in 2577 ms on 10.0.0.133 (executor driver) (199/208) -26/04/01 06:34:59 INFO Executor: Running task 202.0 in stage 14.0 (TID 293) -26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:59 INFO Executor: Finished task 199.0 in stage 14.0 (TID 290). 6563 bytes result sent to driver -26/04/01 06:34:59 INFO TaskSetManager: Starting task 203.0 in stage 14.0 (TID 294) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:34:59 INFO TaskSetManager: Finished task 199.0 in stage 14.0 (TID 290) in 2567 ms on 10.0.0.133 (executor driver) (200/208) -26/04/01 06:34:59 INFO Executor: Running task 203.0 in stage 14.0 (TID 294) -26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:34:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:01 INFO Executor: Finished task 200.0 in stage 14.0 (TID 291). 6563 bytes result sent to driver -26/04/01 06:35:01 INFO TaskSetManager: Starting task 204.0 in stage 14.0 (TID 295) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:35:01 INFO TaskSetManager: Finished task 200.0 in stage 14.0 (TID 291) in 2572 ms on 10.0.0.133 (executor driver) (201/208) -26/04/01 06:35:01 INFO Executor: Running task 204.0 in stage 14.0 (TID 295) -26/04/01 06:35:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:01 INFO Executor: Finished task 201.0 in stage 14.0 (TID 292). 6563 bytes result sent to driver -26/04/01 06:35:01 INFO TaskSetManager: Starting task 205.0 in stage 14.0 (TID 296) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9348 bytes) -26/04/01 06:35:01 INFO TaskSetManager: Finished task 201.0 in stage 14.0 (TID 292) in 2578 ms on 10.0.0.133 (executor driver) (202/208) -26/04/01 06:35:01 INFO Executor: Running task 205.0 in stage 14.0 (TID 296) -26/04/01 06:35:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:01 INFO Executor: Finished task 202.0 in stage 14.0 (TID 293). 6563 bytes result sent to driver -26/04/01 06:35:01 INFO TaskSetManager: Starting task 206.0 in stage 14.0 (TID 297) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:35:01 INFO TaskSetManager: Finished task 202.0 in stage 14.0 (TID 293) in 2579 ms on 10.0.0.133 (executor driver) (203/208) -26/04/01 06:35:01 INFO Executor: Running task 206.0 in stage 14.0 (TID 297) -26/04/01 06:35:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:02 INFO Executor: Finished task 203.0 in stage 14.0 (TID 294). 6563 bytes result sent to driver -26/04/01 06:35:02 INFO TaskSetManager: Starting task 207.0 in stage 14.0 (TID 298) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9347 bytes) -26/04/01 06:35:02 INFO TaskSetManager: Finished task 203.0 in stage 14.0 (TID 294) in 2571 ms on 10.0.0.133 (executor driver) (204/208) -26/04/01 06:35:02 INFO Executor: Running task 207.0 in stage 14.0 (TID 298) -26/04/01 06:35:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO Executor: Finished task 204.0 in stage 14.0 (TID 295). 6563 bytes result sent to driver -26/04/01 06:35:04 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 299) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) -26/04/01 06:35:04 INFO TaskSetManager: Finished task 204.0 in stage 14.0 (TID 295) in 2552 ms on 10.0.0.133 (executor driver) (205/208) -26/04/01 06:35:04 INFO Executor: Running task 0.0 in stage 15.0 (TID 299) -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO Executor: Finished task 0.0 in stage 15.0 (TID 299). 11159 bytes result sent to driver -26/04/01 06:35:04 INFO TaskSetManager: Starting task 0.0 in stage 16.0 (TID 300) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9722 bytes) -26/04/01 06:35:04 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 299) in 15 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:35:04 INFO Executor: Running task 0.0 in stage 16.0 (TID 300) -26/04/01 06:35:04 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool -26/04/01 06:35:04 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 228.291 s -26/04/01 06:35:04 INFO DAGScheduler: Job 14 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:35:04 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished -26/04/01 06:35:04 INFO DAGScheduler: Job 14 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 228.290733 s -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO Utils: Coalesced 1 broadcast batches into 1 (5 rows) -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO MemoryStore: Block broadcast_25 stored as values in memory (estimated size 608.0 B, free 12.3 GiB) -26/04/01 06:35:04 INFO MemoryStore: Block broadcast_25_piece0 stored as bytes in memory (estimated size 563.0 B, free 12.3 GiB) -26/04/01 06:35:04 INFO BlockManagerInfo: Added broadcast_25_piece0 in memory on 10.0.0.133:58442 (size: 563.0 B, free: 12.6 GiB) -26/04/01 06:35:04 INFO SparkContext: Created broadcast 25 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:35:04 INFO Executor: Finished task 205.0 in stage 14.0 (TID 296). 6563 bytes result sent to driver -26/04/01 06:35:04 INFO TaskSetManager: Starting task 1.0 in stage 16.0 (TID 301) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9724 bytes) -26/04/01 06:35:04 INFO Executor: Running task 1.0 in stage 16.0 (TID 301) -26/04/01 06:35:04 INFO TaskSetManager: Finished task 205.0 in stage 14.0 (TID 296) in 2558 ms on 10.0.0.133 (executor driver) (206/208) -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO Executor: Finished task 206.0 in stage 14.0 (TID 297). 6563 bytes result sent to driver -26/04/01 06:35:04 INFO TaskSetManager: Starting task 2.0 in stage 16.0 (TID 302) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9724 bytes) -26/04/01 06:35:04 INFO Executor: Running task 2.0 in stage 16.0 (TID 302) -26/04/01 06:35:04 INFO TaskSetManager: Finished task 206.0 in stage 14.0 (TID 297) in 2552 ms on 10.0.0.133 (executor driver) (207/208) -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO Executor: Finished task 0.0 in stage 16.0 (TID 300). 6563 bytes result sent to driver -26/04/01 06:35:04 INFO TaskSetManager: Starting task 3.0 in stage 16.0 (TID 303) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9721 bytes) -26/04/01 06:35:04 INFO Executor: Running task 3.0 in stage 16.0 (TID 303) -26/04/01 06:35:04 INFO TaskSetManager: Finished task 0.0 in stage 16.0 (TID 300) in 207 ms on 10.0.0.133 (executor driver) (1/4) -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO Executor: Finished task 1.0 in stage 16.0 (TID 301). 6563 bytes result sent to driver -26/04/01 06:35:04 INFO TaskSetManager: Starting task 0.0 in stage 19.0 (TID 304) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:04 INFO Executor: Running task 0.0 in stage 19.0 (TID 304) -26/04/01 06:35:04 INFO TaskSetManager: Finished task 1.0 in stage 16.0 (TID 301) in 211 ms on 10.0.0.133 (executor driver) (2/4) -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO Executor: Finished task 2.0 in stage 16.0 (TID 302). 6563 bytes result sent to driver -26/04/01 06:35:04 INFO TaskSetManager: Starting task 1.0 in stage 19.0 (TID 305) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:04 INFO TaskSetManager: Finished task 2.0 in stage 16.0 (TID 302) in 210 ms on 10.0.0.133 (executor driver) (3/4) -26/04/01 06:35:04 INFO Executor: Running task 1.0 in stage 19.0 (TID 305) -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO Executor: Finished task 207.0 in stage 14.0 (TID 298). 6563 bytes result sent to driver -26/04/01 06:35:04 INFO TaskSetManager: Starting task 2.0 in stage 19.0 (TID 306) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:04 INFO TaskSetManager: Finished task 207.0 in stage 14.0 (TID 298) in 2553 ms on 10.0.0.133 (executor driver) (208/208) -26/04/01 06:35:04 INFO Executor: Running task 2.0 in stage 19.0 (TID 306) -26/04/01 06:35:04 INFO TaskSchedulerImpl: Removed TaskSet 14.0, whose tasks have all completed, from pool -26/04/01 06:35:04 INFO DAGScheduler: ShuffleMapStage 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 228.648 s -26/04/01 06:35:04 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:35:04 INFO DAGScheduler: running: Set(ShuffleMapStage 19, ShuffleMapStage 16) -26/04/01 06:35:04 INFO DAGScheduler: waiting: Set() -26/04/01 06:35:04 INFO DAGScheduler: failed: Set() -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO Executor: Finished task 3.0 in stage 16.0 (TID 303). 6563 bytes result sent to driver -26/04/01 06:35:04 INFO TaskSetManager: Starting task 3.0 in stage 19.0 (TID 307) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:04 INFO TaskSetManager: Finished task 3.0 in stage 16.0 (TID 303) in 208 ms on 10.0.0.133 (executor driver) (4/4) -26/04/01 06:35:04 INFO Executor: Running task 3.0 in stage 19.0 (TID 307) -26/04/01 06:35:04 INFO TaskSchedulerImpl: Removed TaskSet 16.0, whose tasks have all completed, from pool -26/04/01 06:35:04 INFO DAGScheduler: ShuffleMapStage 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 228.699 s -26/04/01 06:35:04 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:35:04 INFO DAGScheduler: running: Set(ShuffleMapStage 19) -26/04/01 06:35:04 INFO DAGScheduler: waiting: Set() -26/04/01 06:35:04 INFO DAGScheduler: failed: Set() -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 16 (20.0 MiB) non-empty blocks including 16 (20.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:04 INFO BlockManagerInfo: Removed broadcast_20_piece0 on 10.0.0.133:58442 in memory (size: 8.0 KiB, free: 12.6 GiB) -26/04/01 06:35:04 INFO BlockManagerInfo: Removed broadcast_22_piece0 on 10.0.0.133:58442 in memory (size: 7.8 KiB, free: 12.6 GiB) -26/04/01 06:35:04 INFO BlockManagerInfo: Removed broadcast_21_piece0 on 10.0.0.133:58442 in memory (size: 30.7 KiB, free: 12.6 GiB) -26/04/01 06:35:15 INFO Executor: Finished task 0.0 in stage 19.0 (TID 304). 9594 bytes result sent to driver -26/04/01 06:35:15 INFO TaskSetManager: Starting task 4.0 in stage 19.0 (TID 308) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:15 INFO Executor: Running task 4.0 in stage 19.0 (TID 308) -26/04/01 06:35:15 INFO TaskSetManager: Finished task 0.0 in stage 19.0 (TID 304) in 11227 ms on 10.0.0.133 (executor driver) (1/6) -26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Getting 16 (20.1 MiB) non-empty blocks including 16 (20.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:16 INFO Executor: Finished task 1.0 in stage 19.0 (TID 305). 9594 bytes result sent to driver -26/04/01 06:35:16 INFO TaskSetManager: Starting task 5.0 in stage 19.0 (TID 309) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:16 INFO TaskSetManager: Finished task 1.0 in stage 19.0 (TID 305) in 11541 ms on 10.0.0.133 (executor driver) (2/6) -26/04/01 06:35:16 INFO Executor: Running task 5.0 in stage 19.0 (TID 309) -26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Getting 16 (11.1 MiB) non-empty blocks including 16 (11.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Getting 64 (23.7 MiB) non-empty blocks including 64 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Getting 16 (11.1 MiB) non-empty blocks including 16 (11.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Getting 64 (23.7 MiB) non-empty blocks including 64 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:16 INFO Executor: Finished task 3.0 in stage 19.0 (TID 307). 9594 bytes result sent to driver -26/04/01 06:35:16 INFO TaskSetManager: Finished task 3.0 in stage 19.0 (TID 307) in 11966 ms on 10.0.0.133 (executor driver) (3/6) -26/04/01 06:35:16 INFO Executor: Finished task 2.0 in stage 19.0 (TID 306). 9594 bytes result sent to driver -26/04/01 06:35:16 INFO TaskSetManager: Finished task 2.0 in stage 19.0 (TID 306) in 12038 ms on 10.0.0.133 (executor driver) (4/6) -26/04/01 06:35:21 INFO Executor: Finished task 5.0 in stage 19.0 (TID 309). 8863 bytes result sent to driver -26/04/01 06:35:21 INFO TaskSetManager: Finished task 5.0 in stage 19.0 (TID 309) in 5567 ms on 10.0.0.133 (executor driver) (5/6) -26/04/01 06:35:25 INFO Executor: Finished task 4.0 in stage 19.0 (TID 308). 9594 bytes result sent to driver -26/04/01 06:35:25 INFO TaskSetManager: Finished task 4.0 in stage 19.0 (TID 308) in 10294 ms on 10.0.0.133 (executor driver) (6/6) -26/04/01 06:35:25 INFO TaskSchedulerImpl: Removed TaskSet 19.0, whose tasks have all completed, from pool -26/04/01 06:35:25 INFO DAGScheduler: ShuffleMapStage 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 229.659 s -26/04/01 06:35:25 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:35:25 INFO DAGScheduler: running: Set() -26/04/01 06:35:25 INFO DAGScheduler: waiting: Set() -26/04/01 06:35:25 INFO DAGScheduler: failed: Set() -26/04/01 06:35:25 INFO ShufflePartitionsUtil: For shuffle(5, 3), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 06:35:25 INFO DAGScheduler: Registering RDD 48 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 6 -26/04/01 06:35:25 INFO DAGScheduler: Got map stage job 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions -26/04/01 06:35:25 INFO DAGScheduler: Final stage: ShuffleMapStage 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:35:25 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 22, ShuffleMapStage 23) -26/04/01 06:35:25 INFO DAGScheduler: Missing parents: List() -26/04/01 06:35:25 INFO DAGScheduler: Submitting ShuffleMapStage 24 (MapPartitionsRDD[48] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:35:26 INFO MemoryStore: Block broadcast_26 stored as values in memory (estimated size 29.4 KiB, free 12.6 GiB) -26/04/01 06:35:26 INFO MemoryStore: Block broadcast_26_piece0 stored as bytes in memory (estimated size 11.7 KiB, free 12.6 GiB) -26/04/01 06:35:26 INFO BlockManagerInfo: Added broadcast_26_piece0 in memory on 10.0.0.133:58442 (size: 11.7 KiB, free: 12.6 GiB) -26/04/01 06:35:26 INFO SparkContext: Created broadcast 26 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:35:26 INFO DAGScheduler: Submitting 200 missing tasks from ShuffleMapStage 24 (MapPartitionsRDD[48] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 06:35:26 INFO TaskSchedulerImpl: Adding task set 24.0 with 200 tasks resource profile 0 -26/04/01 06:35:26 INFO TaskSetManager: Starting task 0.0 in stage 24.0 (TID 310) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:26 INFO TaskSetManager: Starting task 1.0 in stage 24.0 (TID 311) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:26 INFO TaskSetManager: Starting task 2.0 in stage 24.0 (TID 312) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:26 INFO TaskSetManager: Starting task 3.0 in stage 24.0 (TID 313) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:26 INFO Executor: Running task 1.0 in stage 24.0 (TID 311) -26/04/01 06:35:26 INFO Executor: Running task 0.0 in stage 24.0 (TID 310) -26/04/01 06:35:26 INFO Executor: Running task 3.0 in stage 24.0 (TID 313) -26/04/01 06:35:26 INFO Executor: Running task 2.0 in stage 24.0 (TID 312) -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:29 INFO Executor: Finished task 1.0 in stage 24.0 (TID 311). 8470 bytes result sent to driver -26/04/01 06:35:29 INFO TaskSetManager: Starting task 4.0 in stage 24.0 (TID 314) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:29 INFO TaskSetManager: Finished task 1.0 in stage 24.0 (TID 311) in 3517 ms on 10.0.0.133 (executor driver) (1/200) -26/04/01 06:35:29 INFO Executor: Running task 4.0 in stage 24.0 (TID 314) -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:29 INFO Executor: Finished task 2.0 in stage 24.0 (TID 312). 8470 bytes result sent to driver -26/04/01 06:35:29 INFO TaskSetManager: Starting task 5.0 in stage 24.0 (TID 315) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:29 INFO TaskSetManager: Finished task 2.0 in stage 24.0 (TID 312) in 3576 ms on 10.0.0.133 (executor driver) (2/200) -26/04/01 06:35:29 INFO Executor: Running task 5.0 in stage 24.0 (TID 315) -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO Executor: Finished task 0.0 in stage 24.0 (TID 310). 8470 bytes result sent to driver -26/04/01 06:35:29 INFO TaskSetManager: Starting task 6.0 in stage 24.0 (TID 316) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:29 INFO TaskSetManager: Finished task 0.0 in stage 24.0 (TID 310) in 3582 ms on 10.0.0.133 (executor driver) (3/200) -26/04/01 06:35:29 INFO Executor: Running task 6.0 in stage 24.0 (TID 316) -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:29 INFO Executor: Finished task 3.0 in stage 24.0 (TID 313). 8470 bytes result sent to driver -26/04/01 06:35:29 INFO TaskSetManager: Starting task 7.0 in stage 24.0 (TID 317) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:29 INFO TaskSetManager: Finished task 3.0 in stage 24.0 (TID 313) in 3591 ms on 10.0.0.133 (executor driver) (4/200) -26/04/01 06:35:29 INFO Executor: Running task 7.0 in stage 24.0 (TID 317) -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:32 INFO Executor: Finished task 4.0 in stage 24.0 (TID 314). 8470 bytes result sent to driver -26/04/01 06:35:32 INFO TaskSetManager: Starting task 8.0 in stage 24.0 (TID 318) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:32 INFO TaskSetManager: Finished task 4.0 in stage 24.0 (TID 314) in 3395 ms on 10.0.0.133 (executor driver) (5/200) -26/04/01 06:35:32 INFO Executor: Running task 8.0 in stage 24.0 (TID 318) -26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:33 INFO Executor: Finished task 5.0 in stage 24.0 (TID 315). 8470 bytes result sent to driver -26/04/01 06:35:33 INFO TaskSetManager: Starting task 9.0 in stage 24.0 (TID 319) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:33 INFO TaskSetManager: Finished task 5.0 in stage 24.0 (TID 315) in 3449 ms on 10.0.0.133 (executor driver) (6/200) -26/04/01 06:35:33 INFO Executor: Running task 9.0 in stage 24.0 (TID 319) -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:33 INFO Executor: Finished task 6.0 in stage 24.0 (TID 316). 8470 bytes result sent to driver -26/04/01 06:35:33 INFO TaskSetManager: Starting task 10.0 in stage 24.0 (TID 320) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:33 INFO Executor: Running task 10.0 in stage 24.0 (TID 320) -26/04/01 06:35:33 INFO TaskSetManager: Finished task 6.0 in stage 24.0 (TID 316) in 3491 ms on 10.0.0.133 (executor driver) (7/200) -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:33 INFO Executor: Finished task 7.0 in stage 24.0 (TID 317). 8470 bytes result sent to driver -26/04/01 06:35:33 INFO TaskSetManager: Starting task 11.0 in stage 24.0 (TID 321) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:33 INFO Executor: Running task 11.0 in stage 24.0 (TID 321) -26/04/01 06:35:33 INFO TaskSetManager: Finished task 7.0 in stage 24.0 (TID 317) in 3486 ms on 10.0.0.133 (executor driver) (8/200) -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:36 INFO Executor: Finished task 8.0 in stage 24.0 (TID 318). 9201 bytes result sent to driver -26/04/01 06:35:36 INFO TaskSetManager: Starting task 12.0 in stage 24.0 (TID 322) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:36 INFO TaskSetManager: Finished task 8.0 in stage 24.0 (TID 318) in 3404 ms on 10.0.0.133 (executor driver) (9/200) -26/04/01 06:35:36 INFO Executor: Running task 12.0 in stage 24.0 (TID 322) -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:36 INFO Executor: Finished task 9.0 in stage 24.0 (TID 319). 9201 bytes result sent to driver -26/04/01 06:35:36 INFO TaskSetManager: Starting task 13.0 in stage 24.0 (TID 323) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:36 INFO Executor: Running task 13.0 in stage 24.0 (TID 323) -26/04/01 06:35:36 INFO TaskSetManager: Finished task 9.0 in stage 24.0 (TID 319) in 3400 ms on 10.0.0.133 (executor driver) (10/200) -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:36 INFO Executor: Finished task 10.0 in stage 24.0 (TID 320). 9201 bytes result sent to driver -26/04/01 06:35:36 INFO TaskSetManager: Starting task 14.0 in stage 24.0 (TID 324) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:36 INFO TaskSetManager: Finished task 10.0 in stage 24.0 (TID 320) in 3449 ms on 10.0.0.133 (executor driver) (11/200) -26/04/01 06:35:36 INFO Executor: Running task 14.0 in stage 24.0 (TID 324) -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1027.0 KiB) non-empty blocks including 6 (1027.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO Executor: Finished task 11.0 in stage 24.0 (TID 321). 9201 bytes result sent to driver -26/04/01 06:35:36 INFO TaskSetManager: Starting task 15.0 in stage 24.0 (TID 325) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:36 INFO Executor: Running task 15.0 in stage 24.0 (TID 325) -26/04/01 06:35:36 INFO TaskSetManager: Finished task 11.0 in stage 24.0 (TID 321) in 3450 ms on 10.0.0.133 (executor driver) (12/200) -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1027.0 KiB) non-empty blocks including 6 (1027.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:39 INFO Executor: Finished task 12.0 in stage 24.0 (TID 322). 8470 bytes result sent to driver -26/04/01 06:35:39 INFO TaskSetManager: Starting task 16.0 in stage 24.0 (TID 326) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:39 INFO TaskSetManager: Finished task 12.0 in stage 24.0 (TID 322) in 3387 ms on 10.0.0.133 (executor driver) (13/200) -26/04/01 06:35:39 INFO Executor: Running task 16.0 in stage 24.0 (TID 326) -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:39 INFO Executor: Finished task 13.0 in stage 24.0 (TID 323). 8470 bytes result sent to driver -26/04/01 06:35:39 INFO TaskSetManager: Starting task 17.0 in stage 24.0 (TID 327) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:39 INFO Executor: Running task 17.0 in stage 24.0 (TID 327) -26/04/01 06:35:39 INFO TaskSetManager: Finished task 13.0 in stage 24.0 (TID 323) in 3387 ms on 10.0.0.133 (executor driver) (14/200) -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:39 INFO Executor: Finished task 14.0 in stage 24.0 (TID 324). 8470 bytes result sent to driver -26/04/01 06:35:39 INFO TaskSetManager: Starting task 18.0 in stage 24.0 (TID 328) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:39 INFO TaskSetManager: Finished task 14.0 in stage 24.0 (TID 324) in 3441 ms on 10.0.0.133 (executor driver) (15/200) -26/04/01 06:35:39 INFO Executor: Running task 18.0 in stage 24.0 (TID 328) -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO Executor: Finished task 15.0 in stage 24.0 (TID 325). 8470 bytes result sent to driver -26/04/01 06:35:39 INFO TaskSetManager: Starting task 19.0 in stage 24.0 (TID 329) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:39 INFO TaskSetManager: Finished task 15.0 in stage 24.0 (TID 325) in 3441 ms on 10.0.0.133 (executor driver) (16/200) -26/04/01 06:35:39 INFO Executor: Running task 19.0 in stage 24.0 (TID 329) -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:39 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:43 INFO Executor: Finished task 16.0 in stage 24.0 (TID 326). 8470 bytes result sent to driver -26/04/01 06:35:43 INFO TaskSetManager: Starting task 20.0 in stage 24.0 (TID 330) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:43 INFO Executor: Running task 20.0 in stage 24.0 (TID 330) -26/04/01 06:35:43 INFO TaskSetManager: Finished task 16.0 in stage 24.0 (TID 326) in 3376 ms on 10.0.0.133 (executor driver) (17/200) -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:43 INFO Executor: Finished task 17.0 in stage 24.0 (TID 327). 8470 bytes result sent to driver -26/04/01 06:35:43 INFO TaskSetManager: Starting task 21.0 in stage 24.0 (TID 331) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:43 INFO TaskSetManager: Finished task 17.0 in stage 24.0 (TID 327) in 3375 ms on 10.0.0.133 (executor driver) (18/200) -26/04/01 06:35:43 INFO Executor: Running task 21.0 in stage 24.0 (TID 331) -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:43 INFO Executor: Finished task 19.0 in stage 24.0 (TID 329). 8470 bytes result sent to driver -26/04/01 06:35:43 INFO TaskSetManager: Starting task 22.0 in stage 24.0 (TID 332) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:43 INFO TaskSetManager: Finished task 19.0 in stage 24.0 (TID 329) in 3440 ms on 10.0.0.133 (executor driver) (19/200) -26/04/01 06:35:43 INFO Executor: Running task 22.0 in stage 24.0 (TID 332) -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO Executor: Finished task 18.0 in stage 24.0 (TID 328). 8470 bytes result sent to driver -26/04/01 06:35:43 INFO TaskSetManager: Starting task 23.0 in stage 24.0 (TID 333) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:43 INFO TaskSetManager: Finished task 18.0 in stage 24.0 (TID 328) in 3454 ms on 10.0.0.133 (executor driver) (20/200) -26/04/01 06:35:43 INFO Executor: Running task 23.0 in stage 24.0 (TID 333) -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:46 INFO Executor: Finished task 20.0 in stage 24.0 (TID 330). 9201 bytes result sent to driver -26/04/01 06:35:46 INFO TaskSetManager: Starting task 24.0 in stage 24.0 (TID 334) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:46 INFO TaskSetManager: Finished task 20.0 in stage 24.0 (TID 330) in 3384 ms on 10.0.0.133 (executor driver) (21/200) -26/04/01 06:35:46 INFO Executor: Running task 24.0 in stage 24.0 (TID 334) -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:46 INFO Executor: Finished task 21.0 in stage 24.0 (TID 331). 9201 bytes result sent to driver -26/04/01 06:35:46 INFO TaskSetManager: Starting task 25.0 in stage 24.0 (TID 335) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:46 INFO TaskSetManager: Finished task 21.0 in stage 24.0 (TID 331) in 3389 ms on 10.0.0.133 (executor driver) (22/200) -26/04/01 06:35:46 INFO Executor: Running task 25.0 in stage 24.0 (TID 335) -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:46 INFO Executor: Finished task 22.0 in stage 24.0 (TID 332). 9201 bytes result sent to driver -26/04/01 06:35:46 INFO TaskSetManager: Starting task 26.0 in stage 24.0 (TID 336) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:46 INFO TaskSetManager: Finished task 22.0 in stage 24.0 (TID 332) in 3437 ms on 10.0.0.133 (executor driver) (23/200) -26/04/01 06:35:46 INFO Executor: Running task 26.0 in stage 24.0 (TID 336) -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO Executor: Finished task 23.0 in stage 24.0 (TID 333). 9201 bytes result sent to driver -26/04/01 06:35:46 INFO TaskSetManager: Starting task 27.0 in stage 24.0 (TID 337) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:46 INFO TaskSetManager: Finished task 23.0 in stage 24.0 (TID 333) in 3433 ms on 10.0.0.133 (executor driver) (24/200) -26/04/01 06:35:46 INFO Executor: Running task 27.0 in stage 24.0 (TID 337) -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:46 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:49 INFO Executor: Finished task 24.0 in stage 24.0 (TID 334). 8470 bytes result sent to driver -26/04/01 06:35:49 INFO TaskSetManager: Starting task 28.0 in stage 24.0 (TID 338) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:49 INFO TaskSetManager: Finished task 24.0 in stage 24.0 (TID 334) in 3393 ms on 10.0.0.133 (executor driver) (25/200) -26/04/01 06:35:49 INFO Executor: Running task 28.0 in stage 24.0 (TID 338) -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:49 INFO Executor: Finished task 25.0 in stage 24.0 (TID 335). 8470 bytes result sent to driver -26/04/01 06:35:49 INFO TaskSetManager: Starting task 29.0 in stage 24.0 (TID 339) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:49 INFO Executor: Running task 29.0 in stage 24.0 (TID 339) -26/04/01 06:35:49 INFO TaskSetManager: Finished task 25.0 in stage 24.0 (TID 335) in 3394 ms on 10.0.0.133 (executor driver) (26/200) -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:49 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:50 INFO Executor: Finished task 26.0 in stage 24.0 (TID 336). 8470 bytes result sent to driver -26/04/01 06:35:50 INFO TaskSetManager: Starting task 30.0 in stage 24.0 (TID 340) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:50 INFO Executor: Running task 30.0 in stage 24.0 (TID 340) -26/04/01 06:35:50 INFO TaskSetManager: Finished task 26.0 in stage 24.0 (TID 336) in 3474 ms on 10.0.0.133 (executor driver) (27/200) -26/04/01 06:35:50 INFO Executor: Finished task 27.0 in stage 24.0 (TID 337). 8470 bytes result sent to driver -26/04/01 06:35:50 INFO TaskSetManager: Starting task 31.0 in stage 24.0 (TID 341) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:50 INFO TaskSetManager: Finished task 27.0 in stage 24.0 (TID 337) in 3470 ms on 10.0.0.133 (executor driver) (28/200) -26/04/01 06:35:50 INFO Executor: Running task 31.0 in stage 24.0 (TID 341) -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:53 INFO Executor: Finished task 28.0 in stage 24.0 (TID 338). 8470 bytes result sent to driver -26/04/01 06:35:53 INFO TaskSetManager: Starting task 32.0 in stage 24.0 (TID 342) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:53 INFO TaskSetManager: Finished task 28.0 in stage 24.0 (TID 338) in 3377 ms on 10.0.0.133 (executor driver) (29/200) -26/04/01 06:35:53 INFO Executor: Running task 32.0 in stage 24.0 (TID 342) -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:53 INFO Executor: Finished task 29.0 in stage 24.0 (TID 339). 8470 bytes result sent to driver -26/04/01 06:35:53 INFO TaskSetManager: Starting task 33.0 in stage 24.0 (TID 343) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:53 INFO TaskSetManager: Finished task 29.0 in stage 24.0 (TID 339) in 3377 ms on 10.0.0.133 (executor driver) (30/200) -26/04/01 06:35:53 INFO Executor: Running task 33.0 in stage 24.0 (TID 343) -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:53 INFO Executor: Finished task 30.0 in stage 24.0 (TID 340). 8470 bytes result sent to driver -26/04/01 06:35:53 INFO TaskSetManager: Starting task 34.0 in stage 24.0 (TID 344) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:53 INFO Executor: Running task 34.0 in stage 24.0 (TID 344) -26/04/01 06:35:53 INFO TaskSetManager: Finished task 30.0 in stage 24.0 (TID 340) in 3422 ms on 10.0.0.133 (executor driver) (31/200) -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO Executor: Finished task 31.0 in stage 24.0 (TID 341). 8470 bytes result sent to driver -26/04/01 06:35:53 INFO TaskSetManager: Starting task 35.0 in stage 24.0 (TID 345) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:53 INFO TaskSetManager: Finished task 31.0 in stage 24.0 (TID 341) in 3429 ms on 10.0.0.133 (executor driver) (32/200) -26/04/01 06:35:53 INFO Executor: Running task 35.0 in stage 24.0 (TID 345) -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:56 INFO Executor: Finished task 32.0 in stage 24.0 (TID 342). 9201 bytes result sent to driver -26/04/01 06:35:56 INFO TaskSetManager: Starting task 36.0 in stage 24.0 (TID 346) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:56 INFO Executor: Running task 36.0 in stage 24.0 (TID 346) -26/04/01 06:35:56 INFO TaskSetManager: Finished task 32.0 in stage 24.0 (TID 342) in 3359 ms on 10.0.0.133 (executor driver) (33/200) -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:56 INFO Executor: Finished task 33.0 in stage 24.0 (TID 343). 9201 bytes result sent to driver -26/04/01 06:35:56 INFO TaskSetManager: Starting task 37.0 in stage 24.0 (TID 347) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:56 INFO Executor: Running task 37.0 in stage 24.0 (TID 347) -26/04/01 06:35:56 INFO TaskSetManager: Finished task 33.0 in stage 24.0 (TID 343) in 3358 ms on 10.0.0.133 (executor driver) (34/200) -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:57 INFO Executor: Finished task 34.0 in stage 24.0 (TID 344). 9201 bytes result sent to driver -26/04/01 06:35:57 INFO Executor: Finished task 35.0 in stage 24.0 (TID 345). 9201 bytes result sent to driver -26/04/01 06:35:57 INFO TaskSetManager: Starting task 38.0 in stage 24.0 (TID 348) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:57 INFO Executor: Running task 38.0 in stage 24.0 (TID 348) -26/04/01 06:35:57 INFO TaskSetManager: Starting task 39.0 in stage 24.0 (TID 349) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:57 INFO Executor: Running task 39.0 in stage 24.0 (TID 349) -26/04/01 06:35:57 INFO TaskSetManager: Finished task 34.0 in stage 24.0 (TID 344) in 3434 ms on 10.0.0.133 (executor driver) (35/200) -26/04/01 06:35:57 INFO TaskSetManager: Finished task 35.0 in stage 24.0 (TID 345) in 3428 ms on 10.0.0.133 (executor driver) (36/200) -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:59 INFO Executor: Finished task 36.0 in stage 24.0 (TID 346). 8470 bytes result sent to driver -26/04/01 06:35:59 INFO TaskSetManager: Starting task 40.0 in stage 24.0 (TID 350) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9428 bytes) -26/04/01 06:35:59 INFO Executor: Running task 40.0 in stage 24.0 (TID 350) -26/04/01 06:35:59 INFO TaskSetManager: Finished task 36.0 in stage 24.0 (TID 346) in 3370 ms on 10.0.0.133 (executor driver) (37/200) -26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:35:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:35:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:35:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:00 INFO Executor: Finished task 37.0 in stage 24.0 (TID 347). 8470 bytes result sent to driver -26/04/01 06:36:00 INFO TaskSetManager: Starting task 41.0 in stage 24.0 (TID 351) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:00 INFO Executor: Running task 41.0 in stage 24.0 (TID 351) -26/04/01 06:36:00 INFO TaskSetManager: Finished task 37.0 in stage 24.0 (TID 347) in 3373 ms on 10.0.0.133 (executor driver) (38/200) -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:00 INFO Executor: Finished task 38.0 in stage 24.0 (TID 348). 8470 bytes result sent to driver -26/04/01 06:36:00 INFO TaskSetManager: Starting task 42.0 in stage 24.0 (TID 352) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:00 INFO TaskSetManager: Finished task 38.0 in stage 24.0 (TID 348) in 3442 ms on 10.0.0.133 (executor driver) (39/200) -26/04/01 06:36:00 INFO Executor: Running task 42.0 in stage 24.0 (TID 352) -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:00 INFO Executor: Finished task 39.0 in stage 24.0 (TID 349). 8470 bytes result sent to driver -26/04/01 06:36:00 INFO TaskSetManager: Starting task 43.0 in stage 24.0 (TID 353) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:00 INFO TaskSetManager: Finished task 39.0 in stage 24.0 (TID 349) in 3449 ms on 10.0.0.133 (executor driver) (40/200) -26/04/01 06:36:00 INFO Executor: Running task 43.0 in stage 24.0 (TID 353) -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:03 INFO Executor: Finished task 40.0 in stage 24.0 (TID 350). 8470 bytes result sent to driver -26/04/01 06:36:03 INFO TaskSetManager: Starting task 44.0 in stage 24.0 (TID 354) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:03 INFO TaskSetManager: Finished task 40.0 in stage 24.0 (TID 350) in 3358 ms on 10.0.0.133 (executor driver) (41/200) -26/04/01 06:36:03 INFO Executor: Running task 44.0 in stage 24.0 (TID 354) -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:03 INFO Executor: Finished task 41.0 in stage 24.0 (TID 351). 8470 bytes result sent to driver -26/04/01 06:36:03 INFO TaskSetManager: Starting task 45.0 in stage 24.0 (TID 355) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:03 INFO TaskSetManager: Finished task 41.0 in stage 24.0 (TID 351) in 3353 ms on 10.0.0.133 (executor driver) (42/200) -26/04/01 06:36:03 INFO Executor: Running task 45.0 in stage 24.0 (TID 355) -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:04 INFO Executor: Finished task 43.0 in stage 24.0 (TID 353). 8470 bytes result sent to driver -26/04/01 06:36:04 INFO TaskSetManager: Starting task 46.0 in stage 24.0 (TID 356) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:04 INFO TaskSetManager: Finished task 43.0 in stage 24.0 (TID 353) in 3417 ms on 10.0.0.133 (executor driver) (43/200) -26/04/01 06:36:04 INFO Executor: Running task 46.0 in stage 24.0 (TID 356) -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:04 INFO Executor: Finished task 42.0 in stage 24.0 (TID 352). 8470 bytes result sent to driver -26/04/01 06:36:04 INFO TaskSetManager: Starting task 47.0 in stage 24.0 (TID 357) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:04 INFO Executor: Running task 47.0 in stage 24.0 (TID 357) -26/04/01 06:36:04 INFO TaskSetManager: Finished task 42.0 in stage 24.0 (TID 352) in 3429 ms on 10.0.0.133 (executor driver) (44/200) -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:06 INFO Executor: Finished task 44.0 in stage 24.0 (TID 354). 9201 bytes result sent to driver -26/04/01 06:36:06 INFO TaskSetManager: Starting task 48.0 in stage 24.0 (TID 358) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:06 INFO TaskSetManager: Finished task 44.0 in stage 24.0 (TID 354) in 3364 ms on 10.0.0.133 (executor driver) (45/200) -26/04/01 06:36:06 INFO Executor: Running task 48.0 in stage 24.0 (TID 358) -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:06 INFO Executor: Finished task 45.0 in stage 24.0 (TID 355). 9201 bytes result sent to driver -26/04/01 06:36:06 INFO TaskSetManager: Starting task 49.0 in stage 24.0 (TID 359) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:06 INFO Executor: Running task 49.0 in stage 24.0 (TID 359) -26/04/01 06:36:06 INFO TaskSetManager: Finished task 45.0 in stage 24.0 (TID 355) in 3366 ms on 10.0.0.133 (executor driver) (46/200) -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:07 INFO Executor: Finished task 47.0 in stage 24.0 (TID 357). 9201 bytes result sent to driver -26/04/01 06:36:07 INFO Executor: Finished task 46.0 in stage 24.0 (TID 356). 9201 bytes result sent to driver -26/04/01 06:36:07 INFO TaskSetManager: Starting task 50.0 in stage 24.0 (TID 360) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:07 INFO Executor: Running task 50.0 in stage 24.0 (TID 360) -26/04/01 06:36:07 INFO TaskSetManager: Starting task 51.0 in stage 24.0 (TID 361) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:07 INFO Executor: Running task 51.0 in stage 24.0 (TID 361) -26/04/01 06:36:07 INFO TaskSetManager: Finished task 47.0 in stage 24.0 (TID 357) in 3430 ms on 10.0.0.133 (executor driver) (47/200) -26/04/01 06:36:07 INFO TaskSetManager: Finished task 46.0 in stage 24.0 (TID 356) in 3435 ms on 10.0.0.133 (executor driver) (48/200) -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:10 INFO Executor: Finished task 48.0 in stage 24.0 (TID 358). 8470 bytes result sent to driver -26/04/01 06:36:10 INFO TaskSetManager: Starting task 52.0 in stage 24.0 (TID 362) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:10 INFO TaskSetManager: Finished task 48.0 in stage 24.0 (TID 358) in 3391 ms on 10.0.0.133 (executor driver) (49/200) -26/04/01 06:36:10 INFO Executor: Running task 52.0 in stage 24.0 (TID 362) -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:10 INFO Executor: Finished task 49.0 in stage 24.0 (TID 359). 8470 bytes result sent to driver -26/04/01 06:36:10 INFO TaskSetManager: Starting task 53.0 in stage 24.0 (TID 363) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:10 INFO TaskSetManager: Finished task 49.0 in stage 24.0 (TID 359) in 3384 ms on 10.0.0.133 (executor driver) (50/200) -26/04/01 06:36:10 INFO Executor: Running task 53.0 in stage 24.0 (TID 363) -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:10 INFO Executor: Finished task 51.0 in stage 24.0 (TID 361). 8470 bytes result sent to driver -26/04/01 06:36:10 INFO TaskSetManager: Starting task 54.0 in stage 24.0 (TID 364) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:10 INFO Executor: Running task 54.0 in stage 24.0 (TID 364) -26/04/01 06:36:10 INFO TaskSetManager: Finished task 51.0 in stage 24.0 (TID 361) in 3455 ms on 10.0.0.133 (executor driver) (51/200) -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:10 INFO Executor: Finished task 50.0 in stage 24.0 (TID 360). 8470 bytes result sent to driver -26/04/01 06:36:10 INFO TaskSetManager: Starting task 55.0 in stage 24.0 (TID 365) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:10 INFO TaskSetManager: Finished task 50.0 in stage 24.0 (TID 360) in 3473 ms on 10.0.0.133 (executor driver) (52/200) -26/04/01 06:36:10 INFO Executor: Running task 55.0 in stage 24.0 (TID 365) -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:13 INFO Executor: Finished task 52.0 in stage 24.0 (TID 362). 8470 bytes result sent to driver -26/04/01 06:36:13 INFO TaskSetManager: Starting task 56.0 in stage 24.0 (TID 366) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:13 INFO TaskSetManager: Finished task 52.0 in stage 24.0 (TID 362) in 3351 ms on 10.0.0.133 (executor driver) (53/200) -26/04/01 06:36:13 INFO Executor: Running task 56.0 in stage 24.0 (TID 366) -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:13 INFO Executor: Finished task 53.0 in stage 24.0 (TID 363). 8470 bytes result sent to driver -26/04/01 06:36:13 INFO TaskSetManager: Starting task 57.0 in stage 24.0 (TID 367) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:13 INFO TaskSetManager: Finished task 53.0 in stage 24.0 (TID 363) in 3376 ms on 10.0.0.133 (executor driver) (54/200) -26/04/01 06:36:13 INFO Executor: Running task 57.0 in stage 24.0 (TID 367) -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:14 INFO Executor: Finished task 54.0 in stage 24.0 (TID 364). 8470 bytes result sent to driver -26/04/01 06:36:14 INFO TaskSetManager: Starting task 58.0 in stage 24.0 (TID 368) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:14 INFO Executor: Running task 58.0 in stage 24.0 (TID 368) -26/04/01 06:36:14 INFO TaskSetManager: Finished task 54.0 in stage 24.0 (TID 364) in 3403 ms on 10.0.0.133 (executor driver) (55/200) -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:14 INFO Executor: Finished task 55.0 in stage 24.0 (TID 365). 8470 bytes result sent to driver -26/04/01 06:36:14 INFO TaskSetManager: Starting task 59.0 in stage 24.0 (TID 369) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:14 INFO TaskSetManager: Finished task 55.0 in stage 24.0 (TID 365) in 3403 ms on 10.0.0.133 (executor driver) (56/200) -26/04/01 06:36:14 INFO Executor: Running task 59.0 in stage 24.0 (TID 369) -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:16 INFO Executor: Finished task 56.0 in stage 24.0 (TID 366). 9201 bytes result sent to driver -26/04/01 06:36:16 INFO TaskSetManager: Starting task 60.0 in stage 24.0 (TID 370) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:16 INFO Executor: Running task 60.0 in stage 24.0 (TID 370) -26/04/01 06:36:16 INFO TaskSetManager: Finished task 56.0 in stage 24.0 (TID 366) in 3376 ms on 10.0.0.133 (executor driver) (57/200) -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:16 INFO Executor: Finished task 57.0 in stage 24.0 (TID 367). 9201 bytes result sent to driver -26/04/01 06:36:16 INFO TaskSetManager: Starting task 61.0 in stage 24.0 (TID 371) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:16 INFO TaskSetManager: Finished task 57.0 in stage 24.0 (TID 367) in 3376 ms on 10.0.0.133 (executor driver) (58/200) -26/04/01 06:36:16 INFO Executor: Running task 61.0 in stage 24.0 (TID 371) -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:17 INFO Executor: Finished task 59.0 in stage 24.0 (TID 369). 9201 bytes result sent to driver -26/04/01 06:36:17 INFO TaskSetManager: Starting task 62.0 in stage 24.0 (TID 372) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:17 INFO Executor: Running task 62.0 in stage 24.0 (TID 372) -26/04/01 06:36:17 INFO TaskSetManager: Finished task 59.0 in stage 24.0 (TID 369) in 3440 ms on 10.0.0.133 (executor driver) (59/200) -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:17 INFO Executor: Finished task 58.0 in stage 24.0 (TID 368). 9201 bytes result sent to driver -26/04/01 06:36:17 INFO TaskSetManager: Starting task 63.0 in stage 24.0 (TID 373) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:17 INFO Executor: Running task 63.0 in stage 24.0 (TID 373) -26/04/01 06:36:17 INFO TaskSetManager: Finished task 58.0 in stage 24.0 (TID 368) in 3463 ms on 10.0.0.133 (executor driver) (60/200) -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 6 (999.4 KiB) non-empty blocks including 6 (999.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 6 (999.4 KiB) non-empty blocks including 6 (999.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:20 INFO Executor: Finished task 60.0 in stage 24.0 (TID 370). 8470 bytes result sent to driver -26/04/01 06:36:20 INFO TaskSetManager: Starting task 64.0 in stage 24.0 (TID 374) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:20 INFO TaskSetManager: Finished task 60.0 in stage 24.0 (TID 370) in 3374 ms on 10.0.0.133 (executor driver) (61/200) -26/04/01 06:36:20 INFO Executor: Running task 64.0 in stage 24.0 (TID 374) -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.3 MiB) non-empty blocks including 208 (45.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.3 MiB) non-empty blocks including 208 (45.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:20 INFO Executor: Finished task 61.0 in stage 24.0 (TID 371). 8470 bytes result sent to driver -26/04/01 06:36:20 INFO TaskSetManager: Starting task 65.0 in stage 24.0 (TID 375) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:20 INFO TaskSetManager: Finished task 61.0 in stage 24.0 (TID 371) in 3373 ms on 10.0.0.133 (executor driver) (62/200) -26/04/01 06:36:20 INFO Executor: Running task 65.0 in stage 24.0 (TID 375) -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:21 INFO Executor: Finished task 63.0 in stage 24.0 (TID 373). 8470 bytes result sent to driver -26/04/01 06:36:21 INFO TaskSetManager: Starting task 66.0 in stage 24.0 (TID 376) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:21 INFO TaskSetManager: Finished task 63.0 in stage 24.0 (TID 373) in 3426 ms on 10.0.0.133 (executor driver) (63/200) -26/04/01 06:36:21 INFO Executor: Running task 66.0 in stage 24.0 (TID 376) -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:21 INFO Executor: Finished task 62.0 in stage 24.0 (TID 372). 8470 bytes result sent to driver -26/04/01 06:36:21 INFO TaskSetManager: Starting task 67.0 in stage 24.0 (TID 377) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:21 INFO TaskSetManager: Finished task 62.0 in stage 24.0 (TID 372) in 3442 ms on 10.0.0.133 (executor driver) (64/200) -26/04/01 06:36:21 INFO Executor: Running task 67.0 in stage 24.0 (TID 377) -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:23 INFO Executor: Finished task 64.0 in stage 24.0 (TID 374). 8470 bytes result sent to driver -26/04/01 06:36:23 INFO TaskSetManager: Starting task 68.0 in stage 24.0 (TID 378) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:23 INFO Executor: Running task 68.0 in stage 24.0 (TID 378) -26/04/01 06:36:23 INFO TaskSetManager: Finished task 64.0 in stage 24.0 (TID 374) in 3364 ms on 10.0.0.133 (executor driver) (65/200) -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:23 INFO Executor: Finished task 65.0 in stage 24.0 (TID 375). 8470 bytes result sent to driver -26/04/01 06:36:23 INFO TaskSetManager: Starting task 69.0 in stage 24.0 (TID 379) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:23 INFO TaskSetManager: Finished task 65.0 in stage 24.0 (TID 375) in 3368 ms on 10.0.0.133 (executor driver) (66/200) -26/04/01 06:36:23 INFO Executor: Running task 69.0 in stage 24.0 (TID 379) -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:24 INFO Executor: Finished task 67.0 in stage 24.0 (TID 377). 8470 bytes result sent to driver -26/04/01 06:36:24 INFO TaskSetManager: Starting task 70.0 in stage 24.0 (TID 380) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:24 INFO Executor: Running task 70.0 in stage 24.0 (TID 380) -26/04/01 06:36:24 INFO TaskSetManager: Finished task 67.0 in stage 24.0 (TID 377) in 3417 ms on 10.0.0.133 (executor driver) (67/200) -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:24 INFO Executor: Finished task 66.0 in stage 24.0 (TID 376). 8470 bytes result sent to driver -26/04/01 06:36:24 INFO TaskSetManager: Starting task 71.0 in stage 24.0 (TID 381) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:24 INFO Executor: Running task 71.0 in stage 24.0 (TID 381) -26/04/01 06:36:24 INFO TaskSetManager: Finished task 66.0 in stage 24.0 (TID 376) in 3439 ms on 10.0.0.133 (executor driver) (68/200) -26/04/01 06:36:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:26 INFO Executor: Finished task 68.0 in stage 24.0 (TID 378). 9201 bytes result sent to driver -26/04/01 06:36:26 INFO TaskSetManager: Starting task 72.0 in stage 24.0 (TID 382) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:26 INFO TaskSetManager: Finished task 68.0 in stage 24.0 (TID 378) in 3367 ms on 10.0.0.133 (executor driver) (69/200) -26/04/01 06:36:26 INFO Executor: Running task 72.0 in stage 24.0 (TID 382) -26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:27 INFO Executor: Finished task 69.0 in stage 24.0 (TID 379). 9201 bytes result sent to driver -26/04/01 06:36:27 INFO TaskSetManager: Starting task 73.0 in stage 24.0 (TID 383) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:27 INFO Executor: Running task 73.0 in stage 24.0 (TID 383) -26/04/01 06:36:27 INFO TaskSetManager: Finished task 69.0 in stage 24.0 (TID 379) in 3359 ms on 10.0.0.133 (executor driver) (70/200) -26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:28 INFO Executor: Finished task 70.0 in stage 24.0 (TID 380). 9201 bytes result sent to driver -26/04/01 06:36:28 INFO TaskSetManager: Starting task 74.0 in stage 24.0 (TID 384) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:28 INFO TaskSetManager: Finished task 70.0 in stage 24.0 (TID 380) in 3419 ms on 10.0.0.133 (executor driver) (71/200) -26/04/01 06:36:28 INFO Executor: Running task 74.0 in stage 24.0 (TID 384) -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:28 INFO Executor: Finished task 71.0 in stage 24.0 (TID 381). 9201 bytes result sent to driver -26/04/01 06:36:28 INFO TaskSetManager: Starting task 75.0 in stage 24.0 (TID 385) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:28 INFO Executor: Running task 75.0 in stage 24.0 (TID 385) -26/04/01 06:36:28 INFO TaskSetManager: Finished task 71.0 in stage 24.0 (TID 381) in 3415 ms on 10.0.0.133 (executor driver) (72/200) -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:30 INFO Executor: Finished task 72.0 in stage 24.0 (TID 382). 8470 bytes result sent to driver -26/04/01 06:36:30 INFO TaskSetManager: Starting task 76.0 in stage 24.0 (TID 386) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:30 INFO Executor: Running task 76.0 in stage 24.0 (TID 386) -26/04/01 06:36:30 INFO TaskSetManager: Finished task 72.0 in stage 24.0 (TID 382) in 3375 ms on 10.0.0.133 (executor driver) (73/200) -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:30 INFO Executor: Finished task 73.0 in stage 24.0 (TID 383). 8470 bytes result sent to driver -26/04/01 06:36:30 INFO TaskSetManager: Starting task 77.0 in stage 24.0 (TID 387) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:30 INFO TaskSetManager: Finished task 73.0 in stage 24.0 (TID 383) in 3370 ms on 10.0.0.133 (executor driver) (74/200) -26/04/01 06:36:30 INFO Executor: Running task 77.0 in stage 24.0 (TID 387) -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:31 INFO Executor: Finished task 74.0 in stage 24.0 (TID 384). 8470 bytes result sent to driver -26/04/01 06:36:31 INFO TaskSetManager: Starting task 78.0 in stage 24.0 (TID 388) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:31 INFO Executor: Running task 78.0 in stage 24.0 (TID 388) -26/04/01 06:36:31 INFO TaskSetManager: Finished task 74.0 in stage 24.0 (TID 384) in 3444 ms on 10.0.0.133 (executor driver) (75/200) -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:31 INFO Executor: Finished task 75.0 in stage 24.0 (TID 385). 8470 bytes result sent to driver -26/04/01 06:36:31 INFO TaskSetManager: Starting task 79.0 in stage 24.0 (TID 389) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:31 INFO Executor: Running task 79.0 in stage 24.0 (TID 389) -26/04/01 06:36:31 INFO TaskSetManager: Finished task 75.0 in stage 24.0 (TID 385) in 3450 ms on 10.0.0.133 (executor driver) (76/200) -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:33 INFO Executor: Finished task 76.0 in stage 24.0 (TID 386). 8470 bytes result sent to driver -26/04/01 06:36:33 INFO TaskSetManager: Starting task 80.0 in stage 24.0 (TID 390) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:33 INFO Executor: Running task 80.0 in stage 24.0 (TID 390) -26/04/01 06:36:33 INFO TaskSetManager: Finished task 76.0 in stage 24.0 (TID 386) in 3354 ms on 10.0.0.133 (executor driver) (77/200) -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:33 INFO Executor: Finished task 77.0 in stage 24.0 (TID 387). 8470 bytes result sent to driver -26/04/01 06:36:33 INFO TaskSetManager: Starting task 81.0 in stage 24.0 (TID 391) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:33 INFO TaskSetManager: Finished task 77.0 in stage 24.0 (TID 387) in 3356 ms on 10.0.0.133 (executor driver) (78/200) -26/04/01 06:36:33 INFO Executor: Running task 81.0 in stage 24.0 (TID 391) -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:34 INFO Executor: Finished task 79.0 in stage 24.0 (TID 389). 9201 bytes result sent to driver -26/04/01 06:36:34 INFO TaskSetManager: Starting task 82.0 in stage 24.0 (TID 392) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:34 INFO Executor: Running task 82.0 in stage 24.0 (TID 392) -26/04/01 06:36:34 INFO TaskSetManager: Finished task 79.0 in stage 24.0 (TID 389) in 3408 ms on 10.0.0.133 (executor driver) (79/200) -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:34 INFO Executor: Finished task 78.0 in stage 24.0 (TID 388). 9201 bytes result sent to driver -26/04/01 06:36:34 INFO TaskSetManager: Starting task 83.0 in stage 24.0 (TID 393) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:34 INFO Executor: Running task 83.0 in stage 24.0 (TID 393) -26/04/01 06:36:34 INFO TaskSetManager: Finished task 78.0 in stage 24.0 (TID 388) in 3443 ms on 10.0.0.133 (executor driver) (80/200) -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:37 INFO Executor: Finished task 80.0 in stage 24.0 (TID 390). 9201 bytes result sent to driver -26/04/01 06:36:37 INFO TaskSetManager: Starting task 84.0 in stage 24.0 (TID 394) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:37 INFO Executor: Running task 84.0 in stage 24.0 (TID 394) -26/04/01 06:36:37 INFO TaskSetManager: Finished task 80.0 in stage 24.0 (TID 390) in 3382 ms on 10.0.0.133 (executor driver) (81/200) -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:37 INFO Executor: Finished task 81.0 in stage 24.0 (TID 391). 9201 bytes result sent to driver -26/04/01 06:36:37 INFO TaskSetManager: Starting task 85.0 in stage 24.0 (TID 395) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:37 INFO Executor: Running task 85.0 in stage 24.0 (TID 395) -26/04/01 06:36:37 INFO TaskSetManager: Finished task 81.0 in stage 24.0 (TID 391) in 3370 ms on 10.0.0.133 (executor driver) (82/200) -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:38 INFO Executor: Finished task 82.0 in stage 24.0 (TID 392). 8470 bytes result sent to driver -26/04/01 06:36:38 INFO TaskSetManager: Starting task 86.0 in stage 24.0 (TID 396) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:38 INFO Executor: Running task 86.0 in stage 24.0 (TID 396) -26/04/01 06:36:38 INFO TaskSetManager: Finished task 82.0 in stage 24.0 (TID 392) in 3423 ms on 10.0.0.133 (executor driver) (83/200) -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:38 INFO Executor: Finished task 83.0 in stage 24.0 (TID 393). 8470 bytes result sent to driver -26/04/01 06:36:38 INFO TaskSetManager: Starting task 87.0 in stage 24.0 (TID 397) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:38 INFO Executor: Running task 87.0 in stage 24.0 (TID 397) -26/04/01 06:36:38 INFO TaskSetManager: Finished task 83.0 in stage 24.0 (TID 393) in 3414 ms on 10.0.0.133 (executor driver) (84/200) -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:40 INFO Executor: Finished task 84.0 in stage 24.0 (TID 394). 8470 bytes result sent to driver -26/04/01 06:36:40 INFO TaskSetManager: Starting task 88.0 in stage 24.0 (TID 398) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:40 INFO Executor: Running task 88.0 in stage 24.0 (TID 398) -26/04/01 06:36:40 INFO TaskSetManager: Finished task 84.0 in stage 24.0 (TID 394) in 3382 ms on 10.0.0.133 (executor driver) (85/200) -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:40 INFO Executor: Finished task 85.0 in stage 24.0 (TID 395). 8470 bytes result sent to driver -26/04/01 06:36:40 INFO TaskSetManager: Starting task 89.0 in stage 24.0 (TID 399) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:40 INFO Executor: Running task 89.0 in stage 24.0 (TID 399) -26/04/01 06:36:40 INFO TaskSetManager: Finished task 85.0 in stage 24.0 (TID 395) in 3381 ms on 10.0.0.133 (executor driver) (86/200) -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:41 INFO Executor: Finished task 87.0 in stage 24.0 (TID 397). 8470 bytes result sent to driver -26/04/01 06:36:41 INFO TaskSetManager: Starting task 90.0 in stage 24.0 (TID 400) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:41 INFO Executor: Running task 90.0 in stage 24.0 (TID 400) -26/04/01 06:36:41 INFO TaskSetManager: Finished task 87.0 in stage 24.0 (TID 397) in 3422 ms on 10.0.0.133 (executor driver) (87/200) -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:41 INFO Executor: Finished task 86.0 in stage 24.0 (TID 396). 8470 bytes result sent to driver -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:41 INFO TaskSetManager: Starting task 91.0 in stage 24.0 (TID 401) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:41 INFO Executor: Running task 91.0 in stage 24.0 (TID 401) -26/04/01 06:36:41 INFO TaskSetManager: Finished task 86.0 in stage 24.0 (TID 396) in 3441 ms on 10.0.0.133 (executor driver) (88/200) -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:43 INFO Executor: Finished task 88.0 in stage 24.0 (TID 398). 8470 bytes result sent to driver -26/04/01 06:36:43 INFO TaskSetManager: Starting task 92.0 in stage 24.0 (TID 402) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:43 INFO Executor: Running task 92.0 in stage 24.0 (TID 402) -26/04/01 06:36:43 INFO TaskSetManager: Finished task 88.0 in stage 24.0 (TID 398) in 3376 ms on 10.0.0.133 (executor driver) (89/200) -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:43 INFO Executor: Finished task 89.0 in stage 24.0 (TID 399). 8470 bytes result sent to driver -26/04/01 06:36:43 INFO TaskSetManager: Starting task 93.0 in stage 24.0 (TID 403) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:43 INFO Executor: Running task 93.0 in stage 24.0 (TID 403) -26/04/01 06:36:43 INFO TaskSetManager: Finished task 89.0 in stage 24.0 (TID 399) in 3384 ms on 10.0.0.133 (executor driver) (90/200) -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:45 INFO Executor: Finished task 91.0 in stage 24.0 (TID 401). 9201 bytes result sent to driver -26/04/01 06:36:45 INFO TaskSetManager: Starting task 94.0 in stage 24.0 (TID 404) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:45 INFO Executor: Running task 94.0 in stage 24.0 (TID 404) -26/04/01 06:36:45 INFO TaskSetManager: Finished task 91.0 in stage 24.0 (TID 401) in 3424 ms on 10.0.0.133 (executor driver) (91/200) -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:45 INFO Executor: Finished task 90.0 in stage 24.0 (TID 400). 9201 bytes result sent to driver -26/04/01 06:36:45 INFO TaskSetManager: Starting task 95.0 in stage 24.0 (TID 405) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:45 INFO Executor: Running task 95.0 in stage 24.0 (TID 405) -26/04/01 06:36:45 INFO TaskSetManager: Finished task 90.0 in stage 24.0 (TID 400) in 3434 ms on 10.0.0.133 (executor driver) (92/200) -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:47 INFO Executor: Finished task 92.0 in stage 24.0 (TID 402). 9201 bytes result sent to driver -26/04/01 06:36:47 INFO TaskSetManager: Starting task 96.0 in stage 24.0 (TID 406) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:47 INFO Executor: Running task 96.0 in stage 24.0 (TID 406) -26/04/01 06:36:47 INFO TaskSetManager: Finished task 92.0 in stage 24.0 (TID 402) in 3380 ms on 10.0.0.133 (executor driver) (93/200) -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:47 INFO Executor: Finished task 93.0 in stage 24.0 (TID 403). 9201 bytes result sent to driver -26/04/01 06:36:47 INFO TaskSetManager: Starting task 97.0 in stage 24.0 (TID 407) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:47 INFO Executor: Running task 97.0 in stage 24.0 (TID 407) -26/04/01 06:36:47 INFO TaskSetManager: Finished task 93.0 in stage 24.0 (TID 403) in 3390 ms on 10.0.0.133 (executor driver) (94/200) -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:48 INFO Executor: Finished task 94.0 in stage 24.0 (TID 404). 8470 bytes result sent to driver -26/04/01 06:36:48 INFO Executor: Finished task 95.0 in stage 24.0 (TID 405). 8470 bytes result sent to driver -26/04/01 06:36:48 INFO TaskSetManager: Starting task 98.0 in stage 24.0 (TID 408) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:48 INFO Executor: Running task 98.0 in stage 24.0 (TID 408) -26/04/01 06:36:48 INFO TaskSetManager: Starting task 99.0 in stage 24.0 (TID 409) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:48 INFO TaskSetManager: Finished task 94.0 in stage 24.0 (TID 404) in 3442 ms on 10.0.0.133 (executor driver) (95/200) -26/04/01 06:36:48 INFO Executor: Running task 99.0 in stage 24.0 (TID 409) -26/04/01 06:36:48 INFO TaskSetManager: Finished task 95.0 in stage 24.0 (TID 405) in 3440 ms on 10.0.0.133 (executor driver) (96/200) -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:50 INFO Executor: Finished task 96.0 in stage 24.0 (TID 406). 8470 bytes result sent to driver -26/04/01 06:36:50 INFO TaskSetManager: Starting task 100.0 in stage 24.0 (TID 410) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:50 INFO Executor: Running task 100.0 in stage 24.0 (TID 410) -26/04/01 06:36:50 INFO TaskSetManager: Finished task 96.0 in stage 24.0 (TID 406) in 3380 ms on 10.0.0.133 (executor driver) (97/200) -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:50 INFO Executor: Finished task 97.0 in stage 24.0 (TID 407). 8470 bytes result sent to driver -26/04/01 06:36:50 INFO TaskSetManager: Starting task 101.0 in stage 24.0 (TID 411) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:50 INFO Executor: Running task 101.0 in stage 24.0 (TID 411) -26/04/01 06:36:50 INFO TaskSetManager: Finished task 97.0 in stage 24.0 (TID 407) in 3387 ms on 10.0.0.133 (executor driver) (98/200) -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:52 INFO Executor: Finished task 98.0 in stage 24.0 (TID 408). 8470 bytes result sent to driver -26/04/01 06:36:52 INFO TaskSetManager: Starting task 102.0 in stage 24.0 (TID 412) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:52 INFO Executor: Running task 102.0 in stage 24.0 (TID 412) -26/04/01 06:36:52 INFO TaskSetManager: Finished task 98.0 in stage 24.0 (TID 408) in 3441 ms on 10.0.0.133 (executor driver) (99/200) -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:52 INFO Executor: Finished task 99.0 in stage 24.0 (TID 409). 8470 bytes result sent to driver -26/04/01 06:36:52 INFO TaskSetManager: Starting task 103.0 in stage 24.0 (TID 413) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:52 INFO TaskSetManager: Finished task 99.0 in stage 24.0 (TID 409) in 3446 ms on 10.0.0.133 (executor driver) (100/200) -26/04/01 06:36:52 INFO Executor: Running task 103.0 in stage 24.0 (TID 413) -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:53 INFO Executor: Finished task 100.0 in stage 24.0 (TID 410). 8470 bytes result sent to driver -26/04/01 06:36:53 INFO TaskSetManager: Starting task 104.0 in stage 24.0 (TID 414) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:53 INFO Executor: Running task 104.0 in stage 24.0 (TID 414) -26/04/01 06:36:53 INFO TaskSetManager: Finished task 100.0 in stage 24.0 (TID 410) in 3382 ms on 10.0.0.133 (executor driver) (101/200) -26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:54 INFO Executor: Finished task 101.0 in stage 24.0 (TID 411). 8470 bytes result sent to driver -26/04/01 06:36:54 INFO TaskSetManager: Starting task 105.0 in stage 24.0 (TID 415) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:54 INFO Executor: Running task 105.0 in stage 24.0 (TID 415) -26/04/01 06:36:54 INFO TaskSetManager: Finished task 101.0 in stage 24.0 (TID 411) in 3356 ms on 10.0.0.133 (executor driver) (102/200) -26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:55 INFO Executor: Finished task 103.0 in stage 24.0 (TID 413). 9201 bytes result sent to driver -26/04/01 06:36:55 INFO TaskSetManager: Starting task 106.0 in stage 24.0 (TID 416) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:55 INFO Executor: Running task 106.0 in stage 24.0 (TID 416) -26/04/01 06:36:55 INFO TaskSetManager: Finished task 103.0 in stage 24.0 (TID 413) in 3428 ms on 10.0.0.133 (executor driver) (103/200) -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:55 INFO Executor: Finished task 102.0 in stage 24.0 (TID 412). 9201 bytes result sent to driver -26/04/01 06:36:55 INFO TaskSetManager: Starting task 107.0 in stage 24.0 (TID 417) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:55 INFO TaskSetManager: Finished task 102.0 in stage 24.0 (TID 412) in 3443 ms on 10.0.0.133 (executor driver) (104/200) -26/04/01 06:36:55 INFO Executor: Running task 107.0 in stage 24.0 (TID 417) -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:57 INFO Executor: Finished task 104.0 in stage 24.0 (TID 414). 9201 bytes result sent to driver -26/04/01 06:36:57 INFO TaskSetManager: Starting task 108.0 in stage 24.0 (TID 418) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:57 INFO Executor: Running task 108.0 in stage 24.0 (TID 418) -26/04/01 06:36:57 INFO TaskSetManager: Finished task 104.0 in stage 24.0 (TID 414) in 3360 ms on 10.0.0.133 (executor driver) (105/200) -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:57 INFO Executor: Finished task 105.0 in stage 24.0 (TID 415). 9201 bytes result sent to driver -26/04/01 06:36:57 INFO TaskSetManager: Starting task 109.0 in stage 24.0 (TID 419) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:57 INFO Executor: Running task 109.0 in stage 24.0 (TID 419) -26/04/01 06:36:57 INFO TaskSetManager: Finished task 105.0 in stage 24.0 (TID 415) in 3374 ms on 10.0.0.133 (executor driver) (106/200) -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:58 INFO Executor: Finished task 106.0 in stage 24.0 (TID 416). 8470 bytes result sent to driver -26/04/01 06:36:58 INFO TaskSetManager: Starting task 110.0 in stage 24.0 (TID 420) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:58 INFO Executor: Running task 110.0 in stage 24.0 (TID 420) -26/04/01 06:36:58 INFO TaskSetManager: Finished task 106.0 in stage 24.0 (TID 416) in 3441 ms on 10.0.0.133 (executor driver) (107/200) -26/04/01 06:36:58 INFO Executor: Finished task 107.0 in stage 24.0 (TID 417). 8470 bytes result sent to driver -26/04/01 06:36:58 INFO TaskSetManager: Starting task 111.0 in stage 24.0 (TID 421) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9428 bytes) -26/04/01 06:36:58 INFO Executor: Running task 111.0 in stage 24.0 (TID 421) -26/04/01 06:36:58 INFO TaskSetManager: Finished task 107.0 in stage 24.0 (TID 417) in 3433 ms on 10.0.0.133 (executor driver) (108/200) -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:36:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:36:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:00 INFO Executor: Finished task 108.0 in stage 24.0 (TID 418). 8470 bytes result sent to driver -26/04/01 06:37:00 INFO TaskSetManager: Starting task 112.0 in stage 24.0 (TID 422) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:00 INFO Executor: Running task 112.0 in stage 24.0 (TID 422) -26/04/01 06:37:00 INFO TaskSetManager: Finished task 108.0 in stage 24.0 (TID 418) in 3393 ms on 10.0.0.133 (executor driver) (109/200) -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:00 INFO Executor: Finished task 109.0 in stage 24.0 (TID 419). 8470 bytes result sent to driver -26/04/01 06:37:00 INFO TaskSetManager: Starting task 113.0 in stage 24.0 (TID 423) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:00 INFO Executor: Running task 113.0 in stage 24.0 (TID 423) -26/04/01 06:37:00 INFO TaskSetManager: Finished task 109.0 in stage 24.0 (TID 419) in 3384 ms on 10.0.0.133 (executor driver) (110/200) -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:02 INFO Executor: Finished task 111.0 in stage 24.0 (TID 421). 8470 bytes result sent to driver -26/04/01 06:37:02 INFO TaskSetManager: Starting task 114.0 in stage 24.0 (TID 424) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:02 INFO Executor: Running task 114.0 in stage 24.0 (TID 424) -26/04/01 06:37:02 INFO TaskSetManager: Finished task 111.0 in stage 24.0 (TID 421) in 3434 ms on 10.0.0.133 (executor driver) (111/200) -26/04/01 06:37:02 INFO Executor: Finished task 110.0 in stage 24.0 (TID 420). 8470 bytes result sent to driver -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:02 INFO TaskSetManager: Starting task 115.0 in stage 24.0 (TID 425) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:02 INFO Executor: Running task 115.0 in stage 24.0 (TID 425) -26/04/01 06:37:02 INFO TaskSetManager: Finished task 110.0 in stage 24.0 (TID 420) in 3436 ms on 10.0.0.133 (executor driver) (112/200) -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:04 INFO Executor: Finished task 112.0 in stage 24.0 (TID 422). 8470 bytes result sent to driver -26/04/01 06:37:04 INFO TaskSetManager: Starting task 116.0 in stage 24.0 (TID 426) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:04 INFO Executor: Running task 116.0 in stage 24.0 (TID 426) -26/04/01 06:37:04 INFO TaskSetManager: Finished task 112.0 in stage 24.0 (TID 422) in 3383 ms on 10.0.0.133 (executor driver) (113/200) -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:04 INFO Executor: Finished task 113.0 in stage 24.0 (TID 423). 8470 bytes result sent to driver -26/04/01 06:37:04 INFO TaskSetManager: Starting task 117.0 in stage 24.0 (TID 427) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:04 INFO TaskSetManager: Finished task 113.0 in stage 24.0 (TID 423) in 3375 ms on 10.0.0.133 (executor driver) (114/200) -26/04/01 06:37:04 INFO Executor: Running task 117.0 in stage 24.0 (TID 427) -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:05 INFO Executor: Finished task 114.0 in stage 24.0 (TID 424). 9201 bytes result sent to driver -26/04/01 06:37:05 INFO TaskSetManager: Starting task 118.0 in stage 24.0 (TID 428) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:05 INFO Executor: Running task 118.0 in stage 24.0 (TID 428) -26/04/01 06:37:05 INFO TaskSetManager: Finished task 114.0 in stage 24.0 (TID 424) in 3436 ms on 10.0.0.133 (executor driver) (115/200) -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1017.0 KiB) non-empty blocks including 6 (1017.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:05 INFO Executor: Finished task 115.0 in stage 24.0 (TID 425). 9201 bytes result sent to driver -26/04/01 06:37:05 INFO TaskSetManager: Starting task 119.0 in stage 24.0 (TID 429) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:05 INFO TaskSetManager: Finished task 115.0 in stage 24.0 (TID 425) in 3437 ms on 10.0.0.133 (executor driver) (116/200) -26/04/01 06:37:05 INFO Executor: Running task 119.0 in stage 24.0 (TID 429) -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1017.0 KiB) non-empty blocks including 6 (1017.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:07 INFO Executor: Finished task 116.0 in stage 24.0 (TID 426). 9201 bytes result sent to driver -26/04/01 06:37:07 INFO TaskSetManager: Starting task 120.0 in stage 24.0 (TID 430) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:07 INFO TaskSetManager: Finished task 116.0 in stage 24.0 (TID 426) in 3373 ms on 10.0.0.133 (executor driver) (117/200) -26/04/01 06:37:07 INFO Executor: Running task 120.0 in stage 24.0 (TID 430) -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:07 INFO Executor: Finished task 117.0 in stage 24.0 (TID 427). 9201 bytes result sent to driver -26/04/01 06:37:07 INFO TaskSetManager: Starting task 121.0 in stage 24.0 (TID 431) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:07 INFO TaskSetManager: Finished task 117.0 in stage 24.0 (TID 427) in 3379 ms on 10.0.0.133 (executor driver) (118/200) -26/04/01 06:37:07 INFO Executor: Running task 121.0 in stage 24.0 (TID 431) -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:09 INFO Executor: Finished task 119.0 in stage 24.0 (TID 429). 8470 bytes result sent to driver -26/04/01 06:37:09 INFO TaskSetManager: Starting task 122.0 in stage 24.0 (TID 432) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:09 INFO Executor: Running task 122.0 in stage 24.0 (TID 432) -26/04/01 06:37:09 INFO TaskSetManager: Finished task 119.0 in stage 24.0 (TID 429) in 3458 ms on 10.0.0.133 (executor driver) (119/200) -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:09 INFO Executor: Finished task 118.0 in stage 24.0 (TID 428). 8470 bytes result sent to driver -26/04/01 06:37:09 INFO TaskSetManager: Starting task 123.0 in stage 24.0 (TID 433) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:09 INFO Executor: Running task 123.0 in stage 24.0 (TID 433) -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:09 INFO TaskSetManager: Finished task 118.0 in stage 24.0 (TID 428) in 3468 ms on 10.0.0.133 (executor driver) (120/200) -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:10 INFO Executor: Finished task 120.0 in stage 24.0 (TID 430). 8470 bytes result sent to driver -26/04/01 06:37:10 INFO TaskSetManager: Starting task 124.0 in stage 24.0 (TID 434) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:10 INFO Executor: Running task 124.0 in stage 24.0 (TID 434) -26/04/01 06:37:10 INFO TaskSetManager: Finished task 120.0 in stage 24.0 (TID 430) in 3370 ms on 10.0.0.133 (executor driver) (121/200) -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:10 INFO Executor: Finished task 121.0 in stage 24.0 (TID 431). 8470 bytes result sent to driver -26/04/01 06:37:10 INFO TaskSetManager: Starting task 125.0 in stage 24.0 (TID 435) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:10 INFO Executor: Running task 125.0 in stage 24.0 (TID 435) -26/04/01 06:37:10 INFO TaskSetManager: Finished task 121.0 in stage 24.0 (TID 431) in 3367 ms on 10.0.0.133 (executor driver) (122/200) -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:12 INFO Executor: Finished task 122.0 in stage 24.0 (TID 432). 8470 bytes result sent to driver -26/04/01 06:37:12 INFO TaskSetManager: Starting task 126.0 in stage 24.0 (TID 436) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:12 INFO Executor: Running task 126.0 in stage 24.0 (TID 436) -26/04/01 06:37:12 INFO TaskSetManager: Finished task 122.0 in stage 24.0 (TID 432) in 3460 ms on 10.0.0.133 (executor driver) (123/200) -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:12 INFO Executor: Finished task 123.0 in stage 24.0 (TID 433). 8470 bytes result sent to driver -26/04/01 06:37:12 INFO TaskSetManager: Starting task 127.0 in stage 24.0 (TID 437) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:12 INFO Executor: Running task 127.0 in stage 24.0 (TID 437) -26/04/01 06:37:12 INFO TaskSetManager: Finished task 123.0 in stage 24.0 (TID 433) in 3454 ms on 10.0.0.133 (executor driver) (124/200) -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:14 INFO Executor: Finished task 124.0 in stage 24.0 (TID 434). 8470 bytes result sent to driver -26/04/01 06:37:14 INFO TaskSetManager: Starting task 128.0 in stage 24.0 (TID 438) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:14 INFO TaskSetManager: Finished task 124.0 in stage 24.0 (TID 434) in 3421 ms on 10.0.0.133 (executor driver) (125/200) -26/04/01 06:37:14 INFO Executor: Running task 128.0 in stage 24.0 (TID 438) -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:14 INFO Executor: Finished task 125.0 in stage 24.0 (TID 435). 8470 bytes result sent to driver -26/04/01 06:37:14 INFO TaskSetManager: Starting task 129.0 in stage 24.0 (TID 439) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:14 INFO TaskSetManager: Finished task 125.0 in stage 24.0 (TID 435) in 3397 ms on 10.0.0.133 (executor driver) (126/200) -26/04/01 06:37:14 INFO Executor: Running task 129.0 in stage 24.0 (TID 439) -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:16 INFO Executor: Finished task 126.0 in stage 24.0 (TID 436). 9201 bytes result sent to driver -26/04/01 06:37:16 INFO TaskSetManager: Starting task 130.0 in stage 24.0 (TID 440) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:16 INFO Executor: Running task 130.0 in stage 24.0 (TID 440) -26/04/01 06:37:16 INFO TaskSetManager: Finished task 126.0 in stage 24.0 (TID 436) in 3444 ms on 10.0.0.133 (executor driver) (127/200) -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:16 INFO Executor: Finished task 127.0 in stage 24.0 (TID 437). 9201 bytes result sent to driver -26/04/01 06:37:16 INFO TaskSetManager: Starting task 131.0 in stage 24.0 (TID 441) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:16 INFO Executor: Running task 131.0 in stage 24.0 (TID 441) -26/04/01 06:37:16 INFO TaskSetManager: Finished task 127.0 in stage 24.0 (TID 437) in 3446 ms on 10.0.0.133 (executor driver) (128/200) -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:17 INFO Executor: Finished task 128.0 in stage 24.0 (TID 438). 9201 bytes result sent to driver -26/04/01 06:37:17 INFO TaskSetManager: Starting task 132.0 in stage 24.0 (TID 442) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:17 INFO Executor: Running task 132.0 in stage 24.0 (TID 442) -26/04/01 06:37:17 INFO TaskSetManager: Finished task 128.0 in stage 24.0 (TID 438) in 3411 ms on 10.0.0.133 (executor driver) (129/200) -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:17 INFO Executor: Finished task 129.0 in stage 24.0 (TID 439). 9201 bytes result sent to driver -26/04/01 06:37:17 INFO TaskSetManager: Starting task 133.0 in stage 24.0 (TID 443) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:17 INFO TaskSetManager: Finished task 129.0 in stage 24.0 (TID 439) in 3385 ms on 10.0.0.133 (executor driver) (130/200) -26/04/01 06:37:17 INFO Executor: Running task 133.0 in stage 24.0 (TID 443) -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:19 INFO Executor: Finished task 131.0 in stage 24.0 (TID 441). 8470 bytes result sent to driver -26/04/01 06:37:19 INFO TaskSetManager: Starting task 134.0 in stage 24.0 (TID 444) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:19 INFO TaskSetManager: Finished task 131.0 in stage 24.0 (TID 441) in 3441 ms on 10.0.0.133 (executor driver) (131/200) -26/04/01 06:37:19 INFO Executor: Running task 134.0 in stage 24.0 (TID 444) -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:19 INFO Executor: Finished task 130.0 in stage 24.0 (TID 440). 8470 bytes result sent to driver -26/04/01 06:37:19 INFO TaskSetManager: Starting task 135.0 in stage 24.0 (TID 445) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:19 INFO Executor: Running task 135.0 in stage 24.0 (TID 445) -26/04/01 06:37:19 INFO TaskSetManager: Finished task 130.0 in stage 24.0 (TID 440) in 3452 ms on 10.0.0.133 (executor driver) (132/200) -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:21 INFO Executor: Finished task 132.0 in stage 24.0 (TID 442). 8470 bytes result sent to driver -26/04/01 06:37:21 INFO TaskSetManager: Starting task 136.0 in stage 24.0 (TID 446) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:21 INFO Executor: Running task 136.0 in stage 24.0 (TID 446) -26/04/01 06:37:21 INFO TaskSetManager: Finished task 132.0 in stage 24.0 (TID 442) in 3406 ms on 10.0.0.133 (executor driver) (133/200) -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:21 INFO Executor: Finished task 133.0 in stage 24.0 (TID 443). 8470 bytes result sent to driver -26/04/01 06:37:21 INFO TaskSetManager: Starting task 137.0 in stage 24.0 (TID 447) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:21 INFO Executor: Running task 137.0 in stage 24.0 (TID 447) -26/04/01 06:37:21 INFO TaskSetManager: Finished task 133.0 in stage 24.0 (TID 443) in 3394 ms on 10.0.0.133 (executor driver) (134/200) -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:23 INFO Executor: Finished task 135.0 in stage 24.0 (TID 445). 8470 bytes result sent to driver -26/04/01 06:37:23 INFO TaskSetManager: Starting task 138.0 in stage 24.0 (TID 448) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:23 INFO Executor: Running task 138.0 in stage 24.0 (TID 448) -26/04/01 06:37:23 INFO Executor: Finished task 134.0 in stage 24.0 (TID 444). 8470 bytes result sent to driver -26/04/01 06:37:23 INFO TaskSetManager: Finished task 135.0 in stage 24.0 (TID 445) in 3444 ms on 10.0.0.133 (executor driver) (135/200) -26/04/01 06:37:23 INFO TaskSetManager: Starting task 139.0 in stage 24.0 (TID 449) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:23 INFO Executor: Running task 139.0 in stage 24.0 (TID 449) -26/04/01 06:37:23 INFO TaskSetManager: Finished task 134.0 in stage 24.0 (TID 444) in 3450 ms on 10.0.0.133 (executor driver) (136/200) -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:24 INFO Executor: Finished task 136.0 in stage 24.0 (TID 446). 8470 bytes result sent to driver -26/04/01 06:37:24 INFO TaskSetManager: Starting task 140.0 in stage 24.0 (TID 450) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:24 INFO Executor: Running task 140.0 in stage 24.0 (TID 450) -26/04/01 06:37:24 INFO TaskSetManager: Finished task 136.0 in stage 24.0 (TID 446) in 3415 ms on 10.0.0.133 (executor driver) (137/200) -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:24 INFO Executor: Finished task 137.0 in stage 24.0 (TID 447). 8470 bytes result sent to driver -26/04/01 06:37:24 INFO TaskSetManager: Starting task 141.0 in stage 24.0 (TID 451) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:24 INFO Executor: Running task 141.0 in stage 24.0 (TID 451) -26/04/01 06:37:24 INFO TaskSetManager: Finished task 137.0 in stage 24.0 (TID 447) in 3403 ms on 10.0.0.133 (executor driver) (138/200) -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:26 INFO Executor: Finished task 138.0 in stage 24.0 (TID 448). 9201 bytes result sent to driver -26/04/01 06:37:26 INFO TaskSetManager: Starting task 142.0 in stage 24.0 (TID 452) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:26 INFO Executor: Running task 142.0 in stage 24.0 (TID 452) -26/04/01 06:37:26 INFO TaskSetManager: Finished task 138.0 in stage 24.0 (TID 448) in 3436 ms on 10.0.0.133 (executor driver) (139/200) -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:26 INFO Executor: Finished task 139.0 in stage 24.0 (TID 449). 9201 bytes result sent to driver -26/04/01 06:37:26 INFO TaskSetManager: Starting task 143.0 in stage 24.0 (TID 453) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:26 INFO TaskSetManager: Finished task 139.0 in stage 24.0 (TID 449) in 3444 ms on 10.0.0.133 (executor driver) (140/200) -26/04/01 06:37:26 INFO Executor: Running task 143.0 in stage 24.0 (TID 453) -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:27 INFO Executor: Finished task 140.0 in stage 24.0 (TID 450). 9201 bytes result sent to driver -26/04/01 06:37:27 INFO TaskSetManager: Starting task 144.0 in stage 24.0 (TID 454) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:27 INFO Executor: Running task 144.0 in stage 24.0 (TID 454) -26/04/01 06:37:27 INFO TaskSetManager: Finished task 140.0 in stage 24.0 (TID 450) in 3417 ms on 10.0.0.133 (executor driver) (141/200) -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:27 INFO Executor: Finished task 141.0 in stage 24.0 (TID 451). 9201 bytes result sent to driver -26/04/01 06:37:27 INFO TaskSetManager: Starting task 145.0 in stage 24.0 (TID 455) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:27 INFO TaskSetManager: Finished task 141.0 in stage 24.0 (TID 451) in 3410 ms on 10.0.0.133 (executor driver) (142/200) -26/04/01 06:37:27 INFO Executor: Running task 145.0 in stage 24.0 (TID 455) -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:30 INFO Executor: Finished task 142.0 in stage 24.0 (TID 452). 8470 bytes result sent to driver -26/04/01 06:37:30 INFO TaskSetManager: Starting task 146.0 in stage 24.0 (TID 456) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:30 INFO Executor: Running task 146.0 in stage 24.0 (TID 456) -26/04/01 06:37:30 INFO TaskSetManager: Finished task 142.0 in stage 24.0 (TID 452) in 3452 ms on 10.0.0.133 (executor driver) (143/200) -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:30 INFO Executor: Finished task 143.0 in stage 24.0 (TID 453). 8470 bytes result sent to driver -26/04/01 06:37:30 INFO TaskSetManager: Starting task 147.0 in stage 24.0 (TID 457) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:30 INFO Executor: Running task 147.0 in stage 24.0 (TID 457) -26/04/01 06:37:30 INFO TaskSetManager: Finished task 143.0 in stage 24.0 (TID 453) in 3458 ms on 10.0.0.133 (executor driver) (144/200) -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:31 INFO Executor: Finished task 144.0 in stage 24.0 (TID 454). 8470 bytes result sent to driver -26/04/01 06:37:31 INFO TaskSetManager: Starting task 148.0 in stage 24.0 (TID 458) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:31 INFO Executor: Running task 148.0 in stage 24.0 (TID 458) -26/04/01 06:37:31 INFO TaskSetManager: Finished task 144.0 in stage 24.0 (TID 454) in 3431 ms on 10.0.0.133 (executor driver) (145/200) -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:31 INFO Executor: Finished task 145.0 in stage 24.0 (TID 455). 8470 bytes result sent to driver -26/04/01 06:37:31 INFO TaskSetManager: Starting task 149.0 in stage 24.0 (TID 459) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:31 INFO TaskSetManager: Finished task 145.0 in stage 24.0 (TID 455) in 3420 ms on 10.0.0.133 (executor driver) (146/200) -26/04/01 06:37:31 INFO Executor: Running task 149.0 in stage 24.0 (TID 459) -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:33 INFO Executor: Finished task 147.0 in stage 24.0 (TID 457). 8470 bytes result sent to driver -26/04/01 06:37:33 INFO TaskSetManager: Starting task 150.0 in stage 24.0 (TID 460) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:33 INFO Executor: Running task 150.0 in stage 24.0 (TID 460) -26/04/01 06:37:33 INFO TaskSetManager: Finished task 147.0 in stage 24.0 (TID 457) in 3414 ms on 10.0.0.133 (executor driver) (147/200) -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:33 INFO Executor: Finished task 146.0 in stage 24.0 (TID 456). 8470 bytes result sent to driver -26/04/01 06:37:33 INFO TaskSetManager: Starting task 151.0 in stage 24.0 (TID 461) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:33 INFO Executor: Running task 151.0 in stage 24.0 (TID 461) -26/04/01 06:37:33 INFO TaskSetManager: Finished task 146.0 in stage 24.0 (TID 456) in 3433 ms on 10.0.0.133 (executor driver) (148/200) -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:34 INFO Executor: Finished task 149.0 in stage 24.0 (TID 459). 8470 bytes result sent to driver -26/04/01 06:37:34 INFO TaskSetManager: Starting task 152.0 in stage 24.0 (TID 462) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:34 INFO Executor: Running task 152.0 in stage 24.0 (TID 462) -26/04/01 06:37:34 INFO TaskSetManager: Finished task 149.0 in stage 24.0 (TID 459) in 3430 ms on 10.0.0.133 (executor driver) (149/200) -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:34 INFO Executor: Finished task 148.0 in stage 24.0 (TID 458). 8470 bytes result sent to driver -26/04/01 06:37:34 INFO TaskSetManager: Starting task 153.0 in stage 24.0 (TID 463) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:34 INFO Executor: Running task 153.0 in stage 24.0 (TID 463) -26/04/01 06:37:34 INFO TaskSetManager: Finished task 148.0 in stage 24.0 (TID 458) in 3449 ms on 10.0.0.133 (executor driver) (150/200) -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:36 INFO Executor: Finished task 150.0 in stage 24.0 (TID 460). 9201 bytes result sent to driver -26/04/01 06:37:36 INFO TaskSetManager: Starting task 154.0 in stage 24.0 (TID 464) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:36 INFO Executor: Running task 154.0 in stage 24.0 (TID 464) -26/04/01 06:37:36 INFO TaskSetManager: Finished task 150.0 in stage 24.0 (TID 460) in 3438 ms on 10.0.0.133 (executor driver) (151/200) -26/04/01 06:37:36 INFO Executor: Finished task 151.0 in stage 24.0 (TID 461). 9201 bytes result sent to driver -26/04/01 06:37:36 INFO TaskSetManager: Starting task 155.0 in stage 24.0 (TID 465) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:36 INFO Executor: Running task 155.0 in stage 24.0 (TID 465) -26/04/01 06:37:36 INFO TaskSetManager: Finished task 151.0 in stage 24.0 (TID 461) in 3434 ms on 10.0.0.133 (executor driver) (152/200) -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:38 INFO Executor: Finished task 152.0 in stage 24.0 (TID 462). 9201 bytes result sent to driver -26/04/01 06:37:38 INFO TaskSetManager: Starting task 156.0 in stage 24.0 (TID 466) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:38 INFO Executor: Running task 156.0 in stage 24.0 (TID 466) -26/04/01 06:37:38 INFO TaskSetManager: Finished task 152.0 in stage 24.0 (TID 462) in 3436 ms on 10.0.0.133 (executor driver) (153/200) -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:38 INFO Executor: Finished task 153.0 in stage 24.0 (TID 463). 9201 bytes result sent to driver -26/04/01 06:37:38 INFO TaskSetManager: Starting task 157.0 in stage 24.0 (TID 467) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:38 INFO Executor: Running task 157.0 in stage 24.0 (TID 467) -26/04/01 06:37:38 INFO TaskSetManager: Finished task 153.0 in stage 24.0 (TID 463) in 3434 ms on 10.0.0.133 (executor driver) (154/200) -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:40 INFO Executor: Finished task 155.0 in stage 24.0 (TID 465). 8470 bytes result sent to driver -26/04/01 06:37:40 INFO TaskSetManager: Starting task 158.0 in stage 24.0 (TID 468) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:40 INFO Executor: Running task 158.0 in stage 24.0 (TID 468) -26/04/01 06:37:40 INFO TaskSetManager: Finished task 155.0 in stage 24.0 (TID 465) in 3449 ms on 10.0.0.133 (executor driver) (155/200) -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:40 INFO Executor: Finished task 154.0 in stage 24.0 (TID 464). 8470 bytes result sent to driver -26/04/01 06:37:40 INFO TaskSetManager: Starting task 159.0 in stage 24.0 (TID 469) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:40 INFO TaskSetManager: Finished task 154.0 in stage 24.0 (TID 464) in 3453 ms on 10.0.0.133 (executor driver) (156/200) -26/04/01 06:37:40 INFO Executor: Running task 159.0 in stage 24.0 (TID 469) -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:41 INFO Executor: Finished task 157.0 in stage 24.0 (TID 467). 8470 bytes result sent to driver -26/04/01 06:37:41 INFO TaskSetManager: Starting task 160.0 in stage 24.0 (TID 470) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:41 INFO Executor: Running task 160.0 in stage 24.0 (TID 470) -26/04/01 06:37:41 INFO TaskSetManager: Finished task 157.0 in stage 24.0 (TID 467) in 3431 ms on 10.0.0.133 (executor driver) (157/200) -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:41 INFO Executor: Finished task 156.0 in stage 24.0 (TID 466). 8470 bytes result sent to driver -26/04/01 06:37:41 INFO TaskSetManager: Starting task 161.0 in stage 24.0 (TID 471) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:41 INFO Executor: Running task 161.0 in stage 24.0 (TID 471) -26/04/01 06:37:41 INFO TaskSetManager: Finished task 156.0 in stage 24.0 (TID 466) in 3452 ms on 10.0.0.133 (executor driver) (158/200) -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:41 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:43 INFO Executor: Finished task 159.0 in stage 24.0 (TID 469). 8470 bytes result sent to driver -26/04/01 06:37:43 INFO TaskSetManager: Starting task 162.0 in stage 24.0 (TID 472) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:43 INFO Executor: Running task 162.0 in stage 24.0 (TID 472) -26/04/01 06:37:43 INFO TaskSetManager: Finished task 159.0 in stage 24.0 (TID 469) in 3442 ms on 10.0.0.133 (executor driver) (159/200) -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:43 INFO Executor: Finished task 158.0 in stage 24.0 (TID 468). 8470 bytes result sent to driver -26/04/01 06:37:43 INFO TaskSetManager: Starting task 163.0 in stage 24.0 (TID 473) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:43 INFO Executor: Running task 163.0 in stage 24.0 (TID 473) -26/04/01 06:37:43 INFO TaskSetManager: Finished task 158.0 in stage 24.0 (TID 468) in 3453 ms on 10.0.0.133 (executor driver) (160/200) -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:43 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:45 INFO Executor: Finished task 160.0 in stage 24.0 (TID 470). 9201 bytes result sent to driver -26/04/01 06:37:45 INFO TaskSetManager: Starting task 164.0 in stage 24.0 (TID 474) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:45 INFO Executor: Running task 164.0 in stage 24.0 (TID 474) -26/04/01 06:37:45 INFO TaskSetManager: Finished task 160.0 in stage 24.0 (TID 470) in 3438 ms on 10.0.0.133 (executor driver) (161/200) -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:45 INFO Executor: Finished task 161.0 in stage 24.0 (TID 471). 9201 bytes result sent to driver -26/04/01 06:37:45 INFO TaskSetManager: Starting task 165.0 in stage 24.0 (TID 475) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:45 INFO TaskSetManager: Finished task 161.0 in stage 24.0 (TID 471) in 3431 ms on 10.0.0.133 (executor driver) (162/200) -26/04/01 06:37:45 INFO Executor: Running task 165.0 in stage 24.0 (TID 475) -26/04/01 06:37:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Getting 208 (45.4 MiB) non-empty blocks including 208 (45.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:47 INFO Executor: Finished task 162.0 in stage 24.0 (TID 472). 9201 bytes result sent to driver -26/04/01 06:37:47 INFO TaskSetManager: Starting task 166.0 in stage 24.0 (TID 476) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:47 INFO Executor: Running task 166.0 in stage 24.0 (TID 476) -26/04/01 06:37:47 INFO TaskSetManager: Finished task 162.0 in stage 24.0 (TID 472) in 3448 ms on 10.0.0.133 (executor driver) (163/200) -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:47 INFO Executor: Finished task 163.0 in stage 24.0 (TID 473). 9201 bytes result sent to driver -26/04/01 06:37:47 INFO TaskSetManager: Starting task 167.0 in stage 24.0 (TID 477) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:47 INFO TaskSetManager: Finished task 163.0 in stage 24.0 (TID 473) in 3450 ms on 10.0.0.133 (executor driver) (164/200) -26/04/01 06:37:47 INFO Executor: Running task 167.0 in stage 24.0 (TID 477) -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:47 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:48 INFO Executor: Finished task 165.0 in stage 24.0 (TID 475). 8470 bytes result sent to driver -26/04/01 06:37:48 INFO TaskSetManager: Starting task 168.0 in stage 24.0 (TID 478) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:48 INFO Executor: Running task 168.0 in stage 24.0 (TID 478) -26/04/01 06:37:48 INFO TaskSetManager: Finished task 165.0 in stage 24.0 (TID 475) in 3413 ms on 10.0.0.133 (executor driver) (165/200) -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:48 INFO Executor: Finished task 164.0 in stage 24.0 (TID 474). 8470 bytes result sent to driver -26/04/01 06:37:48 INFO TaskSetManager: Starting task 169.0 in stage 24.0 (TID 479) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:48 INFO Executor: Running task 169.0 in stage 24.0 (TID 479) -26/04/01 06:37:48 INFO TaskSetManager: Finished task 164.0 in stage 24.0 (TID 474) in 3441 ms on 10.0.0.133 (executor driver) (166/200) -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:48 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:50 INFO Executor: Finished task 167.0 in stage 24.0 (TID 477). 8470 bytes result sent to driver -26/04/01 06:37:50 INFO Executor: Finished task 166.0 in stage 24.0 (TID 476). 8470 bytes result sent to driver -26/04/01 06:37:50 INFO TaskSetManager: Starting task 170.0 in stage 24.0 (TID 480) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:50 INFO Executor: Running task 170.0 in stage 24.0 (TID 480) -26/04/01 06:37:50 INFO TaskSetManager: Starting task 171.0 in stage 24.0 (TID 481) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:50 INFO TaskSetManager: Finished task 167.0 in stage 24.0 (TID 477) in 3418 ms on 10.0.0.133 (executor driver) (167/200) -26/04/01 06:37:50 INFO Executor: Running task 171.0 in stage 24.0 (TID 481) -26/04/01 06:37:50 INFO TaskSetManager: Finished task 166.0 in stage 24.0 (TID 476) in 3427 ms on 10.0.0.133 (executor driver) (168/200) -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:51 INFO Executor: Finished task 168.0 in stage 24.0 (TID 478). 8470 bytes result sent to driver -26/04/01 06:37:51 INFO TaskSetManager: Starting task 172.0 in stage 24.0 (TID 482) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:51 INFO Executor: Finished task 169.0 in stage 24.0 (TID 479). 8470 bytes result sent to driver -26/04/01 06:37:51 INFO Executor: Running task 172.0 in stage 24.0 (TID 482) -26/04/01 06:37:51 INFO TaskSetManager: Starting task 173.0 in stage 24.0 (TID 483) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:51 INFO Executor: Running task 173.0 in stage 24.0 (TID 483) -26/04/01 06:37:51 INFO TaskSetManager: Finished task 168.0 in stage 24.0 (TID 478) in 3437 ms on 10.0.0.133 (executor driver) (169/200) -26/04/01 06:37:51 INFO TaskSetManager: Finished task 169.0 in stage 24.0 (TID 479) in 3419 ms on 10.0.0.133 (executor driver) (170/200) -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:54 INFO Executor: Finished task 170.0 in stage 24.0 (TID 480). 8470 bytes result sent to driver -26/04/01 06:37:54 INFO Executor: Finished task 171.0 in stage 24.0 (TID 481). 8470 bytes result sent to driver -26/04/01 06:37:54 INFO TaskSetManager: Starting task 174.0 in stage 24.0 (TID 484) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:54 INFO Executor: Running task 174.0 in stage 24.0 (TID 484) -26/04/01 06:37:54 INFO TaskSetManager: Starting task 175.0 in stage 24.0 (TID 485) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:54 INFO TaskSetManager: Finished task 170.0 in stage 24.0 (TID 480) in 3443 ms on 10.0.0.133 (executor driver) (171/200) -26/04/01 06:37:54 INFO Executor: Running task 175.0 in stage 24.0 (TID 485) -26/04/01 06:37:54 INFO TaskSetManager: Finished task 171.0 in stage 24.0 (TID 481) in 3443 ms on 10.0.0.133 (executor driver) (172/200) -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:55 INFO Executor: Finished task 172.0 in stage 24.0 (TID 482). 9201 bytes result sent to driver -26/04/01 06:37:55 INFO TaskSetManager: Starting task 176.0 in stage 24.0 (TID 486) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:55 INFO Executor: Running task 176.0 in stage 24.0 (TID 486) -26/04/01 06:37:55 INFO TaskSetManager: Finished task 172.0 in stage 24.0 (TID 482) in 3443 ms on 10.0.0.133 (executor driver) (173/200) -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:55 INFO Executor: Finished task 173.0 in stage 24.0 (TID 483). 9201 bytes result sent to driver -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:55 INFO TaskSetManager: Starting task 177.0 in stage 24.0 (TID 487) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:55 INFO Executor: Running task 177.0 in stage 24.0 (TID 487) -26/04/01 06:37:55 INFO TaskSetManager: Finished task 173.0 in stage 24.0 (TID 483) in 3450 ms on 10.0.0.133 (executor driver) (174/200) -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:57 INFO Executor: Finished task 175.0 in stage 24.0 (TID 485). 9201 bytes result sent to driver -26/04/01 06:37:57 INFO TaskSetManager: Starting task 178.0 in stage 24.0 (TID 488) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:57 INFO Executor: Running task 178.0 in stage 24.0 (TID 488) -26/04/01 06:37:57 INFO TaskSetManager: Finished task 175.0 in stage 24.0 (TID 485) in 3440 ms on 10.0.0.133 (executor driver) (175/200) -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:57 INFO Executor: Finished task 174.0 in stage 24.0 (TID 484). 9201 bytes result sent to driver -26/04/01 06:37:57 INFO TaskSetManager: Starting task 179.0 in stage 24.0 (TID 489) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:57 INFO TaskSetManager: Finished task 174.0 in stage 24.0 (TID 484) in 3462 ms on 10.0.0.133 (executor driver) (176/200) -26/04/01 06:37:57 INFO Executor: Running task 179.0 in stage 24.0 (TID 489) -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:58 INFO Executor: Finished task 176.0 in stage 24.0 (TID 486). 8470 bytes result sent to driver -26/04/01 06:37:58 INFO TaskSetManager: Starting task 180.0 in stage 24.0 (TID 490) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:58 INFO Executor: Running task 180.0 in stage 24.0 (TID 490) -26/04/01 06:37:58 INFO TaskSetManager: Finished task 176.0 in stage 24.0 (TID 486) in 3463 ms on 10.0.0.133 (executor driver) (177/200) -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:58 INFO Executor: Finished task 177.0 in stage 24.0 (TID 487). 8470 bytes result sent to driver -26/04/01 06:37:58 INFO TaskSetManager: Starting task 181.0 in stage 24.0 (TID 491) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9428 bytes) -26/04/01 06:37:58 INFO Executor: Running task 181.0 in stage 24.0 (TID 491) -26/04/01 06:37:58 INFO TaskSetManager: Finished task 177.0 in stage 24.0 (TID 487) in 3459 ms on 10.0.0.133 (executor driver) (178/200) -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:37:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:37:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:37:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:00 INFO Executor: Finished task 178.0 in stage 24.0 (TID 488). 8470 bytes result sent to driver -26/04/01 06:38:00 INFO TaskSetManager: Starting task 182.0 in stage 24.0 (TID 492) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:00 INFO Executor: Running task 182.0 in stage 24.0 (TID 492) -26/04/01 06:38:00 INFO TaskSetManager: Finished task 178.0 in stage 24.0 (TID 488) in 3435 ms on 10.0.0.133 (executor driver) (179/200) -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:00 INFO Executor: Finished task 179.0 in stage 24.0 (TID 489). 8470 bytes result sent to driver -26/04/01 06:38:00 INFO TaskSetManager: Starting task 183.0 in stage 24.0 (TID 493) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:00 INFO Executor: Running task 183.0 in stage 24.0 (TID 493) -26/04/01 06:38:00 INFO TaskSetManager: Finished task 179.0 in stage 24.0 (TID 489) in 3418 ms on 10.0.0.133 (executor driver) (180/200) -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Getting 208 (45.5 MiB) non-empty blocks including 208 (45.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:02 INFO Executor: Finished task 180.0 in stage 24.0 (TID 490). 8470 bytes result sent to driver -26/04/01 06:38:02 INFO TaskSetManager: Starting task 184.0 in stage 24.0 (TID 494) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:02 INFO Executor: Running task 184.0 in stage 24.0 (TID 494) -26/04/01 06:38:02 INFO TaskSetManager: Finished task 180.0 in stage 24.0 (TID 490) in 3467 ms on 10.0.0.133 (executor driver) (181/200) -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1034.7 KiB) non-empty blocks including 6 (1034.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:02 INFO Executor: Finished task 181.0 in stage 24.0 (TID 491). 8470 bytes result sent to driver -26/04/01 06:38:02 INFO TaskSetManager: Starting task 185.0 in stage 24.0 (TID 495) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:02 INFO Executor: Running task 185.0 in stage 24.0 (TID 495) -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:02 INFO TaskSetManager: Finished task 181.0 in stage 24.0 (TID 491) in 3470 ms on 10.0.0.133 (executor driver) (182/200) -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:04 INFO Executor: Finished task 183.0 in stage 24.0 (TID 493). 8470 bytes result sent to driver -26/04/01 06:38:04 INFO TaskSetManager: Starting task 186.0 in stage 24.0 (TID 496) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:04 INFO Executor: Running task 186.0 in stage 24.0 (TID 496) -26/04/01 06:38:04 INFO TaskSetManager: Finished task 183.0 in stage 24.0 (TID 493) in 3450 ms on 10.0.0.133 (executor driver) (183/200) -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:04 INFO Executor: Finished task 182.0 in stage 24.0 (TID 492). 8470 bytes result sent to driver -26/04/01 06:38:04 INFO TaskSetManager: Starting task 187.0 in stage 24.0 (TID 497) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:04 INFO Executor: Running task 187.0 in stage 24.0 (TID 497) -26/04/01 06:38:04 INFO TaskSetManager: Finished task 182.0 in stage 24.0 (TID 492) in 3459 ms on 10.0.0.133 (executor driver) (184/200) -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:05 INFO Executor: Finished task 184.0 in stage 24.0 (TID 494). 9201 bytes result sent to driver -26/04/01 06:38:05 INFO TaskSetManager: Starting task 188.0 in stage 24.0 (TID 498) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:05 INFO Executor: Running task 188.0 in stage 24.0 (TID 498) -26/04/01 06:38:05 INFO TaskSetManager: Finished task 184.0 in stage 24.0 (TID 494) in 3459 ms on 10.0.0.133 (executor driver) (185/200) -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1044.6 KiB) non-empty blocks including 6 (1044.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:05 INFO Executor: Finished task 185.0 in stage 24.0 (TID 495). 9201 bytes result sent to driver -26/04/01 06:38:05 INFO TaskSetManager: Starting task 189.0 in stage 24.0 (TID 499) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:05 INFO Executor: Running task 189.0 in stage 24.0 (TID 499) -26/04/01 06:38:05 INFO TaskSetManager: Finished task 185.0 in stage 24.0 (TID 495) in 3463 ms on 10.0.0.133 (executor driver) (186/200) -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 6 (1062.3 KiB) non-empty blocks including 6 (1062.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:07 INFO Executor: Finished task 187.0 in stage 24.0 (TID 497). 9201 bytes result sent to driver -26/04/01 06:38:07 INFO TaskSetManager: Starting task 190.0 in stage 24.0 (TID 500) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:07 INFO Executor: Running task 190.0 in stage 24.0 (TID 500) -26/04/01 06:38:07 INFO TaskSetManager: Finished task 187.0 in stage 24.0 (TID 497) in 3438 ms on 10.0.0.133 (executor driver) (187/200) -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:07 INFO Executor: Finished task 186.0 in stage 24.0 (TID 496). 9201 bytes result sent to driver -26/04/01 06:38:07 INFO TaskSetManager: Starting task 191.0 in stage 24.0 (TID 501) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:07 INFO Executor: Running task 191.0 in stage 24.0 (TID 501) -26/04/01 06:38:07 INFO TaskSetManager: Finished task 186.0 in stage 24.0 (TID 496) in 3456 ms on 10.0.0.133 (executor driver) (188/200) -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:09 INFO Executor: Finished task 188.0 in stage 24.0 (TID 498). 8470 bytes result sent to driver -26/04/01 06:38:09 INFO TaskSetManager: Starting task 192.0 in stage 24.0 (TID 502) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:09 INFO Executor: Running task 192.0 in stage 24.0 (TID 502) -26/04/01 06:38:09 INFO TaskSetManager: Finished task 188.0 in stage 24.0 (TID 498) in 3442 ms on 10.0.0.133 (executor driver) (189/200) -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:09 INFO Executor: Finished task 189.0 in stage 24.0 (TID 499). 8470 bytes result sent to driver -26/04/01 06:38:09 INFO TaskSetManager: Starting task 193.0 in stage 24.0 (TID 503) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:09 INFO TaskSetManager: Finished task 189.0 in stage 24.0 (TID 499) in 3439 ms on 10.0.0.133 (executor driver) (190/200) -26/04/01 06:38:09 INFO Executor: Running task 193.0 in stage 24.0 (TID 503) -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:11 INFO Executor: Finished task 191.0 in stage 24.0 (TID 501). 8470 bytes result sent to driver -26/04/01 06:38:11 INFO TaskSetManager: Starting task 194.0 in stage 24.0 (TID 504) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:11 INFO Executor: Running task 194.0 in stage 24.0 (TID 504) -26/04/01 06:38:11 INFO TaskSetManager: Finished task 191.0 in stage 24.0 (TID 501) in 3422 ms on 10.0.0.133 (executor driver) (191/200) -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1079.9 KiB) non-empty blocks including 6 (1079.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.6 MiB) non-empty blocks including 208 (45.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:11 INFO Executor: Finished task 190.0 in stage 24.0 (TID 500). 8470 bytes result sent to driver -26/04/01 06:38:11 INFO TaskSetManager: Starting task 195.0 in stage 24.0 (TID 505) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:11 INFO TaskSetManager: Finished task 190.0 in stage 24.0 (TID 500) in 3461 ms on 10.0.0.133 (executor driver) (192/200) -26/04/01 06:38:11 INFO Executor: Running task 195.0 in stage 24.0 (TID 505) -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Getting 208 (45.9 MiB) non-empty blocks including 208 (45.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:12 INFO Executor: Finished task 193.0 in stage 24.0 (TID 503). 8470 bytes result sent to driver -26/04/01 06:38:12 INFO TaskSetManager: Starting task 196.0 in stage 24.0 (TID 506) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:12 INFO Executor: Running task 196.0 in stage 24.0 (TID 506) -26/04/01 06:38:12 INFO TaskSetManager: Finished task 193.0 in stage 24.0 (TID 503) in 3438 ms on 10.0.0.133 (executor driver) (193/200) -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:12 INFO Executor: Finished task 192.0 in stage 24.0 (TID 502). 8470 bytes result sent to driver -26/04/01 06:38:12 INFO TaskSetManager: Starting task 197.0 in stage 24.0 (TID 507) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:12 INFO Executor: Running task 197.0 in stage 24.0 (TID 507) -26/04/01 06:38:12 INFO TaskSetManager: Finished task 192.0 in stage 24.0 (TID 502) in 3461 ms on 10.0.0.133 (executor driver) (194/200) -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 6 (1052.3 KiB) non-empty blocks including 6 (1052.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:14 INFO Executor: Finished task 195.0 in stage 24.0 (TID 505). 8470 bytes result sent to driver -26/04/01 06:38:14 INFO Executor: Finished task 194.0 in stage 24.0 (TID 504). 8470 bytes result sent to driver -26/04/01 06:38:14 INFO TaskSetManager: Starting task 198.0 in stage 24.0 (TID 508) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:14 INFO Executor: Running task 198.0 in stage 24.0 (TID 508) -26/04/01 06:38:14 INFO TaskSetManager: Starting task 199.0 in stage 24.0 (TID 509) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9428 bytes) -26/04/01 06:38:14 INFO TaskSetManager: Finished task 195.0 in stage 24.0 (TID 505) in 3407 ms on 10.0.0.133 (executor driver) (195/200) -26/04/01 06:38:14 INFO Executor: Running task 199.0 in stage 24.0 (TID 509) -26/04/01 06:38:14 INFO TaskSetManager: Finished task 194.0 in stage 24.0 (TID 504) in 3435 ms on 10.0.0.133 (executor driver) (196/200) -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 6 (1070.0 KiB) non-empty blocks including 6 (1070.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.7 MiB) non-empty blocks including 208 (45.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Getting 208 (45.8 MiB) non-empty blocks including 208 (45.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:16 INFO Executor: Finished task 197.0 in stage 24.0 (TID 507). 9201 bytes result sent to driver -26/04/01 06:38:16 INFO TaskSetManager: Finished task 197.0 in stage 24.0 (TID 507) in 3426 ms on 10.0.0.133 (executor driver) (197/200) -26/04/01 06:38:16 INFO Executor: Finished task 196.0 in stage 24.0 (TID 506). 9201 bytes result sent to driver -26/04/01 06:38:16 INFO TaskSetManager: Finished task 196.0 in stage 24.0 (TID 506) in 3445 ms on 10.0.0.133 (executor driver) (198/200) -26/04/01 06:38:18 INFO Executor: Finished task 198.0 in stage 24.0 (TID 508). 9201 bytes result sent to driver -26/04/01 06:38:18 INFO TaskSetManager: Finished task 198.0 in stage 24.0 (TID 508) in 3303 ms on 10.0.0.133 (executor driver) (199/200) -26/04/01 06:38:18 INFO Executor: Finished task 199.0 in stage 24.0 (TID 509). 9201 bytes result sent to driver -26/04/01 06:38:18 INFO TaskSetManager: Finished task 199.0 in stage 24.0 (TID 509) in 3317 ms on 10.0.0.133 (executor driver) (200/200) -26/04/01 06:38:18 INFO TaskSchedulerImpl: Removed TaskSet 24.0, whose tasks have all completed, from pool -26/04/01 06:38:18 INFO DAGScheduler: ShuffleMapStage 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 172.034 s -26/04/01 06:38:18 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:38:18 INFO DAGScheduler: running: Set() -26/04/01 06:38:18 INFO DAGScheduler: waiting: Set() -26/04/01 06:38:18 INFO DAGScheduler: failed: Set() -26/04/01 06:38:18 INFO ShufflePartitionsUtil: For shuffle(6, 4), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 06:38:18 INFO DAGScheduler: Registering RDD 54 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 7 -26/04/01 06:38:18 INFO DAGScheduler: Got map stage job 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 25 output partitions -26/04/01 06:38:18 INFO DAGScheduler: Final stage: ShuffleMapStage 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:38:18 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 30, ShuffleMapStage 29) -26/04/01 06:38:18 INFO DAGScheduler: Missing parents: List() -26/04/01 06:38:18 INFO DAGScheduler: Submitting ShuffleMapStage 31 (MapPartitionsRDD[54] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:38:18 INFO MemoryStore: Block broadcast_27 stored as values in memory (estimated size 39.3 KiB, free 12.6 GiB) -26/04/01 06:38:18 INFO MemoryStore: Block broadcast_27_piece0 stored as bytes in memory (estimated size 12.3 KiB, free 12.6 GiB) -26/04/01 06:38:18 INFO BlockManagerInfo: Added broadcast_27_piece0 in memory on 10.0.0.133:58442 (size: 12.3 KiB, free: 12.6 GiB) -26/04/01 06:38:18 INFO SparkContext: Created broadcast 27 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:38:18 INFO DAGScheduler: Submitting 25 missing tasks from ShuffleMapStage 31 (MapPartitionsRDD[54] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 06:38:18 INFO TaskSchedulerImpl: Adding task set 31.0 with 25 tasks resource profile 0 -26/04/01 06:38:18 INFO TaskSetManager: Starting task 0.0 in stage 31.0 (TID 510) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:18 INFO TaskSetManager: Starting task 1.0 in stage 31.0 (TID 511) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:18 INFO TaskSetManager: Starting task 2.0 in stage 31.0 (TID 512) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:18 INFO TaskSetManager: Starting task 3.0 in stage 31.0 (TID 513) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:18 INFO Executor: Running task 1.0 in stage 31.0 (TID 511) -26/04/01 06:38:18 INFO Executor: Running task 3.0 in stage 31.0 (TID 513) -26/04/01 06:38:18 INFO Executor: Running task 2.0 in stage 31.0 (TID 512) -26/04/01 06:38:18 INFO Executor: Running task 0.0 in stage 31.0 (TID 510) -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (320.9 KiB) non-empty blocks including 4 (320.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Getting 4 (320.9 KiB) non-empty blocks including 4 (320.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:26 INFO Executor: Finished task 3.0 in stage 31.0 (TID 513). 11071 bytes result sent to driver -26/04/01 06:38:26 INFO TaskSetManager: Starting task 4.0 in stage 31.0 (TID 514) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:26 INFO Executor: Running task 4.0 in stage 31.0 (TID 514) -26/04/01 06:38:26 INFO TaskSetManager: Finished task 3.0 in stage 31.0 (TID 513) in 8854 ms on 10.0.0.133 (executor driver) (1/25) -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO Executor: Finished task 0.0 in stage 31.0 (TID 510). 11071 bytes result sent to driver -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO TaskSetManager: Starting task 5.0 in stage 31.0 (TID 515) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:26 INFO Executor: Running task 5.0 in stage 31.0 (TID 515) -26/04/01 06:38:26 INFO TaskSetManager: Finished task 0.0 in stage 31.0 (TID 510) in 8862 ms on 10.0.0.133 (executor driver) (2/25) -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:26 INFO Executor: Finished task 2.0 in stage 31.0 (TID 512). 11071 bytes result sent to driver -26/04/01 06:38:26 INFO TaskSetManager: Starting task 6.0 in stage 31.0 (TID 516) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:26 INFO Executor: Running task 6.0 in stage 31.0 (TID 516) -26/04/01 06:38:26 INFO TaskSetManager: Finished task 2.0 in stage 31.0 (TID 512) in 8872 ms on 10.0.0.133 (executor driver) (3/25) -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO Executor: Finished task 1.0 in stage 31.0 (TID 511). 11071 bytes result sent to driver -26/04/01 06:38:26 INFO TaskSetManager: Starting task 7.0 in stage 31.0 (TID 517) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:26 INFO Executor: Running task 7.0 in stage 31.0 (TID 517) -26/04/01 06:38:26 INFO TaskSetManager: Finished task 1.0 in stage 31.0 (TID 511) in 8874 ms on 10.0.0.133 (executor driver) (4/25) -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:35 INFO Executor: Finished task 5.0 in stage 31.0 (TID 515). 11071 bytes result sent to driver -26/04/01 06:38:35 INFO TaskSetManager: Starting task 8.0 in stage 31.0 (TID 518) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:35 INFO Executor: Running task 8.0 in stage 31.0 (TID 518) -26/04/01 06:38:35 INFO TaskSetManager: Finished task 5.0 in stage 31.0 (TID 515) in 8743 ms on 10.0.0.133 (executor driver) (5/25) -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:35 INFO Executor: Finished task 4.0 in stage 31.0 (TID 514). 11071 bytes result sent to driver -26/04/01 06:38:35 INFO TaskSetManager: Starting task 9.0 in stage 31.0 (TID 519) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:35 INFO TaskSetManager: Finished task 4.0 in stage 31.0 (TID 514) in 8776 ms on 10.0.0.133 (executor driver) (6/25) -26/04/01 06:38:35 INFO Executor: Running task 9.0 in stage 31.0 (TID 519) -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (317.2 KiB) non-empty blocks including 4 (317.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (317.2 KiB) non-empty blocks including 4 (317.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:35 INFO Executor: Finished task 6.0 in stage 31.0 (TID 516). 11071 bytes result sent to driver -26/04/01 06:38:35 INFO TaskSetManager: Starting task 10.0 in stage 31.0 (TID 520) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:35 INFO TaskSetManager: Finished task 6.0 in stage 31.0 (TID 516) in 8805 ms on 10.0.0.133 (executor driver) (7/25) -26/04/01 06:38:35 INFO Executor: Running task 10.0 in stage 31.0 (TID 520) -26/04/01 06:38:35 INFO Executor: Finished task 7.0 in stage 31.0 (TID 517). 11071 bytes result sent to driver -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO TaskSetManager: Starting task 11.0 in stage 31.0 (TID 521) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:35 INFO Executor: Running task 11.0 in stage 31.0 (TID 521) -26/04/01 06:38:35 INFO TaskSetManager: Finished task 7.0 in stage 31.0 (TID 517) in 8805 ms on 10.0.0.133 (executor driver) (8/25) -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:44 INFO Executor: Finished task 8.0 in stage 31.0 (TID 518). 10340 bytes result sent to driver -26/04/01 06:38:44 INFO TaskSetManager: Starting task 12.0 in stage 31.0 (TID 522) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:44 INFO Executor: Running task 12.0 in stage 31.0 (TID 522) -26/04/01 06:38:44 INFO TaskSetManager: Finished task 8.0 in stage 31.0 (TID 518) in 8729 ms on 10.0.0.133 (executor driver) (9/25) -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:44 INFO Executor: Finished task 9.0 in stage 31.0 (TID 519). 10340 bytes result sent to driver -26/04/01 06:38:44 INFO TaskSetManager: Starting task 13.0 in stage 31.0 (TID 523) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:44 INFO TaskSetManager: Finished task 9.0 in stage 31.0 (TID 519) in 8738 ms on 10.0.0.133 (executor driver) (10/25) -26/04/01 06:38:44 INFO Executor: Running task 13.0 in stage 31.0 (TID 523) -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (322.7 KiB) non-empty blocks including 4 (322.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (322.7 KiB) non-empty blocks including 4 (322.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:44 INFO Executor: Finished task 10.0 in stage 31.0 (TID 520). 10340 bytes result sent to driver -26/04/01 06:38:44 INFO TaskSetManager: Starting task 14.0 in stage 31.0 (TID 524) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:44 INFO Executor: Running task 14.0 in stage 31.0 (TID 524) -26/04/01 06:38:44 INFO TaskSetManager: Finished task 10.0 in stage 31.0 (TID 520) in 8775 ms on 10.0.0.133 (executor driver) (11/25) -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO Executor: Finished task 11.0 in stage 31.0 (TID 521). 10340 bytes result sent to driver -26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:44 INFO TaskSetManager: Starting task 15.0 in stage 31.0 (TID 525) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:44 INFO TaskSetManager: Finished task 11.0 in stage 31.0 (TID 521) in 8783 ms on 10.0.0.133 (executor driver) (12/25) -26/04/01 06:38:44 INFO Executor: Running task 15.0 in stage 31.0 (TID 525) -26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:44 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:53 INFO Executor: Finished task 12.0 in stage 31.0 (TID 522). 11071 bytes result sent to driver -26/04/01 06:38:53 INFO TaskSetManager: Starting task 16.0 in stage 31.0 (TID 526) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:53 INFO TaskSetManager: Finished task 12.0 in stage 31.0 (TID 522) in 8714 ms on 10.0.0.133 (executor driver) (13/25) -26/04/01 06:38:53 INFO Executor: Running task 16.0 in stage 31.0 (TID 526) -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:53 INFO Executor: Finished task 13.0 in stage 31.0 (TID 523). 11071 bytes result sent to driver -26/04/01 06:38:53 INFO TaskSetManager: Starting task 17.0 in stage 31.0 (TID 527) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:53 INFO Executor: Running task 17.0 in stage 31.0 (TID 527) -26/04/01 06:38:53 INFO TaskSetManager: Finished task 13.0 in stage 31.0 (TID 523) in 8701 ms on 10.0.0.133 (executor driver) (14/25) -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:53 INFO Executor: Finished task 14.0 in stage 31.0 (TID 524). 11071 bytes result sent to driver -26/04/01 06:38:53 INFO TaskSetManager: Starting task 18.0 in stage 31.0 (TID 528) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:53 INFO Executor: Running task 18.0 in stage 31.0 (TID 528) -26/04/01 06:38:53 INFO TaskSetManager: Finished task 14.0 in stage 31.0 (TID 524) in 8668 ms on 10.0.0.133 (executor driver) (15/25) -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (318.0 KiB) non-empty blocks including 4 (318.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:53 INFO Executor: Finished task 15.0 in stage 31.0 (TID 525). 11071 bytes result sent to driver -26/04/01 06:38:53 INFO TaskSetManager: Starting task 19.0 in stage 31.0 (TID 529) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9719 bytes) -26/04/01 06:38:53 INFO Executor: Running task 19.0 in stage 31.0 (TID 529) -26/04/01 06:38:53 INFO TaskSetManager: Finished task 15.0 in stage 31.0 (TID 525) in 8672 ms on 10.0.0.133 (executor driver) (16/25) -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Getting 4 (318.9 KiB) non-empty blocks including 4 (318.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:38:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:39:01 INFO Executor: Finished task 16.0 in stage 31.0 (TID 526). 11071 bytes result sent to driver -26/04/01 06:39:01 INFO TaskSetManager: Starting task 20.0 in stage 31.0 (TID 530) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9719 bytes) -26/04/01 06:39:01 INFO Executor: Running task 20.0 in stage 31.0 (TID 530) -26/04/01 06:39:01 INFO TaskSetManager: Finished task 16.0 in stage 31.0 (TID 526) in 8795 ms on 10.0.0.133 (executor driver) (17/25) -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:39:01 INFO Executor: Finished task 18.0 in stage 31.0 (TID 528). 11071 bytes result sent to driver -26/04/01 06:39:01 INFO TaskSetManager: Starting task 21.0 in stage 31.0 (TID 531) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9719 bytes) -26/04/01 06:39:01 INFO Executor: Running task 21.0 in stage 31.0 (TID 531) -26/04/01 06:39:01 INFO TaskSetManager: Finished task 18.0 in stage 31.0 (TID 528) in 8757 ms on 10.0.0.133 (executor driver) (18/25) -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (59.9 MiB) non-empty blocks including 200 (59.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (319.9 KiB) non-empty blocks including 4 (319.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:39:01 INFO Executor: Finished task 19.0 in stage 31.0 (TID 529). 11071 bytes result sent to driver -26/04/01 06:39:01 INFO TaskSetManager: Starting task 22.0 in stage 31.0 (TID 532) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9719 bytes) -26/04/01 06:39:01 INFO Executor: Running task 22.0 in stage 31.0 (TID 532) -26/04/01 06:39:01 INFO TaskSetManager: Finished task 19.0 in stage 31.0 (TID 529) in 8754 ms on 10.0.0.133 (executor driver) (19/25) -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (317.1 KiB) non-empty blocks including 4 (317.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:39:01 INFO Executor: Finished task 17.0 in stage 31.0 (TID 527). 11071 bytes result sent to driver -26/04/01 06:39:01 INFO TaskSetManager: Starting task 23.0 in stage 31.0 (TID 533) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9719 bytes) -26/04/01 06:39:01 INFO Executor: Running task 23.0 in stage 31.0 (TID 533) -26/04/01 06:39:01 INFO TaskSetManager: Finished task 17.0 in stage 31.0 (TID 527) in 8831 ms on 10.0.0.133 (executor driver) (20/25) -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (322.8 KiB) non-empty blocks including 4 (322.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Getting 4 (322.8 KiB) non-empty blocks including 4 (322.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:39:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:39:10 INFO Executor: Finished task 21.0 in stage 31.0 (TID 531). 11071 bytes result sent to driver -26/04/01 06:39:10 INFO TaskSetManager: Starting task 24.0 in stage 31.0 (TID 534) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9719 bytes) -26/04/01 06:39:10 INFO Executor: Running task 24.0 in stage 31.0 (TID 534) -26/04/01 06:39:10 INFO TaskSetManager: Finished task 21.0 in stage 31.0 (TID 531) in 8697 ms on 10.0.0.133 (executor driver) (21/25) -26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Getting 200 (60.0 MiB) non-empty blocks including 200 (60.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Getting 4 (320.8 KiB) non-empty blocks including 4 (320.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:39:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:39:10 INFO Executor: Finished task 20.0 in stage 31.0 (TID 530). 11071 bytes result sent to driver -26/04/01 06:39:10 INFO TaskSetManager: Finished task 20.0 in stage 31.0 (TID 530) in 8754 ms on 10.0.0.133 (executor driver) (22/25) -26/04/01 06:39:10 INFO Executor: Finished task 22.0 in stage 31.0 (TID 532). 11071 bytes result sent to driver -26/04/01 06:39:10 INFO TaskSetManager: Finished task 22.0 in stage 31.0 (TID 532) in 8748 ms on 10.0.0.133 (executor driver) (23/25) -26/04/01 06:39:10 INFO Executor: Finished task 23.0 in stage 31.0 (TID 533). 11071 bytes result sent to driver -26/04/01 06:39:10 INFO TaskSetManager: Finished task 23.0 in stage 31.0 (TID 533) in 8773 ms on 10.0.0.133 (executor driver) (24/25) -26/04/01 06:39:16 INFO Executor: Finished task 24.0 in stage 31.0 (TID 534). 11071 bytes result sent to driver -26/04/01 06:39:16 INFO TaskSetManager: Finished task 24.0 in stage 31.0 (TID 534) in 6282 ms on 10.0.0.133 (executor driver) (25/25) -26/04/01 06:39:16 INFO TaskSchedulerImpl: Removed TaskSet 31.0, whose tasks have all completed, from pool -26/04/01 06:39:16 INFO DAGScheduler: ShuffleMapStage 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 58.857 s -26/04/01 06:39:16 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:39:16 INFO DAGScheduler: running: Set() -26/04/01 06:39:16 INFO DAGScheduler: waiting: Set() -26/04/01 06:39:16 INFO DAGScheduler: failed: Set() -26/04/01 06:39:16 INFO ShufflePartitionsUtil: For shuffle(7), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 06:39:16 INFO CodeGenerator: Code generated in 62.440042 ms -26/04/01 06:39:17 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:39:17 INFO DAGScheduler: Got job 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:39:17 INFO DAGScheduler: Final stage: ResultStage 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:39:17 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 38) -26/04/01 06:39:17 INFO DAGScheduler: Missing parents: List() -26/04/01 06:39:17 INFO DAGScheduler: Submitting ResultStage 39 (MapPartitionsRDD[59] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:39:17 INFO MemoryStore: Block broadcast_28 stored as values in memory (estimated size 22.7 KiB, free 12.6 GiB) -26/04/01 06:39:17 INFO MemoryStore: Block broadcast_28_piece0 stored as bytes in memory (estimated size 8.3 KiB, free 12.6 GiB) -26/04/01 06:39:17 INFO BlockManagerInfo: Added broadcast_28_piece0 in memory on 10.0.0.133:58442 (size: 8.3 KiB, free: 12.6 GiB) -26/04/01 06:39:17 INFO SparkContext: Created broadcast 28 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:39:17 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 39 (MapPartitionsRDD[59] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:39:17 INFO TaskSchedulerImpl: Adding task set 39.0 with 1 tasks resource profile 0 -26/04/01 06:39:17 INFO TaskSetManager: Starting task 0.0 in stage 39.0 (TID 535) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) -26/04/01 06:39:17 INFO Executor: Running task 0.0 in stage 39.0 (TID 535) -26/04/01 06:39:17 INFO ShuffleBlockFetcherIterator: Getting 25 (54.3 KiB) non-empty blocks including 25 (54.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:39:17 INFO CodeGenerator: Code generated in 8.900458 ms -26/04/01 06:39:17 INFO Executor: Finished task 0.0 in stage 39.0 (TID 535). 5177 bytes result sent to driver -26/04/01 06:39:17 INFO TaskSetManager: Finished task 0.0 in stage 39.0 (TID 535) in 55 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:39:17 INFO TaskSchedulerImpl: Removed TaskSet 39.0, whose tasks have all completed, from pool -26/04/01 06:39:17 INFO DAGScheduler: ResultStage 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.057 s -26/04/01 06:39:17 INFO DAGScheduler: Job 19 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:39:17 INFO TaskSchedulerImpl: Killing all running tasks in stage 39: Stage finished -26/04/01 06:39:17 INFO DAGScheduler: Job 19 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.058802 s -26/04/01 06:39:17 INFO DAGScheduler: Registering RDD 60 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 8 -26/04/01 06:39:17 INFO DAGScheduler: Got map stage job 20 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:39:17 INFO DAGScheduler: Final stage: ShuffleMapStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:39:17 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 46) -26/04/01 06:39:17 INFO DAGScheduler: Missing parents: List() -26/04/01 06:39:17 INFO DAGScheduler: Submitting ShuffleMapStage 47 (MapPartitionsRDD[60] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:39:17 INFO MemoryStore: Block broadcast_29 stored as values in memory (estimated size 25.0 KiB, free 12.6 GiB) -26/04/01 06:39:17 INFO MemoryStore: Block broadcast_29_piece0 stored as bytes in memory (estimated size 9.3 KiB, free 12.6 GiB) -26/04/01 06:39:17 INFO BlockManagerInfo: Added broadcast_29_piece0 in memory on 10.0.0.133:58442 (size: 9.3 KiB, free: 12.6 GiB) -26/04/01 06:39:17 INFO SparkContext: Created broadcast 29 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:39:17 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 47 (MapPartitionsRDD[60] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:39:17 INFO TaskSchedulerImpl: Adding task set 47.0 with 1 tasks resource profile 0 -26/04/01 06:39:17 INFO TaskSetManager: Starting task 0.0 in stage 47.0 (TID 536) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9404 bytes) -26/04/01 06:39:17 INFO Executor: Running task 0.0 in stage 47.0 (TID 536) -26/04/01 06:39:17 INFO CodeGenerator: Code generated in 3.410292 ms -26/04/01 06:39:17 INFO ShuffleBlockFetcherIterator: Getting 25 (54.3 KiB) non-empty blocks including 25 (54.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:39:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:39:17 INFO Executor: Finished task 0.0 in stage 47.0 (TID 536). 6566 bytes result sent to driver -26/04/01 06:39:17 INFO TaskSetManager: Finished task 0.0 in stage 47.0 (TID 536) in 49 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:39:17 INFO TaskSchedulerImpl: Removed TaskSet 47.0, whose tasks have all completed, from pool -26/04/01 06:39:17 INFO DAGScheduler: ShuffleMapStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.052 s -26/04/01 06:39:17 INFO DAGScheduler: looking for newly runnable stages -26/04/01 06:39:17 INFO DAGScheduler: running: Set() -26/04/01 06:39:17 INFO DAGScheduler: waiting: Set() -26/04/01 06:39:17 INFO DAGScheduler: failed: Set() -26/04/01 06:39:17 INFO ShufflePartitionsUtil: For shuffle(8), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 06:39:17 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 06:39:17 INFO DAGScheduler: Got job 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 06:39:17 INFO DAGScheduler: Final stage: ResultStage 56 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 06:39:17 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 55) -26/04/01 06:39:17 INFO DAGScheduler: Missing parents: List() -26/04/01 06:39:17 INFO DAGScheduler: Submitting ResultStage 56 (MapPartitionsRDD[64] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 06:39:17 INFO MemoryStore: Block broadcast_30 stored as values in memory (estimated size 18.6 KiB, free 12.6 GiB) -26/04/01 06:39:17 INFO MemoryStore: Block broadcast_30_piece0 stored as bytes in memory (estimated size 8.8 KiB, free 12.6 GiB) -26/04/01 06:39:17 INFO BlockManagerInfo: Added broadcast_30_piece0 in memory on 10.0.0.133:58442 (size: 8.8 KiB, free: 12.6 GiB) -26/04/01 06:39:17 INFO SparkContext: Created broadcast 30 from broadcast at DAGScheduler.scala:1611 -26/04/01 06:39:17 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 56 (MapPartitionsRDD[64] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 06:39:17 INFO TaskSchedulerImpl: Adding task set 56.0 with 1 tasks resource profile 0 -26/04/01 06:39:17 INFO TaskSetManager: Starting task 0.0 in stage 56.0 (TID 537) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) -26/04/01 06:39:17 INFO Executor: Running task 0.0 in stage 56.0 (TID 537) -26/04/01 06:39:17 INFO ShuffleBlockFetcherIterator: Getting 1 (1620.0 B) non-empty blocks including 1 (1620.0 B) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 06:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 06:39:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=2048 MB -26/04/01 06:39:17 INFO Executor: Finished task 0.0 in stage 56.0 (TID 537). 5280 bytes result sent to driver -26/04/01 06:39:17 INFO TaskSetManager: Finished task 0.0 in stage 56.0 (TID 537) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 06:39:17 INFO TaskSchedulerImpl: Removed TaskSet 56.0, whose tasks have all completed, from pool -26/04/01 06:39:17 INFO DAGScheduler: ResultStage 56 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.006 s -26/04/01 06:39:17 INFO DAGScheduler: Job 21 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 06:39:17 INFO TaskSchedulerImpl: Killing all running tasks in stage 56: Stage finished -26/04/01 06:39:17 INFO DAGScheduler: Job 21 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.007150 s -26/04/01 06:39:17 INFO SparkContext: SparkContext is stopping with exitCode 0. -26/04/01 06:39:17 INFO CometDriverPlugin: CometDriverPlugin shutdown -26/04/01 06:39:17 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! -26/04/01 06:39:17 INFO MemoryStore: MemoryStore cleared -26/04/01 06:39:17 INFO BlockManager: BlockManager stopped -26/04/01 06:39:17 INFO BlockManagerMaster: BlockManagerMaster stopped -26/04/01 06:39:17 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! -26/04/01 06:39:17 INFO SparkContext: Successfully stopped SparkContext -26/04/01 06:39:17 INFO ShutdownHookManager: Shutdown hook called -26/04/01 06:39:17 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-3bd87ee3-3f8e-4785-87c2-97740eb236dc/pyspark-d1a60d25-754f-493a-95c3-482062c25e17 -26/04/01 06:39:17 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-3bd87ee3-3f8e-4785-87c2-97740eb236dc -26/04/01 06:39:17 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-acac7b05-334c-4692-bbb8-80e26ae2634f - 487.97 real 2095.14 user 21.36 sys - 5704859648 maximum resident set size - 0 average shared memory size - 0 average unshared data size - 0 average unshared stack size - 373734 page reclaims - 36 page faults - 0 swaps - 0 block input operations - 0 block output operations - 1274 messages sent - 2360 messages received - 18 signals received - 38220 voluntary context switches - 761034 involuntary context switches - 33268443983164 instructions retired - 7705612041471 cycles elapsed - 2460538496 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.log b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.log deleted file mode 100644 index b4ba2b8a65..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.log +++ /dev/null @@ -1,281 +0,0 @@ -Registering table customer from /opt/tpch/sf100/customer -Registering table lineitem from /opt/tpch/sf100/lineitem -Registering table nation from /opt/tpch/sf100/nation -Registering table orders from /opt/tpch/sf100/orders -Registering table part from /opt/tpch/sf100/part -Registering table partsupp from /opt/tpch/sf100/partsupp -Registering table region from /opt/tpch/sf100/region -Registering table supplier from /opt/tpch/sf100/supplier - -============================================================ -Starting iteration 1 of 1 -============================================================ - -Running query 9 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q9.sql -Executing: -- CometBench-H query 9 derived from TPC-H query 9 under the terms of the TPC Fair Use Policy. --- TP... -== Physical Plan == -AdaptiveSparkPlan (46) -+- CometSort (45) - +- CometExchange (44) - +- CometHashAggregate (43) - +- CometExchange (42) - +- CometHashAggregate (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (35) - : +- CometSortMergeJoin (34) - : :- CometSort (29) - : : +- CometExchange (28) - : : +- CometProject (27) - : : +- CometSortMergeJoin (26) - : : :- CometSort (21) - : : : +- CometExchange (20) - : : : +- CometProject (19) - : : : +- CometSortMergeJoin (18) - : : : :- CometSort (13) - : : : : +- CometExchange (12) - : : : : +- CometProject (11) - : : : : +- CometSortMergeJoin (10) - : : : : :- CometSort (5) - : : : : : +- CometExchange (4) - : : : : : +- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometNativeScan parquet (1) - : : : : +- CometSort (9) - : : : : +- CometExchange (8) - : : : : +- CometFilter (7) - : : : : +- CometNativeScan parquet (6) - : : : +- CometSort (17) - : : : +- CometExchange (16) - : : : +- CometFilter (15) - : : : +- CometNativeScan parquet (14) - : : +- CometSort (25) - : : +- CometExchange (24) - : : +- CometFilter (23) - : : +- CometNativeScan parquet (22) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometFilter (31) - : +- CometNativeScan parquet (30) - +- CometBroadcastExchange (38) - +- CometFilter (37) - +- CometNativeScan parquet (36) - - -(1) CometNativeScan parquet -Output [2]: [p_partkey#74L, p_name#75] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/part] -PushedFilters: [IsNotNull(p_name), StringContains(p_name,moccasin), IsNotNull(p_partkey)] -ReadSchema: struct - -(2) CometFilter -Input [2]: [p_partkey#74L, p_name#75] -Condition : ((isnotnull(p_name#75) AND Contains(p_name#75, moccasin)) AND isnotnull(p_partkey#74L)) - -(3) CometProject -Input [2]: [p_partkey#74L, p_name#75] -Arguments: [p_partkey#74L], [p_partkey#74L] - -(4) CometExchange -Input [1]: [p_partkey#74L] -Arguments: hashpartitioning(p_partkey#74L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=376] - -(5) CometSort -Input [1]: [p_partkey#74L] -Arguments: [p_partkey#74L], [p_partkey#74L ASC NULLS FIRST] - -(6) CometNativeScan parquet -Output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] -PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] -ReadSchema: struct - -(7) CometFilter -Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Condition : ((isnotnull(l_partkey#17L) AND isnotnull(l_suppkey#18L)) AND isnotnull(l_orderkey#16L)) - -(8) CometExchange -Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: hashpartitioning(l_partkey#17L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=380] - -(9) CometSort -Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22], [l_partkey#17L ASC NULLS FIRST] - -(10) CometSortMergeJoin -Left output [1]: [p_partkey#74L] -Right output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: [p_partkey#74L], [l_partkey#17L], Inner - -(11) CometProject -Input [7]: [p_partkey#74L, l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22], [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] - -(12) CometExchange -Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: hashpartitioning(l_suppkey#18L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=384] - -(13) CometSort -Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22], [l_suppkey#18L ASC NULLS FIRST] - -(14) CometNativeScan parquet -Output [2]: [s_suppkey#108L, s_nationkey#111L] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/supplier] -PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] -ReadSchema: struct - -(15) CometFilter -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Condition : (isnotnull(s_suppkey#108L) AND isnotnull(s_nationkey#111L)) - -(16) CometExchange -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: hashpartitioning(s_suppkey#108L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=388] - -(17) CometSort -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: [s_suppkey#108L, s_nationkey#111L], [s_suppkey#108L ASC NULLS FIRST] - -(18) CometSortMergeJoin -Left output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Right output [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: [l_suppkey#18L], [s_suppkey#108L], Inner - -(19) CometProject -Input [8]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_suppkey#108L, s_nationkey#111L] -Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L], [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] - -(20) CometExchange -Input [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] -Arguments: hashpartitioning(l_suppkey#18L, l_partkey#17L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=392] - -(21) CometSort -Input [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] -Arguments: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L], [l_suppkey#18L ASC NULLS FIRST, l_partkey#17L ASC NULLS FIRST] - -(22) CometNativeScan parquet -Output [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/partsupp] -PushedFilters: [IsNotNull(ps_suppkey), IsNotNull(ps_partkey)] -ReadSchema: struct - -(23) CometFilter -Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Condition : (isnotnull(ps_suppkey#93L) AND isnotnull(ps_partkey#92L)) - -(24) CometExchange -Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Arguments: hashpartitioning(ps_suppkey#93L, ps_partkey#92L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=396] - -(25) CometSort -Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Arguments: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95], [ps_suppkey#93L ASC NULLS FIRST, ps_partkey#92L ASC NULLS FIRST] - -(26) CometSortMergeJoin -Left output [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] -Right output [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Arguments: [l_suppkey#18L, l_partkey#17L], [ps_suppkey#93L, ps_partkey#92L], Inner - -(27) CometProject -Input [10]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Arguments: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95], [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] - -(28) CometExchange -Input [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] -Arguments: hashpartitioning(l_orderkey#16L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=400] - -(29) CometSort -Input [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] -Arguments: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95], [l_orderkey#16L ASC NULLS FIRST] - -(30) CometNativeScan parquet -Output [2]: [o_orderkey#56L, o_orderdate#60] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/orders] -PushedFilters: [IsNotNull(o_orderkey)] -ReadSchema: struct - -(31) CometFilter -Input [2]: [o_orderkey#56L, o_orderdate#60] -Condition : isnotnull(o_orderkey#56L) - -(32) CometExchange -Input [2]: [o_orderkey#56L, o_orderdate#60] -Arguments: hashpartitioning(o_orderkey#56L, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=404] - -(33) CometSort -Input [2]: [o_orderkey#56L, o_orderdate#60] -Arguments: [o_orderkey#56L, o_orderdate#60], [o_orderkey#56L ASC NULLS FIRST] - -(34) CometSortMergeJoin -Left output [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] -Right output [2]: [o_orderkey#56L, o_orderdate#60] -Arguments: [l_orderkey#16L], [o_orderkey#56L], Inner - -(35) CometProject -Input [8]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderkey#56L, o_orderdate#60] -Arguments: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60], [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60] - -(36) CometNativeScan parquet -Output [2]: [n_nationkey#48L, n_name#49] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/nation] -PushedFilters: [IsNotNull(n_nationkey)] -ReadSchema: struct - -(37) CometFilter -Input [2]: [n_nationkey#48L, n_name#49] -Condition : isnotnull(n_nationkey#48L) - -(38) CometBroadcastExchange -Input [2]: [n_nationkey#48L, n_name#49] -Arguments: [n_nationkey#48L, n_name#49] - -(39) CometBroadcastHashJoin -Left output [6]: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60] -Right output [2]: [n_nationkey#48L, n_name#49] -Arguments: [s_nationkey#111L], [n_nationkey#48L], Inner, BuildRight - -(40) CometProject -Input [8]: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60, n_nationkey#48L, n_name#49] -Arguments: [nation#122, o_year#123, amount#124], [n_name#49 AS nation#122, year(o_orderdate#60) AS o_year#123, ((l_extendedprice#21 * (1 - l_discount#22)) - (ps_supplycost#95 * l_quantity#20)) AS amount#124] - -(41) CometHashAggregate -Input [3]: [nation#122, o_year#123, amount#124] -Keys [2]: [nation#122, o_year#123] -Functions [1]: [partial_sum(amount#124)] - -(42) CometExchange -Input [4]: [nation#122, o_year#123, sum#132, isEmpty#133] -Arguments: hashpartitioning(nation#122, o_year#123, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=413] - -(43) CometHashAggregate -Input [4]: [nation#122, o_year#123, sum#132, isEmpty#133] -Keys [2]: [nation#122, o_year#123] -Functions [1]: [sum(amount#124)] - -(44) CometExchange -Input [3]: [nation#122, o_year#123, sum_profit#125] -Arguments: rangepartitioning(nation#122 ASC NULLS FIRST, o_year#123 DESC NULLS LAST, 200), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=415] - -(45) CometSort -Input [3]: [nation#122, o_year#123, sum_profit#125] -Arguments: [nation#122, o_year#123, sum_profit#125], [nation#122 ASC NULLS FIRST, o_year#123 DESC NULLS LAST] - -(46) AdaptiveSparkPlan -Output [3]: [nation#122, o_year#123, sum_profit#125] -Arguments: isFinalPlan=false - - -Query 9 returned 175 rows, hash=fff4ba3023e74505f304fa3243cbeeb6 -Query 9 took 374.36 seconds - -Iteration 1 took 374.36 seconds - -Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/comet-offheap8g-q9-tpch-1775055133970.json diff --git a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.time b/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.time deleted file mode 100644 index 1b7dbc2463..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/comet-offheap8g-q9.time +++ /dev/null @@ -1,6868 +0,0 @@ -26/04/01 08:45:54 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) -26/04/01 08:45:54 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address -26/04/01 08:45:54 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable -26/04/01 08:45:55 INFO SparkContext: Running Spark version 3.5.8 -26/04/01 08:45:55 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 08:45:55 INFO SparkContext: Java version 17.0.17 -26/04/01 08:45:55 INFO ResourceUtils: ============================================================== -26/04/01 08:45:55 INFO ResourceUtils: No custom resources configured for spark.driver. -26/04/01 08:45:55 INFO ResourceUtils: ============================================================== -26/04/01 08:45:55 INFO SparkContext: Submitted application: comet-offheap8g-q9 benchmark derived from tpch -26/04/01 08:45:55 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 8192, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) -26/04/01 08:45:55 INFO ResourceProfile: Limiting resource is cpu -26/04/01 08:45:55 INFO ResourceProfileManager: Added ResourceProfile id: 0 -26/04/01 08:45:55 INFO SecurityManager: Changing view acls to: andy -26/04/01 08:45:55 INFO SecurityManager: Changing modify acls to: andy -26/04/01 08:45:55 INFO SecurityManager: Changing view acls groups to: -26/04/01 08:45:55 INFO SecurityManager: Changing modify acls groups to: -26/04/01 08:45:55 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY -26/04/01 08:45:55 INFO Utils: Successfully started service 'sparkDriver' on port 59331. -26/04/01 08:45:55 INFO SparkEnv: Registering MapOutputTracker -26/04/01 08:45:55 INFO SparkEnv: Registering BlockManagerMaster -26/04/01 08:45:55 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information -26/04/01 08:45:55 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up -26/04/01 08:45:55 INFO SparkEnv: Registering BlockManagerMasterHeartbeat -26/04/01 08:45:55 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-592952cc-8fb6-4441-88fb-62d0d58a71d3 -26/04/01 08:45:55 INFO MemoryStore: MemoryStore started with capacity 12.6 GiB -26/04/01 08:45:55 INFO SparkEnv: Registering OutputCommitCoordinator -26/04/01 08:45:55 INFO SparkContext: Added JAR file:///Users/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar at spark://10.0.0.133:59331/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775054755287 -26/04/01 08:45:55 INFO CometDriverPlugin: CometDriverPlugin init -26/04/01 08:45:55 INFO CometDriverPlugin: Setting spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions -26/04/01 08:45:55 INFO CometDriverPlugin: Comet metrics reporting is disabled. Set spark.comet.metrics.enabled=true to enable. -26/04/01 08:45:55 INFO CometDriverPlugin: Comet is running in unified memory mode and sharing off-heap memory with Spark -26/04/01 08:45:55 INFO DriverPluginContainer: Initialized driver component for plugin org.apache.spark.CometPlugin. -26/04/01 08:45:55 INFO Executor: Starting executor ID driver on host 10.0.0.133 -26/04/01 08:45:55 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 08:45:55 INFO Executor: Java version 17.0.17 -26/04/01 08:45:55 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' -26/04/01 08:45:55 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@13820261 for default. -26/04/01 08:45:55 INFO Executor: Fetching spark://10.0.0.133:59331/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar with timestamp 1775054755287 -26/04/01 08:45:55 INFO TransportClientFactory: Successfully created connection to /10.0.0.133:59331 after 8 ms (0 ms spent in bootstraps) -26/04/01 08:45:55 INFO Utils: Fetching spark://10.0.0.133:59331/jars/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-7b7c98ca-005e-4989-9d8f-bb80c5e1e895/userFiles-2ec5bf60-08ed-4482-8793-c54b3bc0e259/fetchFileTemp13161011170765808055.tmp -26/04/01 08:45:55 INFO Executor: Adding file:/private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-7b7c98ca-005e-4989-9d8f-bb80c5e1e895/userFiles-2ec5bf60-08ed-4482-8793-c54b3bc0e259/comet-spark-spark3.5_2.12-0.15.0-SNAPSHOT.jar to class loader default -26/04/01 08:45:55 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 59333. -26/04/01 08:45:55 INFO NettyBlockTransferService: Server created on 10.0.0.133:59333 -26/04/01 08:45:55 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy -26/04/01 08:45:55 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 59333, None) -26/04/01 08:45:55 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:59333 with 12.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 59333, None) -26/04/01 08:45:55 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 59333, None) -26/04/01 08:45:55 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 59333, None) -26/04/01 08:45:55 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. -26/04/01 08:45:55 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. -26/04/01 08:45:56 INFO InMemoryFileIndex: It took 17 ms to list leaf files for 1 paths. -26/04/01 08:45:56 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:45:56 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:45:56 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:45:56 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:45:56 INFO DAGScheduler: Missing parents: List() -26/04/01 08:45:56 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:45:56 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 08:45:56 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) -26/04/01 08:45:56 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:56 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:45:56 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:45:56 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 -26/04/01 08:45:56 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 08:45:56 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) -26/04/01 08:45:56 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 2041 bytes result sent to driver -26/04/01 08:45:56 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 125 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:45:56 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool -26/04/01 08:45:56 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.407 s -26/04/01 08:45:56 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:45:56 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished -26/04/01 08:45:56 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.418894 s -26/04/01 08:45:59 INFO NativeBase: Couldn't locate log file from either COMET_CONF_DIR or comet.log.file.path. Using default log configuration with INFO log level which emits to stderr -26/04/01 08:45:59 INFO core/src/lib.rs: Comet native library version 0.15.0 initialized -26/04/01 08:45:59 INFO NativeBase: Setting system property arrow.enable_unsafe_memory_access to true -26/04/01 08:45:59 INFO NativeBase: Setting system property arrow.enable_null_check_for_get to false -26/04/01 08:45:59 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. -26/04/01 08:45:59 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:45:59 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:45:59 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:45:59 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:45:59 INFO DAGScheduler: Missing parents: List() -26/04/01 08:45:59 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:45:59 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 08:45:59 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) -26/04/01 08:45:59 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:59 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:45:59 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:45:59 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 -26/04/01 08:45:59 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 08:45:59 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) -26/04/01 08:45:59 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2344 bytes result sent to driver -26/04/01 08:45:59 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 16 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:45:59 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool -26/04/01 08:45:59 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.024 s -26/04/01 08:45:59 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:45:59 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished -26/04/01 08:45:59 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.026210 s -26/04/01 08:45:59 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 08:45:59 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:45:59 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:45:59 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:45:59 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:45:59 INFO DAGScheduler: Missing parents: List() -26/04/01 08:45:59 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:45:59 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 08:45:59 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) -26/04/01 08:45:59 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:59 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:45:59 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:45:59 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 -26/04/01 08:45:59 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 08:45:59 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) -26/04/01 08:45:59 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver -26/04/01 08:45:59 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 9 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:45:59 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool -26/04/01 08:45:59 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.017 s -26/04/01 08:45:59 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:45:59 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished -26/04/01 08:45:59 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.018918 s -26/04/01 08:45:59 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. -26/04/01 08:45:59 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:45:59 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:45:59 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:45:59 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:45:59 INFO DAGScheduler: Missing parents: List() -26/04/01 08:45:59 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:45:59 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 08:45:59 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) -26/04/01 08:45:59 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:59 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:45:59 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:45:59 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 -26/04/01 08:45:59 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 08:45:59 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) -26/04/01 08:45:59 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver -26/04/01 08:45:59 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 7 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:45:59 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool -26/04/01 08:45:59 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s -26/04/01 08:45:59 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:45:59 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished -26/04/01 08:45:59 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.016055 s -26/04/01 08:45:59 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. -26/04/01 08:45:59 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:45:59 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:45:59 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:45:59 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:45:59 INFO DAGScheduler: Missing parents: List() -26/04/01 08:45:59 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:45:59 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 08:45:59 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) -26/04/01 08:45:59 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:59 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:45:59 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:45:59 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 -26/04/01 08:45:59 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9299 bytes) -26/04/01 08:45:59 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) -26/04/01 08:45:59 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver -26/04/01 08:45:59 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 7 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:45:59 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool -26/04/01 08:45:59 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 08:45:59 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:45:59 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished -26/04/01 08:45:59 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.015169 s -26/04/01 08:45:59 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 08:45:59 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:45:59 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:45:59 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:45:59 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:45:59 INFO DAGScheduler: Missing parents: List() -26/04/01 08:45:59 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:45:59 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 08:45:59 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) -26/04/01 08:45:59 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:59 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:45:59 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:45:59 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 -26/04/01 08:45:59 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 08:45:59 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) -26/04/01 08:45:59 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver -26/04/01 08:45:59 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:45:59 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool -26/04/01 08:45:59 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 08:45:59 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:45:59 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished -26/04/01 08:45:59 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.014371 s -26/04/01 08:45:59 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. -26/04/01 08:45:59 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:45:59 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:45:59 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:45:59 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:45:59 INFO DAGScheduler: Missing parents: List() -26/04/01 08:45:59 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:45:59 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 08:45:59 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) -26/04/01 08:45:59 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:59 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:45:59 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:45:59 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 -26/04/01 08:45:59 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9303 bytes) -26/04/01 08:45:59 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) -26/04/01 08:45:59 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver -26/04/01 08:45:59 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:45:59 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool -26/04/01 08:45:59 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 08:45:59 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:45:59 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished -26/04/01 08:45:59 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.013381 s -26/04/01 08:45:59 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 08:45:59 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:45:59 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:45:59 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:45:59 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:45:59 INFO DAGScheduler: Missing parents: List() -26/04/01 08:45:59 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:45:59 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 104.8 KiB, free 12.6 GiB) -26/04/01 08:45:59 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 38.0 KiB, free 12.6 GiB) -26/04/01 08:45:59 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:59333 (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:59 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:45:59 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:45:59 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 -26/04/01 08:45:59 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9307 bytes) -26/04/01 08:45:59 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) -26/04/01 08:45:59 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver -26/04/01 08:45:59 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:45:59 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool -26/04/01 08:45:59 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s -26/04/01 08:45:59 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:45:59 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished -26/04/01 08:45:59 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.014852 s -26/04/01 08:45:59 INFO FileSourceStrategy: Pushed Filters: IsNotNull(p_name),StringContains(p_name,moccasin),IsNotNull(p_partkey) -26/04/01 08:45:59 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(p_name#75),Contains(p_name#75, moccasin),isnotnull(p_partkey#74L) -26/04/01 08:45:59 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_partkey),IsNotNull(l_suppkey),IsNotNull(l_orderkey) -26/04/01 08:45:59 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_partkey#17L),isnotnull(l_suppkey#18L),isnotnull(l_orderkey#16L) -26/04/01 08:45:59 INFO FileSourceStrategy: Pushed Filters: IsNotNull(s_suppkey),IsNotNull(s_nationkey) -26/04/01 08:45:59 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(s_suppkey#108L),isnotnull(s_nationkey#111L) -26/04/01 08:45:59 INFO FileSourceStrategy: Pushed Filters: IsNotNull(ps_suppkey),IsNotNull(ps_partkey) -26/04/01 08:45:59 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(ps_suppkey#93L),isnotnull(ps_partkey#92L) -26/04/01 08:45:59 INFO FileSourceStrategy: Pushed Filters: IsNotNull(o_orderkey) -26/04/01 08:45:59 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(o_orderkey#56L) -26/04/01 08:45:59 INFO FileSourceStrategy: Pushed Filters: IsNotNull(n_nationkey) -26/04/01 08:45:59 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(n_nationkey#48L) -26/04/01 08:45:59 INFO CometScanExec: Planning scan with bin packing, max size: 102898790 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:59 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:59333 in memory (size: 38.0 KiB, free: 12.6 GiB) -26/04/01 08:45:59 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:45:59 INFO CometScanExec: Planning scan with bin packing, max size: 27960063 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:45:59 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:45:59 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:45:59 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:46:00 INFO CometScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:46:00 INFO CometScanExec: Planning scan with bin packing, max size: 102898790 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:59333 (size: 35.4 KiB, free: 12.6 GiB) -26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:59333 (size: 35.4 KiB, free: 12.6 GiB) -26/04/01 08:46:00 INFO SparkContext: Created broadcast 9 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:46:00 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:46:00 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:46:00 INFO FileSourceScanExec: Planning scan with bin packing, max size: 102898790 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:46:00 INFO DAGScheduler: Registering RDD 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 -26/04/01 08:46:00 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:46:00 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 8 output partitions -26/04/01 08:46:00 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:46:00 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:46:00 INFO DAGScheduler: Missing parents: List() -26/04/01 08:46:00 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[21] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:46:00 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 20.5 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 8.1 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:59333 (size: 8.1 KiB, free: 12.6 GiB) -26/04/01 08:46:00 INFO SparkContext: Created broadcast 11 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:46:00 INFO DAGScheduler: Submitting 8 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[21] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7)) -26/04/01 08:46:00 INFO TaskSchedulerImpl: Adding task set 8.0 with 8 tasks resource profile 0 -26/04/01 08:46:00 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 08:46:00 INFO DAGScheduler: Final stage: ResultStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:46:00 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:46:00 INFO DAGScheduler: Missing parents: List() -26/04/01 08:46:00 INFO DAGScheduler: Submitting ResultStage 9 (MapPartitionsRDD[20] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:46:00 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9429 bytes) -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO TaskSetManager: Starting task 1.0 in stage 8.0 (TID 9) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9431 bytes) -26/04/01 08:46:00 INFO TaskSetManager: Starting task 2.0 in stage 8.0 (TID 10) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9431 bytes) -26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:59333 (size: 35.4 KiB, free: 12.6 GiB) -26/04/01 08:46:00 INFO TaskSetManager: Starting task 3.0 in stage 8.0 (TID 11) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9431 bytes) -26/04/01 08:46:00 INFO TaskSetManager: Starting task 4.0 in stage 8.0 (TID 12) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9430 bytes) -26/04/01 08:46:00 INFO SparkContext: Created broadcast 10 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:46:00 INFO TaskSetManager: Starting task 5.0 in stage 8.0 (TID 13) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9432 bytes) -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 10.8 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO TaskSetManager: Starting task 6.0 in stage 8.0 (TID 14) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9431 bytes) -26/04/01 08:46:00 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:46:00 INFO TaskSetManager: Starting task 7.0 in stage 8.0 (TID 15) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9432 bytes) -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 5.3 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) -26/04/01 08:46:00 INFO Executor: Running task 1.0 in stage 8.0 (TID 9) -26/04/01 08:46:00 INFO Executor: Running task 2.0 in stage 8.0 (TID 10) -26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:59333 (size: 5.3 KiB, free: 12.6 GiB) -26/04/01 08:46:00 INFO Executor: Running task 3.0 in stage 8.0 (TID 11) -26/04/01 08:46:00 INFO Executor: Running task 4.0 in stage 8.0 (TID 12) -26/04/01 08:46:00 INFO SparkContext: Created broadcast 12 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:46:00 INFO Executor: Running task 5.0 in stage 8.0 (TID 13) -26/04/01 08:46:00 INFO Executor: Running task 6.0 in stage 8.0 (TID 14) -26/04/01 08:46:00 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 9 (MapPartitionsRDD[20] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:46:00 INFO TaskSchedulerImpl: Adding task set 9.0 with 1 tasks resource profile 0 -26/04/01 08:46:00 INFO Executor: Running task 7.0 in stage 8.0 (TID 15) -26/04/01 08:46:00 INFO DAGScheduler: Registering RDD 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 -26/04/01 08:46:00 INFO CometScanExec: Planning scan with bin packing, max size: 27960063 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:46:00 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions -26/04/01 08:46:00 INFO DAGScheduler: Final stage: ShuffleMapStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:46:00 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:46:00 INFO DAGScheduler: Missing parents: List() -26/04/01 08:46:00 INFO DAGScheduler: Submitting ShuffleMapStage 10 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_13 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_14 stored as values in memory (estimated size 20.6 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_14_piece0 stored as bytes in memory (estimated size 8.2 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_14_piece0 in memory on 10.0.0.133:59333 (size: 8.2 KiB, free: 12.6 GiB) -26/04/01 08:46:00 INFO SparkContext: Created broadcast 14 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:46:00 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 10 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:46:00 INFO TaskSchedulerImpl: Adding task set 10.0 with 208 tasks resource profile 0 -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_13_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_13_piece0 in memory on 10.0.0.133:59333 (size: 35.4 KiB, free: 12.6 GiB) -26/04/01 08:46:00 INFO SparkContext: Created broadcast 13 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:46:00 INFO FileSourceScanExec: Planning scan with bin packing, max size: 27960063 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:46:00 INFO DAGScheduler: Registering RDD 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 2 -26/04/01 08:46:00 INFO DAGScheduler: Got map stage job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 8 output partitions -26/04/01 08:46:00 INFO DAGScheduler: Final stage: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:46:00 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:46:00 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:46:00 INFO DAGScheduler: Missing parents: List() -26/04/01 08:46:00 INFO DAGScheduler: Submitting ShuffleMapStage 11 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_15 stored as values in memory (estimated size 18.0 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_15_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_15_piece0 in memory on 10.0.0.133:59333 (size: 7.9 KiB, free: 12.6 GiB) -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_16 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO SparkContext: Created broadcast 15 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:46:00 INFO DAGScheduler: Submitting 8 missing tasks from ShuffleMapStage 11 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7)) -26/04/01 08:46:00 INFO TaskSchedulerImpl: Adding task set 11.0 with 8 tasks resource profile 0 -26/04/01 08:46:00 INFO BaseAllocator: Debug mode disabled. Enable with the VM option -Darrow.memory.debug.allocator=true. -26/04/01 08:46:00 INFO DefaultAllocationManagerOption: allocation manager type not specified, using netty as the default type -26/04/01 08:46:00 INFO CheckAllocator: Using DefaultAllocationManager at memory/unsafe/DefaultAllocationManagerFactory.class -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_16_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_16_piece0 in memory on 10.0.0.133:59333 (size: 35.4 KiB, free: 12.6 GiB) -26/04/01 08:46:00 INFO SparkContext: Created broadcast 16 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:46:00 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:46:00 INFO DAGScheduler: Registering RDD 30 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 3 -26/04/01 08:46:00 INFO DAGScheduler: Got map stage job 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 37 output partitions -26/04/01 08:46:00 INFO CometScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:46:00 INFO DAGScheduler: Final stage: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:46:00 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:46:00 INFO DAGScheduler: Missing parents: List() -26/04/01 08:46:00 INFO DAGScheduler: Submitting ShuffleMapStage 12 (MapPartitionsRDD[30] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_18 stored as values in memory (estimated size 18.2 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_18_piece0 stored as bytes in memory (estimated size 8.0 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_17 stored as values in memory (estimated size 203.0 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_18_piece0 in memory on 10.0.0.133:59333 (size: 8.0 KiB, free: 12.6 GiB) -26/04/01 08:46:00 INFO core/src/execution/jni_api.rs: Comet tokio runtime: using spark.executor.cores=8 worker threads -26/04/01 08:46:00 INFO SparkContext: Created broadcast 18 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:46:00 INFO DAGScheduler: Submitting 37 missing tasks from ShuffleMapStage 12 (MapPartitionsRDD[30] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:46:00 INFO TaskSchedulerImpl: Adding task set 12.0 with 37 tasks resource profile 0 -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_17_piece0 stored as bytes in memory (estimated size 35.4 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_17_piece0 in memory on 10.0.0.133:59333 (size: 35.4 KiB, free: 12.6 GiB) -26/04/01 08:46:00 INFO SparkContext: Created broadcast 17 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:46:00 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:46:00 INFO DAGScheduler: Registering RDD 33 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 4 -26/04/01 08:46:00 INFO DAGScheduler: Got map stage job 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 64 output partitions -26/04/01 08:46:00 INFO DAGScheduler: Final stage: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:46:00 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:46:00 INFO DAGScheduler: Missing parents: List() -26/04/01 08:46:00 INFO DAGScheduler: Submitting ShuffleMapStage 13 (MapPartitionsRDD[33] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_19 stored as values in memory (estimated size 16.6 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO MemoryStore: Block broadcast_19_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 12.6 GiB) -26/04/01 08:46:00 INFO BlockManagerInfo: Added broadcast_19_piece0 in memory on 10.0.0.133:59333 (size: 7.9 KiB, free: 12.6 GiB) -26/04/01 08:46:00 INFO SparkContext: Created broadcast 19 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:46:00 INFO DAGScheduler: Submitting 64 missing tasks from ShuffleMapStage 13 (MapPartitionsRDD[33] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:46:00 INFO TaskSchedulerImpl: Adding task set 13.0 with 64 tasks resource profile 0 -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO Executor: Finished task 6.0 in stage 8.0 (TID 14). 6761 bytes result sent to driver -26/04/01 08:46:02 INFO Executor: Finished task 5.0 in stage 8.0 (TID 13). 6761 bytes result sent to driver -26/04/01 08:46:02 INFO Executor: Finished task 2.0 in stage 8.0 (TID 10). 6761 bytes result sent to driver -26/04/01 08:46:02 INFO Executor: Finished task 1.0 in stage 8.0 (TID 9). 6761 bytes result sent to driver -26/04/01 08:46:02 INFO Executor: Finished task 3.0 in stage 8.0 (TID 11). 6761 bytes result sent to driver -26/04/01 08:46:02 INFO Executor: Finished task 7.0 in stage 8.0 (TID 15). 6761 bytes result sent to driver -26/04/01 08:46:02 INFO Executor: Finished task 4.0 in stage 8.0 (TID 12). 6761 bytes result sent to driver -26/04/01 08:46:02 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 6761 bytes result sent to driver -26/04/01 08:46:02 INFO TaskSetManager: Starting task 0.0 in stage 9.0 (TID 16) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:46:02 INFO Executor: Running task 0.0 in stage 9.0 (TID 16) -26/04/01 08:46:02 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 17) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:46:02 INFO TaskSetManager: Finished task 6.0 in stage 8.0 (TID 14) in 2484 ms on 10.0.0.133 (executor driver) (1/8) -26/04/01 08:46:02 INFO Executor: Running task 0.0 in stage 10.0 (TID 17) -26/04/01 08:46:02 INFO TaskSetManager: Starting task 1.0 in stage 10.0 (TID 18) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:02 INFO TaskSetManager: Finished task 5.0 in stage 8.0 (TID 13) in 2484 ms on 10.0.0.133 (executor driver) (2/8) -26/04/01 08:46:02 INFO Executor: Running task 1.0 in stage 10.0 (TID 18) -26/04/01 08:46:02 INFO TaskSetManager: Starting task 2.0 in stage 10.0 (TID 19) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:02 INFO Executor: Running task 2.0 in stage 10.0 (TID 19) -26/04/01 08:46:02 INFO TaskSetManager: Starting task 3.0 in stage 10.0 (TID 20) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:02 INFO TaskSetManager: Finished task 1.0 in stage 8.0 (TID 9) in 2485 ms on 10.0.0.133 (executor driver) (3/8) -26/04/01 08:46:02 INFO Executor: Running task 3.0 in stage 10.0 (TID 20) -26/04/01 08:46:02 INFO TaskSetManager: Starting task 4.0 in stage 10.0 (TID 21) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:02 INFO Executor: Running task 4.0 in stage 10.0 (TID 21) -26/04/01 08:46:02 INFO TaskSetManager: Starting task 5.0 in stage 10.0 (TID 22) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:02 INFO TaskSetManager: Finished task 7.0 in stage 8.0 (TID 15) in 2485 ms on 10.0.0.133 (executor driver) (4/8) -26/04/01 08:46:02 INFO Executor: Running task 5.0 in stage 10.0 (TID 22) -26/04/01 08:46:02 INFO TaskSetManager: Starting task 6.0 in stage 10.0 (TID 23) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:46:02 INFO TaskSetManager: Finished task 3.0 in stage 8.0 (TID 11) in 2486 ms on 10.0.0.133 (executor driver) (5/8) -26/04/01 08:46:02 INFO Executor: Running task 6.0 in stage 10.0 (TID 23) -26/04/01 08:46:02 INFO TaskSetManager: Finished task 2.0 in stage 8.0 (TID 10) in 2486 ms on 10.0.0.133 (executor driver) (6/8) -26/04/01 08:46:02 INFO TaskSetManager: Finished task 4.0 in stage 8.0 (TID 12) in 2487 ms on 10.0.0.133 (executor driver) (7/8) -26/04/01 08:46:02 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 2488 ms on 10.0.0.133 (executor driver) (8/8) -26/04/01 08:46:02 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool -26/04/01 08:46:02 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.498 s -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:46:02 INFO DAGScheduler: running: Set(ShuffleMapStage 12, ResultStage 9, ShuffleMapStage 13, ShuffleMapStage 10, ShuffleMapStage 11) -26/04/01 08:46:02 INFO DAGScheduler: waiting: Set() -26/04/01 08:46:02 INFO DAGScheduler: failed: Set() -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO ShufflePartitionsUtil: For shuffle(0), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 08:46:02 INFO Executor: Finished task 0.0 in stage 9.0 (TID 16). 4805 bytes result sent to driver -26/04/01 08:46:02 INFO TaskSetManager: Starting task 7.0 in stage 10.0 (TID 24) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:02 INFO Executor: Running task 7.0 in stage 10.0 (TID 24) -26/04/01 08:46:02 INFO TaskSetManager: Finished task 0.0 in stage 9.0 (TID 16) in 44 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:46:02 INFO TaskSchedulerImpl: Removed TaskSet 9.0, whose tasks have all completed, from pool -26/04/01 08:46:02 INFO DAGScheduler: ResultStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 2.527 s -26/04/01 08:46:02 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:46:02 INFO TaskSchedulerImpl: Killing all running tasks in stage 9: Stage finished -26/04/01 08:46:02 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 2.537089 s -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:02 INFO Utils: Coalesced 1 broadcast batches into 1 (25 rows) -26/04/01 08:46:02 INFO MemoryStore: Block broadcast_20 stored as values in memory (estimated size 880.0 B, free 12.6 GiB) -26/04/01 08:46:02 INFO MemoryStore: Block broadcast_20_piece0 stored as bytes in memory (estimated size 831.0 B, free 12.6 GiB) -26/04/01 08:46:02 INFO BlockManagerInfo: Added broadcast_20_piece0 in memory on 10.0.0.133:59333 (size: 831.0 B, free: 12.6 GiB) -26/04/01 08:46:02 INFO SparkContext: Created broadcast 20 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:46:02 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:46:02 INFO DAGScheduler: Got job 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions -26/04/01 08:46:02 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:46:02 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 14) -26/04/01 08:46:02 INFO DAGScheduler: Missing parents: List() -26/04/01 08:46:02 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[35] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:46:02 INFO MemoryStore: Block broadcast_21 stored as values in memory (estimated size 14.5 KiB, free 12.6 GiB) -26/04/01 08:46:02 INFO MemoryStore: Block broadcast_21_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 12.6 GiB) -26/04/01 08:46:02 INFO BlockManagerInfo: Added broadcast_21_piece0 in memory on 10.0.0.133:59333 (size: 7.9 KiB, free: 12.6 GiB) -26/04/01 08:46:02 INFO SparkContext: Created broadcast 21 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:46:02 INFO DAGScheduler: Submitting 200 missing tasks from ResultStage 15 (MapPartitionsRDD[35] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:46:02 INFO TaskSchedulerImpl: Adding task set 15.0 with 200 tasks resource profile 0 -26/04/01 08:46:10 INFO Executor: Finished task 1.0 in stage 10.0 (TID 18). 6649 bytes result sent to driver -26/04/01 08:46:10 INFO TaskSetManager: Starting task 8.0 in stage 10.0 (TID 25) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:10 INFO Executor: Running task 8.0 in stage 10.0 (TID 25) -26/04/01 08:46:10 INFO TaskSetManager: Finished task 1.0 in stage 10.0 (TID 18) in 8046 ms on 10.0.0.133 (executor driver) (1/208) -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:10 INFO Executor: Finished task 0.0 in stage 10.0 (TID 17). 6649 bytes result sent to driver -26/04/01 08:46:10 INFO TaskSetManager: Starting task 9.0 in stage 10.0 (TID 26) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:10 INFO Executor: Running task 9.0 in stage 10.0 (TID 26) -26/04/01 08:46:10 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 17) in 8054 ms on 10.0.0.133 (executor driver) (2/208) -26/04/01 08:46:10 INFO Executor: Finished task 5.0 in stage 10.0 (TID 22). 6649 bytes result sent to driver -26/04/01 08:46:10 INFO TaskSetManager: Starting task 10.0 in stage 10.0 (TID 27) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:10 INFO Executor: Running task 10.0 in stage 10.0 (TID 27) -26/04/01 08:46:10 INFO TaskSetManager: Finished task 5.0 in stage 10.0 (TID 22) in 8053 ms on 10.0.0.133 (executor driver) (3/208) -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:10 INFO Executor: Finished task 6.0 in stage 10.0 (TID 23). 6649 bytes result sent to driver -26/04/01 08:46:10 INFO TaskSetManager: Starting task 11.0 in stage 10.0 (TID 28) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:10 INFO TaskSetManager: Finished task 6.0 in stage 10.0 (TID 23) in 8059 ms on 10.0.0.133 (executor driver) (4/208) -26/04/01 08:46:10 INFO Executor: Running task 11.0 in stage 10.0 (TID 28) -26/04/01 08:46:10 INFO Executor: Finished task 7.0 in stage 10.0 (TID 24). 6649 bytes result sent to driver -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:10 INFO TaskSetManager: Starting task 12.0 in stage 10.0 (TID 29) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:46:10 INFO TaskSetManager: Finished task 7.0 in stage 10.0 (TID 24) in 8021 ms on 10.0.0.133 (executor driver) (5/208) -26/04/01 08:46:10 INFO Executor: Running task 12.0 in stage 10.0 (TID 29) -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:10 INFO Executor: Finished task 2.0 in stage 10.0 (TID 19). 6649 bytes result sent to driver -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:10 INFO TaskSetManager: Starting task 13.0 in stage 10.0 (TID 30) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:10 INFO Executor: Running task 13.0 in stage 10.0 (TID 30) -26/04/01 08:46:10 INFO TaskSetManager: Finished task 2.0 in stage 10.0 (TID 19) in 8066 ms on 10.0.0.133 (executor driver) (6/208) -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:10 INFO Executor: Finished task 4.0 in stage 10.0 (TID 21). 6649 bytes result sent to driver -26/04/01 08:46:10 INFO TaskSetManager: Starting task 14.0 in stage 10.0 (TID 31) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:10 INFO Executor: Running task 14.0 in stage 10.0 (TID 31) -26/04/01 08:46:10 INFO TaskSetManager: Finished task 4.0 in stage 10.0 (TID 21) in 8080 ms on 10.0.0.133 (executor driver) (7/208) -26/04/01 08:46:10 INFO Executor: Finished task 3.0 in stage 10.0 (TID 20). 6649 bytes result sent to driver -26/04/01 08:46:10 INFO TaskSetManager: Starting task 15.0 in stage 10.0 (TID 32) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:10 INFO Executor: Running task 15.0 in stage 10.0 (TID 32) -26/04/01 08:46:10 INFO TaskSetManager: Finished task 3.0 in stage 10.0 (TID 20) in 8082 ms on 10.0.0.133 (executor driver) (8/208) -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO Executor: Finished task 9.0 in stage 10.0 (TID 26). 6649 bytes result sent to driver -26/04/01 08:46:18 INFO TaskSetManager: Starting task 16.0 in stage 10.0 (TID 33) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:18 INFO TaskSetManager: Finished task 9.0 in stage 10.0 (TID 26) in 7900 ms on 10.0.0.133 (executor driver) (9/208) -26/04/01 08:46:18 INFO Executor: Running task 16.0 in stage 10.0 (TID 33) -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO Executor: Finished task 14.0 in stage 10.0 (TID 31). 6606 bytes result sent to driver -26/04/01 08:46:18 INFO TaskSetManager: Starting task 17.0 in stage 10.0 (TID 34) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:18 INFO Executor: Finished task 8.0 in stage 10.0 (TID 25). 6649 bytes result sent to driver -26/04/01 08:46:18 INFO Executor: Running task 17.0 in stage 10.0 (TID 34) -26/04/01 08:46:18 INFO TaskSetManager: Finished task 14.0 in stage 10.0 (TID 31) in 7890 ms on 10.0.0.133 (executor driver) (10/208) -26/04/01 08:46:18 INFO TaskSetManager: Starting task 18.0 in stage 10.0 (TID 35) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:46:18 INFO Executor: Running task 18.0 in stage 10.0 (TID 35) -26/04/01 08:46:18 INFO TaskSetManager: Finished task 8.0 in stage 10.0 (TID 25) in 7926 ms on 10.0.0.133 (executor driver) (11/208) -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO Executor: Finished task 10.0 in stage 10.0 (TID 27). 6606 bytes result sent to driver -26/04/01 08:46:18 INFO TaskSetManager: Starting task 19.0 in stage 10.0 (TID 36) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:46:18 INFO TaskSetManager: Finished task 10.0 in stage 10.0 (TID 27) in 7922 ms on 10.0.0.133 (executor driver) (12/208) -26/04/01 08:46:18 INFO Executor: Running task 19.0 in stage 10.0 (TID 36) -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO Executor: Finished task 15.0 in stage 10.0 (TID 32). 6606 bytes result sent to driver -26/04/01 08:46:18 INFO Executor: Finished task 12.0 in stage 10.0 (TID 29). 6606 bytes result sent to driver -26/04/01 08:46:18 INFO TaskSetManager: Starting task 20.0 in stage 10.0 (TID 37) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:18 INFO Executor: Running task 20.0 in stage 10.0 (TID 37) -26/04/01 08:46:18 INFO TaskSetManager: Starting task 21.0 in stage 10.0 (TID 38) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:18 INFO TaskSetManager: Finished task 15.0 in stage 10.0 (TID 32) in 7935 ms on 10.0.0.133 (executor driver) (13/208) -26/04/01 08:46:18 INFO Executor: Running task 21.0 in stage 10.0 (TID 38) -26/04/01 08:46:18 INFO TaskSetManager: Finished task 12.0 in stage 10.0 (TID 29) in 7955 ms on 10.0.0.133 (executor driver) (14/208) -26/04/01 08:46:18 INFO Executor: Finished task 13.0 in stage 10.0 (TID 30). 6606 bytes result sent to driver -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO TaskSetManager: Starting task 22.0 in stage 10.0 (TID 39) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:18 INFO Executor: Running task 22.0 in stage 10.0 (TID 39) -26/04/01 08:46:18 INFO TaskSetManager: Finished task 13.0 in stage 10.0 (TID 30) in 7953 ms on 10.0.0.133 (executor driver) (15/208) -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO Executor: Finished task 11.0 in stage 10.0 (TID 28). 6606 bytes result sent to driver -26/04/01 08:46:18 INFO TaskSetManager: Starting task 23.0 in stage 10.0 (TID 40) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:18 INFO Executor: Running task 23.0 in stage 10.0 (TID 40) -26/04/01 08:46:18 INFO TaskSetManager: Finished task 11.0 in stage 10.0 (TID 28) in 7966 ms on 10.0.0.133 (executor driver) (16/208) -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO Executor: Finished task 17.0 in stage 10.0 (TID 34). 6606 bytes result sent to driver -26/04/01 08:46:26 INFO TaskSetManager: Starting task 24.0 in stage 10.0 (TID 41) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:46:26 INFO TaskSetManager: Finished task 17.0 in stage 10.0 (TID 34) in 7926 ms on 10.0.0.133 (executor driver) (17/208) -26/04/01 08:46:26 INFO Executor: Running task 24.0 in stage 10.0 (TID 41) -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO Executor: Finished task 18.0 in stage 10.0 (TID 35). 6606 bytes result sent to driver -26/04/01 08:46:26 INFO TaskSetManager: Starting task 25.0 in stage 10.0 (TID 42) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:26 INFO TaskSetManager: Finished task 18.0 in stage 10.0 (TID 35) in 7930 ms on 10.0.0.133 (executor driver) (18/208) -26/04/01 08:46:26 INFO Executor: Running task 25.0 in stage 10.0 (TID 42) -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO Executor: Finished task 16.0 in stage 10.0 (TID 33). 6606 bytes result sent to driver -26/04/01 08:46:26 INFO TaskSetManager: Starting task 26.0 in stage 10.0 (TID 43) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:26 INFO TaskSetManager: Finished task 16.0 in stage 10.0 (TID 33) in 7955 ms on 10.0.0.133 (executor driver) (19/208) -26/04/01 08:46:26 INFO Executor: Running task 26.0 in stage 10.0 (TID 43) -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO Executor: Finished task 19.0 in stage 10.0 (TID 36). 6606 bytes result sent to driver -26/04/01 08:46:26 INFO TaskSetManager: Starting task 27.0 in stage 10.0 (TID 44) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO TaskSetManager: Finished task 19.0 in stage 10.0 (TID 36) in 7934 ms on 10.0.0.133 (executor driver) (20/208) -26/04/01 08:46:26 INFO Executor: Running task 27.0 in stage 10.0 (TID 44) -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO Executor: Finished task 22.0 in stage 10.0 (TID 39). 6606 bytes result sent to driver -26/04/01 08:46:26 INFO TaskSetManager: Starting task 28.0 in stage 10.0 (TID 45) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:26 INFO Executor: Running task 28.0 in stage 10.0 (TID 45) -26/04/01 08:46:26 INFO TaskSetManager: Finished task 22.0 in stage 10.0 (TID 39) in 7900 ms on 10.0.0.133 (executor driver) (21/208) -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO Executor: Finished task 21.0 in stage 10.0 (TID 38). 6606 bytes result sent to driver -26/04/01 08:46:26 INFO TaskSetManager: Starting task 29.0 in stage 10.0 (TID 46) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:26 INFO TaskSetManager: Finished task 21.0 in stage 10.0 (TID 38) in 7924 ms on 10.0.0.133 (executor driver) (22/208) -26/04/01 08:46:26 INFO Executor: Running task 29.0 in stage 10.0 (TID 46) -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO Executor: Finished task 23.0 in stage 10.0 (TID 40). 6606 bytes result sent to driver -26/04/01 08:46:26 INFO TaskSetManager: Starting task 30.0 in stage 10.0 (TID 47) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:46:26 INFO TaskSetManager: Finished task 23.0 in stage 10.0 (TID 40) in 7927 ms on 10.0.0.133 (executor driver) (23/208) -26/04/01 08:46:26 INFO Executor: Running task 30.0 in stage 10.0 (TID 47) -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO Executor: Finished task 20.0 in stage 10.0 (TID 37). 6606 bytes result sent to driver -26/04/01 08:46:26 INFO TaskSetManager: Starting task 31.0 in stage 10.0 (TID 48) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:26 INFO TaskSetManager: Finished task 20.0 in stage 10.0 (TID 37) in 7944 ms on 10.0.0.133 (executor driver) (24/208) -26/04/01 08:46:26 INFO Executor: Running task 31.0 in stage 10.0 (TID 48) -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO Executor: Finished task 25.0 in stage 10.0 (TID 42). 6649 bytes result sent to driver -26/04/01 08:46:34 INFO TaskSetManager: Starting task 32.0 in stage 10.0 (TID 49) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:34 INFO Executor: Running task 32.0 in stage 10.0 (TID 49) -26/04/01 08:46:34 INFO TaskSetManager: Finished task 25.0 in stage 10.0 (TID 42) in 7892 ms on 10.0.0.133 (executor driver) (25/208) -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO Executor: Finished task 28.0 in stage 10.0 (TID 45). 6606 bytes result sent to driver -26/04/01 08:46:34 INFO TaskSetManager: Starting task 33.0 in stage 10.0 (TID 50) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:34 INFO TaskSetManager: Finished task 28.0 in stage 10.0 (TID 45) in 7914 ms on 10.0.0.133 (executor driver) (26/208) -26/04/01 08:46:34 INFO Executor: Running task 33.0 in stage 10.0 (TID 50) -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO Executor: Finished task 26.0 in stage 10.0 (TID 43). 6606 bytes result sent to driver -26/04/01 08:46:34 INFO TaskSetManager: Starting task 34.0 in stage 10.0 (TID 51) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:34 INFO TaskSetManager: Finished task 26.0 in stage 10.0 (TID 43) in 7929 ms on 10.0.0.133 (executor driver) (27/208) -26/04/01 08:46:34 INFO Executor: Running task 34.0 in stage 10.0 (TID 51) -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO Executor: Finished task 27.0 in stage 10.0 (TID 44). 6606 bytes result sent to driver -26/04/01 08:46:34 INFO TaskSetManager: Starting task 35.0 in stage 10.0 (TID 52) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:34 INFO TaskSetManager: Finished task 27.0 in stage 10.0 (TID 44) in 7929 ms on 10.0.0.133 (executor driver) (28/208) -26/04/01 08:46:34 INFO Executor: Running task 35.0 in stage 10.0 (TID 52) -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO Executor: Finished task 24.0 in stage 10.0 (TID 41). 6606 bytes result sent to driver -26/04/01 08:46:34 INFO TaskSetManager: Starting task 36.0 in stage 10.0 (TID 53) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:46:34 INFO Executor: Running task 36.0 in stage 10.0 (TID 53) -26/04/01 08:46:34 INFO TaskSetManager: Finished task 24.0 in stage 10.0 (TID 41) in 7946 ms on 10.0.0.133 (executor driver) (29/208) -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO Executor: Finished task 29.0 in stage 10.0 (TID 46). 6606 bytes result sent to driver -26/04/01 08:46:34 INFO TaskSetManager: Starting task 37.0 in stage 10.0 (TID 54) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:34 INFO Executor: Running task 37.0 in stage 10.0 (TID 54) -26/04/01 08:46:34 INFO TaskSetManager: Finished task 29.0 in stage 10.0 (TID 46) in 7908 ms on 10.0.0.133 (executor driver) (30/208) -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO Executor: Finished task 31.0 in stage 10.0 (TID 48). 6606 bytes result sent to driver -26/04/01 08:46:34 INFO TaskSetManager: Starting task 38.0 in stage 10.0 (TID 55) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:34 INFO TaskSetManager: Finished task 31.0 in stage 10.0 (TID 48) in 7918 ms on 10.0.0.133 (executor driver) (31/208) -26/04/01 08:46:34 INFO Executor: Running task 38.0 in stage 10.0 (TID 55) -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO Executor: Finished task 30.0 in stage 10.0 (TID 47). 6606 bytes result sent to driver -26/04/01 08:46:34 INFO TaskSetManager: Starting task 39.0 in stage 10.0 (TID 56) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:34 INFO Executor: Running task 39.0 in stage 10.0 (TID 56) -26/04/01 08:46:34 INFO TaskSetManager: Finished task 30.0 in stage 10.0 (TID 47) in 7938 ms on 10.0.0.133 (executor driver) (32/208) -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO Executor: Finished task 32.0 in stage 10.0 (TID 49). 6606 bytes result sent to driver -26/04/01 08:46:42 INFO TaskSetManager: Starting task 40.0 in stage 10.0 (TID 57) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:42 INFO TaskSetManager: Finished task 32.0 in stage 10.0 (TID 49) in 7939 ms on 10.0.0.133 (executor driver) (33/208) -26/04/01 08:46:42 INFO Executor: Running task 40.0 in stage 10.0 (TID 57) -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO Executor: Finished task 34.0 in stage 10.0 (TID 51). 6606 bytes result sent to driver -26/04/01 08:46:42 INFO TaskSetManager: Starting task 41.0 in stage 10.0 (TID 58) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:42 INFO TaskSetManager: Finished task 34.0 in stage 10.0 (TID 51) in 7922 ms on 10.0.0.133 (executor driver) (34/208) -26/04/01 08:46:42 INFO Executor: Running task 41.0 in stage 10.0 (TID 58) -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO Executor: Finished task 36.0 in stage 10.0 (TID 53). 6606 bytes result sent to driver -26/04/01 08:46:42 INFO TaskSetManager: Starting task 42.0 in stage 10.0 (TID 59) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:46:42 INFO TaskSetManager: Finished task 36.0 in stage 10.0 (TID 53) in 7918 ms on 10.0.0.133 (executor driver) (35/208) -26/04/01 08:46:42 INFO Executor: Running task 42.0 in stage 10.0 (TID 59) -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO Executor: Finished task 35.0 in stage 10.0 (TID 52). 6606 bytes result sent to driver -26/04/01 08:46:42 INFO TaskSetManager: Starting task 43.0 in stage 10.0 (TID 60) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:46:42 INFO Executor: Finished task 33.0 in stage 10.0 (TID 50). 6606 bytes result sent to driver -26/04/01 08:46:42 INFO Executor: Running task 43.0 in stage 10.0 (TID 60) -26/04/01 08:46:42 INFO TaskSetManager: Starting task 44.0 in stage 10.0 (TID 61) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:42 INFO TaskSetManager: Finished task 35.0 in stage 10.0 (TID 52) in 7950 ms on 10.0.0.133 (executor driver) (36/208) -26/04/01 08:46:42 INFO Executor: Running task 44.0 in stage 10.0 (TID 61) -26/04/01 08:46:42 INFO TaskSetManager: Finished task 33.0 in stage 10.0 (TID 50) in 7957 ms on 10.0.0.133 (executor driver) (37/208) -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO Executor: Finished task 37.0 in stage 10.0 (TID 54). 6606 bytes result sent to driver -26/04/01 08:46:42 INFO TaskSetManager: Starting task 45.0 in stage 10.0 (TID 62) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:42 INFO TaskSetManager: Finished task 37.0 in stage 10.0 (TID 54) in 7944 ms on 10.0.0.133 (executor driver) (38/208) -26/04/01 08:46:42 INFO Executor: Running task 45.0 in stage 10.0 (TID 62) -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO Executor: Finished task 38.0 in stage 10.0 (TID 55). 6606 bytes result sent to driver -26/04/01 08:46:42 INFO TaskSetManager: Starting task 46.0 in stage 10.0 (TID 63) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:42 INFO Executor: Running task 46.0 in stage 10.0 (TID 63) -26/04/01 08:46:42 INFO TaskSetManager: Finished task 38.0 in stage 10.0 (TID 55) in 7917 ms on 10.0.0.133 (executor driver) (39/208) -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO Executor: Finished task 39.0 in stage 10.0 (TID 56). 6606 bytes result sent to driver -26/04/01 08:46:42 INFO TaskSetManager: Starting task 47.0 in stage 10.0 (TID 64) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:42 INFO Executor: Running task 47.0 in stage 10.0 (TID 64) -26/04/01 08:46:42 INFO TaskSetManager: Finished task 39.0 in stage 10.0 (TID 56) in 7964 ms on 10.0.0.133 (executor driver) (40/208) -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:49 INFO Executor: Finished task 45.0 in stage 10.0 (TID 62). 6606 bytes result sent to driver -26/04/01 08:46:49 INFO TaskSetManager: Starting task 48.0 in stage 10.0 (TID 65) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:46:49 INFO TaskSetManager: Finished task 45.0 in stage 10.0 (TID 62) in 7595 ms on 10.0.0.133 (executor driver) (41/208) -26/04/01 08:46:49 INFO Executor: Running task 48.0 in stage 10.0 (TID 65) -26/04/01 08:46:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:50 INFO Executor: Finished task 40.0 in stage 10.0 (TID 57). 6606 bytes result sent to driver -26/04/01 08:46:50 INFO TaskSetManager: Starting task 49.0 in stage 10.0 (TID 66) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:50 INFO TaskSetManager: Finished task 40.0 in stage 10.0 (TID 57) in 7902 ms on 10.0.0.133 (executor driver) (42/208) -26/04/01 08:46:50 INFO Executor: Running task 49.0 in stage 10.0 (TID 66) -26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:50 INFO Executor: Finished task 42.0 in stage 10.0 (TID 59). 6606 bytes result sent to driver -26/04/01 08:46:50 INFO TaskSetManager: Starting task 50.0 in stage 10.0 (TID 67) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:50 INFO TaskSetManager: Finished task 42.0 in stage 10.0 (TID 59) in 7890 ms on 10.0.0.133 (executor driver) (43/208) -26/04/01 08:46:50 INFO Executor: Running task 50.0 in stage 10.0 (TID 67) -26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:50 INFO Executor: Finished task 41.0 in stage 10.0 (TID 58). 6606 bytes result sent to driver -26/04/01 08:46:50 INFO TaskSetManager: Starting task 51.0 in stage 10.0 (TID 68) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:50 INFO TaskSetManager: Finished task 41.0 in stage 10.0 (TID 58) in 7905 ms on 10.0.0.133 (executor driver) (44/208) -26/04/01 08:46:50 INFO Executor: Running task 51.0 in stage 10.0 (TID 68) -26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:50 INFO Executor: Finished task 46.0 in stage 10.0 (TID 63). 6606 bytes result sent to driver -26/04/01 08:46:50 INFO TaskSetManager: Starting task 52.0 in stage 10.0 (TID 69) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:50 INFO Executor: Running task 52.0 in stage 10.0 (TID 69) -26/04/01 08:46:50 INFO TaskSetManager: Finished task 46.0 in stage 10.0 (TID 63) in 7883 ms on 10.0.0.133 (executor driver) (45/208) -26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:50 INFO Executor: Finished task 43.0 in stage 10.0 (TID 60). 6606 bytes result sent to driver -26/04/01 08:46:50 INFO TaskSetManager: Starting task 53.0 in stage 10.0 (TID 70) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:50 INFO Executor: Running task 53.0 in stage 10.0 (TID 70) -26/04/01 08:46:50 INFO TaskSetManager: Finished task 43.0 in stage 10.0 (TID 60) in 7899 ms on 10.0.0.133 (executor driver) (46/208) -26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:50 INFO Executor: Finished task 47.0 in stage 10.0 (TID 64). 6606 bytes result sent to driver -26/04/01 08:46:50 INFO TaskSetManager: Starting task 54.0 in stage 10.0 (TID 71) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:46:50 INFO Executor: Running task 54.0 in stage 10.0 (TID 71) -26/04/01 08:46:50 INFO TaskSetManager: Finished task 47.0 in stage 10.0 (TID 64) in 7875 ms on 10.0.0.133 (executor driver) (47/208) -26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:50 INFO Executor: Finished task 44.0 in stage 10.0 (TID 61). 6606 bytes result sent to driver -26/04/01 08:46:50 INFO TaskSetManager: Starting task 55.0 in stage 10.0 (TID 72) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:50 INFO TaskSetManager: Finished task 44.0 in stage 10.0 (TID 61) in 8174 ms on 10.0.0.133 (executor driver) (48/208) -26/04/01 08:46:50 INFO Executor: Running task 55.0 in stage 10.0 (TID 72) -26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:57 INFO Executor: Finished task 48.0 in stage 10.0 (TID 65). 6606 bytes result sent to driver -26/04/01 08:46:57 INFO TaskSetManager: Starting task 56.0 in stage 10.0 (TID 73) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:57 INFO TaskSetManager: Finished task 48.0 in stage 10.0 (TID 65) in 7957 ms on 10.0.0.133 (executor driver) (49/208) -26/04/01 08:46:57 INFO Executor: Running task 56.0 in stage 10.0 (TID 73) -26/04/01 08:46:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:58 INFO Executor: Finished task 49.0 in stage 10.0 (TID 66). 6606 bytes result sent to driver -26/04/01 08:46:58 INFO TaskSetManager: Starting task 57.0 in stage 10.0 (TID 74) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:58 INFO TaskSetManager: Finished task 49.0 in stage 10.0 (TID 66) in 7910 ms on 10.0.0.133 (executor driver) (50/208) -26/04/01 08:46:58 INFO Executor: Running task 57.0 in stage 10.0 (TID 74) -26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:58 INFO Executor: Finished task 50.0 in stage 10.0 (TID 67). 6606 bytes result sent to driver -26/04/01 08:46:58 INFO TaskSetManager: Starting task 58.0 in stage 10.0 (TID 75) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:58 INFO Executor: Running task 58.0 in stage 10.0 (TID 75) -26/04/01 08:46:58 INFO TaskSetManager: Finished task 50.0 in stage 10.0 (TID 67) in 7929 ms on 10.0.0.133 (executor driver) (51/208) -26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:58 INFO Executor: Finished task 51.0 in stage 10.0 (TID 68). 6606 bytes result sent to driver -26/04/01 08:46:58 INFO TaskSetManager: Starting task 59.0 in stage 10.0 (TID 76) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:58 INFO TaskSetManager: Finished task 51.0 in stage 10.0 (TID 68) in 7932 ms on 10.0.0.133 (executor driver) (52/208) -26/04/01 08:46:58 INFO Executor: Running task 59.0 in stage 10.0 (TID 76) -26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:58 INFO Executor: Finished task 53.0 in stage 10.0 (TID 70). 6606 bytes result sent to driver -26/04/01 08:46:58 INFO Executor: Finished task 52.0 in stage 10.0 (TID 69). 6606 bytes result sent to driver -26/04/01 08:46:58 INFO TaskSetManager: Starting task 60.0 in stage 10.0 (TID 77) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:46:58 INFO Executor: Running task 60.0 in stage 10.0 (TID 77) -26/04/01 08:46:58 INFO TaskSetManager: Starting task 61.0 in stage 10.0 (TID 78) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:46:58 INFO TaskSetManager: Finished task 53.0 in stage 10.0 (TID 70) in 7933 ms on 10.0.0.133 (executor driver) (53/208) -26/04/01 08:46:58 INFO Executor: Running task 61.0 in stage 10.0 (TID 78) -26/04/01 08:46:58 INFO TaskSetManager: Finished task 52.0 in stage 10.0 (TID 69) in 7944 ms on 10.0.0.133 (executor driver) (54/208) -26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:58 INFO Executor: Finished task 54.0 in stage 10.0 (TID 71). 6606 bytes result sent to driver -26/04/01 08:46:58 INFO TaskSetManager: Starting task 62.0 in stage 10.0 (TID 79) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:58 INFO TaskSetManager: Finished task 54.0 in stage 10.0 (TID 71) in 7959 ms on 10.0.0.133 (executor driver) (55/208) -26/04/01 08:46:58 INFO Executor: Running task 62.0 in stage 10.0 (TID 79) -26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:58 INFO Executor: Finished task 55.0 in stage 10.0 (TID 72). 6606 bytes result sent to driver -26/04/01 08:46:58 INFO TaskSetManager: Starting task 63.0 in stage 10.0 (TID 80) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:46:58 INFO TaskSetManager: Finished task 55.0 in stage 10.0 (TID 72) in 7933 ms on 10.0.0.133 (executor driver) (56/208) -26/04/01 08:46:58 INFO Executor: Running task 63.0 in stage 10.0 (TID 80) -26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:46:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:05 INFO Executor: Finished task 56.0 in stage 10.0 (TID 73). 6606 bytes result sent to driver -26/04/01 08:47:05 INFO TaskSetManager: Starting task 64.0 in stage 10.0 (TID 81) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:05 INFO TaskSetManager: Finished task 56.0 in stage 10.0 (TID 73) in 7952 ms on 10.0.0.133 (executor driver) (57/208) -26/04/01 08:47:05 INFO Executor: Running task 64.0 in stage 10.0 (TID 81) -26/04/01 08:47:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:06 INFO Executor: Finished task 57.0 in stage 10.0 (TID 74). 6606 bytes result sent to driver -26/04/01 08:47:06 INFO TaskSetManager: Starting task 65.0 in stage 10.0 (TID 82) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:06 INFO TaskSetManager: Finished task 57.0 in stage 10.0 (TID 74) in 7949 ms on 10.0.0.133 (executor driver) (58/208) -26/04/01 08:47:06 INFO Executor: Running task 65.0 in stage 10.0 (TID 82) -26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:06 INFO Executor: Finished task 59.0 in stage 10.0 (TID 76). 6606 bytes result sent to driver -26/04/01 08:47:06 INFO TaskSetManager: Starting task 66.0 in stage 10.0 (TID 83) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:47:06 INFO TaskSetManager: Finished task 59.0 in stage 10.0 (TID 76) in 7902 ms on 10.0.0.133 (executor driver) (59/208) -26/04/01 08:47:06 INFO Executor: Running task 66.0 in stage 10.0 (TID 83) -26/04/01 08:47:06 INFO Executor: Finished task 58.0 in stage 10.0 (TID 75). 6606 bytes result sent to driver -26/04/01 08:47:06 INFO TaskSetManager: Starting task 67.0 in stage 10.0 (TID 84) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:47:06 INFO TaskSetManager: Finished task 58.0 in stage 10.0 (TID 75) in 7918 ms on 10.0.0.133 (executor driver) (60/208) -26/04/01 08:47:06 INFO Executor: Running task 67.0 in stage 10.0 (TID 84) -26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:06 INFO Executor: Finished task 60.0 in stage 10.0 (TID 77). 6606 bytes result sent to driver -26/04/01 08:47:06 INFO TaskSetManager: Starting task 68.0 in stage 10.0 (TID 85) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:06 INFO TaskSetManager: Finished task 60.0 in stage 10.0 (TID 77) in 7937 ms on 10.0.0.133 (executor driver) (61/208) -26/04/01 08:47:06 INFO Executor: Running task 68.0 in stage 10.0 (TID 85) -26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:06 INFO Executor: Finished task 61.0 in stage 10.0 (TID 78). 6606 bytes result sent to driver -26/04/01 08:47:06 INFO TaskSetManager: Starting task 69.0 in stage 10.0 (TID 86) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:06 INFO TaskSetManager: Finished task 61.0 in stage 10.0 (TID 78) in 7949 ms on 10.0.0.133 (executor driver) (62/208) -26/04/01 08:47:06 INFO Executor: Running task 69.0 in stage 10.0 (TID 86) -26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:06 INFO Executor: Finished task 62.0 in stage 10.0 (TID 79). 6606 bytes result sent to driver -26/04/01 08:47:06 INFO TaskSetManager: Starting task 70.0 in stage 10.0 (TID 87) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:06 INFO TaskSetManager: Finished task 62.0 in stage 10.0 (TID 79) in 7950 ms on 10.0.0.133 (executor driver) (63/208) -26/04/01 08:47:06 INFO Executor: Running task 70.0 in stage 10.0 (TID 87) -26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:06 INFO Executor: Finished task 63.0 in stage 10.0 (TID 80). 6606 bytes result sent to driver -26/04/01 08:47:06 INFO TaskSetManager: Starting task 71.0 in stage 10.0 (TID 88) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:06 INFO TaskSetManager: Finished task 63.0 in stage 10.0 (TID 80) in 7929 ms on 10.0.0.133 (executor driver) (64/208) -26/04/01 08:47:06 INFO Executor: Running task 71.0 in stage 10.0 (TID 88) -26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:13 INFO Executor: Finished task 64.0 in stage 10.0 (TID 81). 6606 bytes result sent to driver -26/04/01 08:47:13 INFO TaskSetManager: Starting task 72.0 in stage 10.0 (TID 89) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:47:13 INFO TaskSetManager: Finished task 64.0 in stage 10.0 (TID 81) in 7953 ms on 10.0.0.133 (executor driver) (65/208) -26/04/01 08:47:13 INFO Executor: Running task 72.0 in stage 10.0 (TID 89) -26/04/01 08:47:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:14 INFO Executor: Finished task 65.0 in stage 10.0 (TID 82). 6606 bytes result sent to driver -26/04/01 08:47:14 INFO TaskSetManager: Starting task 73.0 in stage 10.0 (TID 90) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:47:14 INFO TaskSetManager: Finished task 65.0 in stage 10.0 (TID 82) in 7940 ms on 10.0.0.133 (executor driver) (66/208) -26/04/01 08:47:14 INFO Executor: Running task 73.0 in stage 10.0 (TID 90) -26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:14 INFO Executor: Finished task 67.0 in stage 10.0 (TID 84). 6606 bytes result sent to driver -26/04/01 08:47:14 INFO TaskSetManager: Starting task 74.0 in stage 10.0 (TID 91) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:14 INFO Executor: Running task 74.0 in stage 10.0 (TID 91) -26/04/01 08:47:14 INFO TaskSetManager: Finished task 67.0 in stage 10.0 (TID 84) in 7960 ms on 10.0.0.133 (executor driver) (67/208) -26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:14 INFO Executor: Finished task 66.0 in stage 10.0 (TID 83). 6606 bytes result sent to driver -26/04/01 08:47:14 INFO TaskSetManager: Starting task 75.0 in stage 10.0 (TID 92) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:14 INFO TaskSetManager: Finished task 66.0 in stage 10.0 (TID 83) in 7968 ms on 10.0.0.133 (executor driver) (68/208) -26/04/01 08:47:14 INFO Executor: Running task 75.0 in stage 10.0 (TID 92) -26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:14 INFO Executor: Finished task 68.0 in stage 10.0 (TID 85). 6606 bytes result sent to driver -26/04/01 08:47:14 INFO TaskSetManager: Starting task 76.0 in stage 10.0 (TID 93) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:14 INFO TaskSetManager: Finished task 68.0 in stage 10.0 (TID 85) in 7934 ms on 10.0.0.133 (executor driver) (69/208) -26/04/01 08:47:14 INFO Executor: Running task 76.0 in stage 10.0 (TID 93) -26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:14 INFO Executor: Finished task 69.0 in stage 10.0 (TID 86). 6606 bytes result sent to driver -26/04/01 08:47:14 INFO TaskSetManager: Starting task 77.0 in stage 10.0 (TID 94) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:14 INFO TaskSetManager: Finished task 69.0 in stage 10.0 (TID 86) in 7946 ms on 10.0.0.133 (executor driver) (70/208) -26/04/01 08:47:14 INFO Executor: Running task 77.0 in stage 10.0 (TID 94) -26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:14 INFO Executor: Finished task 70.0 in stage 10.0 (TID 87). 6606 bytes result sent to driver -26/04/01 08:47:14 INFO TaskSetManager: Starting task 78.0 in stage 10.0 (TID 95) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:47:14 INFO TaskSetManager: Finished task 70.0 in stage 10.0 (TID 87) in 7929 ms on 10.0.0.133 (executor driver) (71/208) -26/04/01 08:47:14 INFO Executor: Running task 78.0 in stage 10.0 (TID 95) -26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:14 INFO Executor: Finished task 71.0 in stage 10.0 (TID 88). 6606 bytes result sent to driver -26/04/01 08:47:14 INFO TaskSetManager: Starting task 79.0 in stage 10.0 (TID 96) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:14 INFO TaskSetManager: Finished task 71.0 in stage 10.0 (TID 88) in 7949 ms on 10.0.0.133 (executor driver) (72/208) -26/04/01 08:47:14 INFO Executor: Running task 79.0 in stage 10.0 (TID 96) -26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:21 INFO Executor: Finished task 72.0 in stage 10.0 (TID 89). 6606 bytes result sent to driver -26/04/01 08:47:21 INFO TaskSetManager: Starting task 80.0 in stage 10.0 (TID 97) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:21 INFO Executor: Running task 80.0 in stage 10.0 (TID 97) -26/04/01 08:47:21 INFO TaskSetManager: Finished task 72.0 in stage 10.0 (TID 89) in 7943 ms on 10.0.0.133 (executor driver) (73/208) -26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:21 INFO Executor: Finished task 74.0 in stage 10.0 (TID 91). 6606 bytes result sent to driver -26/04/01 08:47:21 INFO TaskSetManager: Starting task 81.0 in stage 10.0 (TID 98) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:21 INFO TaskSetManager: Finished task 74.0 in stage 10.0 (TID 91) in 7873 ms on 10.0.0.133 (executor driver) (74/208) -26/04/01 08:47:21 INFO Executor: Running task 81.0 in stage 10.0 (TID 98) -26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:21 INFO Executor: Finished task 73.0 in stage 10.0 (TID 90). 6606 bytes result sent to driver -26/04/01 08:47:21 INFO TaskSetManager: Starting task 82.0 in stage 10.0 (TID 99) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:21 INFO TaskSetManager: Finished task 73.0 in stage 10.0 (TID 90) in 7952 ms on 10.0.0.133 (executor driver) (75/208) -26/04/01 08:47:21 INFO Executor: Running task 82.0 in stage 10.0 (TID 99) -26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:21 INFO Executor: Finished task 75.0 in stage 10.0 (TID 92). 6606 bytes result sent to driver -26/04/01 08:47:21 INFO TaskSetManager: Starting task 83.0 in stage 10.0 (TID 100) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:21 INFO TaskSetManager: Finished task 75.0 in stage 10.0 (TID 92) in 7945 ms on 10.0.0.133 (executor driver) (76/208) -26/04/01 08:47:21 INFO Executor: Running task 83.0 in stage 10.0 (TID 100) -26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:22 INFO Executor: Finished task 76.0 in stage 10.0 (TID 93). 6649 bytes result sent to driver -26/04/01 08:47:22 INFO TaskSetManager: Starting task 84.0 in stage 10.0 (TID 101) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:47:22 INFO TaskSetManager: Finished task 76.0 in stage 10.0 (TID 93) in 7951 ms on 10.0.0.133 (executor driver) (77/208) -26/04/01 08:47:22 INFO Executor: Running task 84.0 in stage 10.0 (TID 101) -26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:22 INFO Executor: Finished task 77.0 in stage 10.0 (TID 94). 6606 bytes result sent to driver -26/04/01 08:47:22 INFO TaskSetManager: Starting task 85.0 in stage 10.0 (TID 102) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:22 INFO Executor: Running task 85.0 in stage 10.0 (TID 102) -26/04/01 08:47:22 INFO TaskSetManager: Finished task 77.0 in stage 10.0 (TID 94) in 7948 ms on 10.0.0.133 (executor driver) (78/208) -26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:22 INFO Executor: Finished task 78.0 in stage 10.0 (TID 95). 6606 bytes result sent to driver -26/04/01 08:47:22 INFO TaskSetManager: Starting task 86.0 in stage 10.0 (TID 103) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:22 INFO TaskSetManager: Finished task 78.0 in stage 10.0 (TID 95) in 7941 ms on 10.0.0.133 (executor driver) (79/208) -26/04/01 08:47:22 INFO Executor: Running task 86.0 in stage 10.0 (TID 103) -26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:22 INFO Executor: Finished task 79.0 in stage 10.0 (TID 96). 6606 bytes result sent to driver -26/04/01 08:47:22 INFO TaskSetManager: Starting task 87.0 in stage 10.0 (TID 104) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:22 INFO TaskSetManager: Finished task 79.0 in stage 10.0 (TID 96) in 7910 ms on 10.0.0.133 (executor driver) (80/208) -26/04/01 08:47:22 INFO Executor: Running task 87.0 in stage 10.0 (TID 104) -26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:29 INFO Executor: Finished task 80.0 in stage 10.0 (TID 97). 6606 bytes result sent to driver -26/04/01 08:47:29 INFO TaskSetManager: Starting task 88.0 in stage 10.0 (TID 105) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:29 INFO TaskSetManager: Finished task 80.0 in stage 10.0 (TID 97) in 7929 ms on 10.0.0.133 (executor driver) (81/208) -26/04/01 08:47:29 INFO Executor: Running task 88.0 in stage 10.0 (TID 105) -26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:29 INFO Executor: Finished task 81.0 in stage 10.0 (TID 98). 6606 bytes result sent to driver -26/04/01 08:47:29 INFO TaskSetManager: Starting task 89.0 in stage 10.0 (TID 106) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:29 INFO TaskSetManager: Finished task 81.0 in stage 10.0 (TID 98) in 7941 ms on 10.0.0.133 (executor driver) (82/208) -26/04/01 08:47:29 INFO Executor: Running task 89.0 in stage 10.0 (TID 106) -26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:29 INFO Executor: Finished task 82.0 in stage 10.0 (TID 99). 6606 bytes result sent to driver -26/04/01 08:47:29 INFO TaskSetManager: Starting task 90.0 in stage 10.0 (TID 107) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:47:29 INFO TaskSetManager: Finished task 82.0 in stage 10.0 (TID 99) in 7938 ms on 10.0.0.133 (executor driver) (83/208) -26/04/01 08:47:29 INFO Executor: Running task 90.0 in stage 10.0 (TID 107) -26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:29 INFO Executor: Finished task 83.0 in stage 10.0 (TID 100). 6606 bytes result sent to driver -26/04/01 08:47:29 INFO TaskSetManager: Starting task 91.0 in stage 10.0 (TID 108) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:29 INFO TaskSetManager: Finished task 83.0 in stage 10.0 (TID 100) in 7924 ms on 10.0.0.133 (executor driver) (84/208) -26/04/01 08:47:29 INFO Executor: Running task 91.0 in stage 10.0 (TID 108) -26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:29 INFO Executor: Finished task 84.0 in stage 10.0 (TID 101). 6606 bytes result sent to driver -26/04/01 08:47:29 INFO TaskSetManager: Starting task 92.0 in stage 10.0 (TID 109) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:29 INFO TaskSetManager: Finished task 84.0 in stage 10.0 (TID 101) in 7917 ms on 10.0.0.133 (executor driver) (85/208) -26/04/01 08:47:29 INFO Executor: Running task 92.0 in stage 10.0 (TID 109) -26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:29 INFO Executor: Finished task 85.0 in stage 10.0 (TID 102). 6606 bytes result sent to driver -26/04/01 08:47:29 INFO TaskSetManager: Starting task 93.0 in stage 10.0 (TID 110) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:29 INFO Executor: Running task 93.0 in stage 10.0 (TID 110) -26/04/01 08:47:29 INFO TaskSetManager: Finished task 85.0 in stage 10.0 (TID 102) in 7940 ms on 10.0.0.133 (executor driver) (86/208) -26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:30 INFO Executor: Finished task 86.0 in stage 10.0 (TID 103). 6606 bytes result sent to driver -26/04/01 08:47:30 INFO TaskSetManager: Starting task 94.0 in stage 10.0 (TID 111) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:30 INFO TaskSetManager: Finished task 86.0 in stage 10.0 (TID 103) in 7933 ms on 10.0.0.133 (executor driver) (87/208) -26/04/01 08:47:30 INFO Executor: Running task 94.0 in stage 10.0 (TID 111) -26/04/01 08:47:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:30 INFO Executor: Finished task 87.0 in stage 10.0 (TID 104). 6606 bytes result sent to driver -26/04/01 08:47:30 INFO TaskSetManager: Starting task 95.0 in stage 10.0 (TID 112) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:30 INFO TaskSetManager: Finished task 87.0 in stage 10.0 (TID 104) in 7936 ms on 10.0.0.133 (executor driver) (88/208) -26/04/01 08:47:30 INFO Executor: Running task 95.0 in stage 10.0 (TID 112) -26/04/01 08:47:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:37 INFO Executor: Finished task 91.0 in stage 10.0 (TID 108). 6606 bytes result sent to driver -26/04/01 08:47:37 INFO TaskSetManager: Starting task 96.0 in stage 10.0 (TID 113) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:47:37 INFO TaskSetManager: Finished task 91.0 in stage 10.0 (TID 108) in 7659 ms on 10.0.0.133 (executor driver) (89/208) -26/04/01 08:47:37 INFO Executor: Running task 96.0 in stage 10.0 (TID 113) -26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:37 INFO Executor: Finished task 88.0 in stage 10.0 (TID 105). 6606 bytes result sent to driver -26/04/01 08:47:37 INFO TaskSetManager: Starting task 97.0 in stage 10.0 (TID 114) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:47:37 INFO TaskSetManager: Finished task 88.0 in stage 10.0 (TID 105) in 7941 ms on 10.0.0.133 (executor driver) (90/208) -26/04/01 08:47:37 INFO Executor: Running task 97.0 in stage 10.0 (TID 114) -26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:37 INFO Executor: Finished task 89.0 in stage 10.0 (TID 106). 6606 bytes result sent to driver -26/04/01 08:47:37 INFO TaskSetManager: Starting task 98.0 in stage 10.0 (TID 115) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:37 INFO TaskSetManager: Finished task 89.0 in stage 10.0 (TID 106) in 7958 ms on 10.0.0.133 (executor driver) (91/208) -26/04/01 08:47:37 INFO Executor: Running task 98.0 in stage 10.0 (TID 115) -26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:37 INFO Executor: Finished task 92.0 in stage 10.0 (TID 109). 6606 bytes result sent to driver -26/04/01 08:47:37 INFO TaskSetManager: Starting task 99.0 in stage 10.0 (TID 116) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:37 INFO TaskSetManager: Finished task 92.0 in stage 10.0 (TID 109) in 7909 ms on 10.0.0.133 (executor driver) (92/208) -26/04/01 08:47:37 INFO Executor: Running task 99.0 in stage 10.0 (TID 116) -26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:37 INFO Executor: Finished task 93.0 in stage 10.0 (TID 110). 6606 bytes result sent to driver -26/04/01 08:47:37 INFO TaskSetManager: Starting task 100.0 in stage 10.0 (TID 117) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:37 INFO TaskSetManager: Finished task 93.0 in stage 10.0 (TID 110) in 7943 ms on 10.0.0.133 (executor driver) (93/208) -26/04/01 08:47:37 INFO Executor: Running task 100.0 in stage 10.0 (TID 117) -26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:37 INFO Executor: Finished task 94.0 in stage 10.0 (TID 111). 6606 bytes result sent to driver -26/04/01 08:47:37 INFO TaskSetManager: Starting task 101.0 in stage 10.0 (TID 118) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:37 INFO TaskSetManager: Finished task 94.0 in stage 10.0 (TID 111) in 7911 ms on 10.0.0.133 (executor driver) (94/208) -26/04/01 08:47:37 INFO Executor: Running task 101.0 in stage 10.0 (TID 118) -26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:38 INFO Executor: Finished task 90.0 in stage 10.0 (TID 107). 6606 bytes result sent to driver -26/04/01 08:47:38 INFO TaskSetManager: Starting task 102.0 in stage 10.0 (TID 119) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:47:38 INFO TaskSetManager: Finished task 90.0 in stage 10.0 (TID 107) in 8209 ms on 10.0.0.133 (executor driver) (95/208) -26/04/01 08:47:38 INFO Executor: Running task 102.0 in stage 10.0 (TID 119) -26/04/01 08:47:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:38 INFO Executor: Finished task 95.0 in stage 10.0 (TID 112). 6606 bytes result sent to driver -26/04/01 08:47:38 INFO TaskSetManager: Starting task 103.0 in stage 10.0 (TID 120) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:38 INFO TaskSetManager: Finished task 95.0 in stage 10.0 (TID 112) in 7936 ms on 10.0.0.133 (executor driver) (96/208) -26/04/01 08:47:38 INFO Executor: Running task 103.0 in stage 10.0 (TID 120) -26/04/01 08:47:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:45 INFO Executor: Finished task 96.0 in stage 10.0 (TID 113). 6606 bytes result sent to driver -26/04/01 08:47:45 INFO TaskSetManager: Starting task 104.0 in stage 10.0 (TID 121) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:45 INFO TaskSetManager: Finished task 96.0 in stage 10.0 (TID 113) in 7987 ms on 10.0.0.133 (executor driver) (97/208) -26/04/01 08:47:45 INFO Executor: Running task 104.0 in stage 10.0 (TID 121) -26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:45 INFO Executor: Finished task 97.0 in stage 10.0 (TID 114). 6606 bytes result sent to driver -26/04/01 08:47:45 INFO TaskSetManager: Starting task 105.0 in stage 10.0 (TID 122) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:45 INFO TaskSetManager: Finished task 97.0 in stage 10.0 (TID 114) in 7959 ms on 10.0.0.133 (executor driver) (98/208) -26/04/01 08:47:45 INFO Executor: Running task 105.0 in stage 10.0 (TID 122) -26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:45 INFO Executor: Finished task 98.0 in stage 10.0 (TID 115). 6606 bytes result sent to driver -26/04/01 08:47:45 INFO TaskSetManager: Starting task 106.0 in stage 10.0 (TID 123) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:45 INFO Executor: Running task 106.0 in stage 10.0 (TID 123) -26/04/01 08:47:45 INFO TaskSetManager: Finished task 98.0 in stage 10.0 (TID 115) in 7930 ms on 10.0.0.133 (executor driver) (99/208) -26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:45 INFO Executor: Finished task 99.0 in stage 10.0 (TID 116). 6606 bytes result sent to driver -26/04/01 08:47:45 INFO TaskSetManager: Starting task 107.0 in stage 10.0 (TID 124) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:45 INFO TaskSetManager: Finished task 99.0 in stage 10.0 (TID 116) in 7946 ms on 10.0.0.133 (executor driver) (100/208) -26/04/01 08:47:45 INFO Executor: Running task 107.0 in stage 10.0 (TID 124) -26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:45 INFO Executor: Finished task 100.0 in stage 10.0 (TID 117). 6606 bytes result sent to driver -26/04/01 08:47:45 INFO TaskSetManager: Starting task 108.0 in stage 10.0 (TID 125) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:47:45 INFO TaskSetManager: Finished task 100.0 in stage 10.0 (TID 117) in 7930 ms on 10.0.0.133 (executor driver) (101/208) -26/04/01 08:47:45 INFO Executor: Running task 108.0 in stage 10.0 (TID 125) -26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:45 INFO Executor: Finished task 101.0 in stage 10.0 (TID 118). 6606 bytes result sent to driver -26/04/01 08:47:45 INFO TaskSetManager: Starting task 109.0 in stage 10.0 (TID 126) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:47:45 INFO TaskSetManager: Finished task 101.0 in stage 10.0 (TID 118) in 7940 ms on 10.0.0.133 (executor driver) (102/208) -26/04/01 08:47:45 INFO Executor: Running task 109.0 in stage 10.0 (TID 126) -26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:46 INFO Executor: Finished task 102.0 in stage 10.0 (TID 119). 6606 bytes result sent to driver -26/04/01 08:47:46 INFO TaskSetManager: Starting task 110.0 in stage 10.0 (TID 127) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:46 INFO TaskSetManager: Finished task 102.0 in stage 10.0 (TID 119) in 7942 ms on 10.0.0.133 (executor driver) (103/208) -26/04/01 08:47:46 INFO Executor: Running task 110.0 in stage 10.0 (TID 127) -26/04/01 08:47:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:46 INFO Executor: Finished task 103.0 in stage 10.0 (TID 120). 6606 bytes result sent to driver -26/04/01 08:47:46 INFO TaskSetManager: Starting task 111.0 in stage 10.0 (TID 128) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:46 INFO TaskSetManager: Finished task 103.0 in stage 10.0 (TID 120) in 7947 ms on 10.0.0.133 (executor driver) (104/208) -26/04/01 08:47:46 INFO Executor: Running task 111.0 in stage 10.0 (TID 128) -26/04/01 08:47:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:53 INFO Executor: Finished task 104.0 in stage 10.0 (TID 121). 6606 bytes result sent to driver -26/04/01 08:47:53 INFO TaskSetManager: Starting task 112.0 in stage 10.0 (TID 129) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:53 INFO TaskSetManager: Finished task 104.0 in stage 10.0 (TID 121) in 7931 ms on 10.0.0.133 (executor driver) (105/208) -26/04/01 08:47:53 INFO Executor: Running task 112.0 in stage 10.0 (TID 129) -26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:53 INFO Executor: Finished task 105.0 in stage 10.0 (TID 122). 6606 bytes result sent to driver -26/04/01 08:47:53 INFO TaskSetManager: Starting task 113.0 in stage 10.0 (TID 130) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:53 INFO TaskSetManager: Finished task 105.0 in stage 10.0 (TID 122) in 7953 ms on 10.0.0.133 (executor driver) (106/208) -26/04/01 08:47:53 INFO Executor: Running task 113.0 in stage 10.0 (TID 130) -26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:53 INFO Executor: Finished task 106.0 in stage 10.0 (TID 123). 6606 bytes result sent to driver -26/04/01 08:47:53 INFO TaskSetManager: Starting task 114.0 in stage 10.0 (TID 131) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:47:53 INFO TaskSetManager: Finished task 106.0 in stage 10.0 (TID 123) in 7951 ms on 10.0.0.133 (executor driver) (107/208) -26/04/01 08:47:53 INFO Executor: Running task 114.0 in stage 10.0 (TID 131) -26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:53 INFO Executor: Finished task 107.0 in stage 10.0 (TID 124). 6606 bytes result sent to driver -26/04/01 08:47:53 INFO TaskSetManager: Starting task 115.0 in stage 10.0 (TID 132) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:47:53 INFO Executor: Running task 115.0 in stage 10.0 (TID 132) -26/04/01 08:47:53 INFO TaskSetManager: Finished task 107.0 in stage 10.0 (TID 124) in 7951 ms on 10.0.0.133 (executor driver) (108/208) -26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:53 INFO Executor: Finished task 108.0 in stage 10.0 (TID 125). 6606 bytes result sent to driver -26/04/01 08:47:53 INFO TaskSetManager: Starting task 116.0 in stage 10.0 (TID 133) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:53 INFO TaskSetManager: Finished task 108.0 in stage 10.0 (TID 125) in 7926 ms on 10.0.0.133 (executor driver) (109/208) -26/04/01 08:47:53 INFO Executor: Running task 116.0 in stage 10.0 (TID 133) -26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:53 INFO Executor: Finished task 109.0 in stage 10.0 (TID 126). 6606 bytes result sent to driver -26/04/01 08:47:53 INFO TaskSetManager: Starting task 117.0 in stage 10.0 (TID 134) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:53 INFO Executor: Running task 117.0 in stage 10.0 (TID 134) -26/04/01 08:47:53 INFO TaskSetManager: Finished task 109.0 in stage 10.0 (TID 126) in 7933 ms on 10.0.0.133 (executor driver) (110/208) -26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:53 INFO Executor: Finished task 110.0 in stage 10.0 (TID 127). 6606 bytes result sent to driver -26/04/01 08:47:53 INFO TaskSetManager: Starting task 118.0 in stage 10.0 (TID 135) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:53 INFO Executor: Running task 118.0 in stage 10.0 (TID 135) -26/04/01 08:47:53 INFO TaskSetManager: Finished task 110.0 in stage 10.0 (TID 127) in 7919 ms on 10.0.0.133 (executor driver) (111/208) -26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:54 INFO Executor: Finished task 111.0 in stage 10.0 (TID 128). 6606 bytes result sent to driver -26/04/01 08:47:54 INFO TaskSetManager: Starting task 119.0 in stage 10.0 (TID 136) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:47:54 INFO Executor: Running task 119.0 in stage 10.0 (TID 136) -26/04/01 08:47:54 INFO TaskSetManager: Finished task 111.0 in stage 10.0 (TID 128) in 7942 ms on 10.0.0.133 (executor driver) (112/208) -26/04/01 08:47:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:47:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO Executor: Finished task 112.0 in stage 10.0 (TID 129). 6563 bytes result sent to driver -26/04/01 08:48:01 INFO TaskSetManager: Starting task 120.0 in stage 10.0 (TID 137) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:48:01 INFO Executor: Running task 120.0 in stage 10.0 (TID 137) -26/04/01 08:48:01 INFO TaskSetManager: Finished task 112.0 in stage 10.0 (TID 129) in 7974 ms on 10.0.0.133 (executor driver) (113/208) -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO Executor: Finished task 113.0 in stage 10.0 (TID 130). 6563 bytes result sent to driver -26/04/01 08:48:01 INFO TaskSetManager: Starting task 121.0 in stage 10.0 (TID 138) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:01 INFO TaskSetManager: Finished task 113.0 in stage 10.0 (TID 130) in 7957 ms on 10.0.0.133 (executor driver) (114/208) -26/04/01 08:48:01 INFO Executor: Running task 121.0 in stage 10.0 (TID 138) -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO Executor: Finished task 114.0 in stage 10.0 (TID 131). 6563 bytes result sent to driver -26/04/01 08:48:01 INFO TaskSetManager: Starting task 122.0 in stage 10.0 (TID 139) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:01 INFO TaskSetManager: Finished task 114.0 in stage 10.0 (TID 131) in 7939 ms on 10.0.0.133 (executor driver) (115/208) -26/04/01 08:48:01 INFO Executor: Running task 122.0 in stage 10.0 (TID 139) -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO Executor: Finished task 115.0 in stage 10.0 (TID 132). 6563 bytes result sent to driver -26/04/01 08:48:01 INFO TaskSetManager: Starting task 123.0 in stage 10.0 (TID 140) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:01 INFO TaskSetManager: Finished task 115.0 in stage 10.0 (TID 132) in 7941 ms on 10.0.0.133 (executor driver) (116/208) -26/04/01 08:48:01 INFO Executor: Running task 123.0 in stage 10.0 (TID 140) -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO Executor: Finished task 116.0 in stage 10.0 (TID 133). 6563 bytes result sent to driver -26/04/01 08:48:01 INFO TaskSetManager: Starting task 124.0 in stage 10.0 (TID 141) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:01 INFO Executor: Running task 124.0 in stage 10.0 (TID 141) -26/04/01 08:48:01 INFO TaskSetManager: Finished task 116.0 in stage 10.0 (TID 133) in 7949 ms on 10.0.0.133 (executor driver) (117/208) -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO Executor: Finished task 117.0 in stage 10.0 (TID 134). 6563 bytes result sent to driver -26/04/01 08:48:01 INFO TaskSetManager: Starting task 125.0 in stage 10.0 (TID 142) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:01 INFO Executor: Running task 125.0 in stage 10.0 (TID 142) -26/04/01 08:48:01 INFO TaskSetManager: Finished task 117.0 in stage 10.0 (TID 134) in 7938 ms on 10.0.0.133 (executor driver) (118/208) -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO Executor: Finished task 118.0 in stage 10.0 (TID 135). 6563 bytes result sent to driver -26/04/01 08:48:01 INFO TaskSetManager: Starting task 126.0 in stage 10.0 (TID 143) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:48:01 INFO TaskSetManager: Finished task 118.0 in stage 10.0 (TID 135) in 7955 ms on 10.0.0.133 (executor driver) (119/208) -26/04/01 08:48:01 INFO Executor: Running task 126.0 in stage 10.0 (TID 143) -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO Executor: Finished task 119.0 in stage 10.0 (TID 136). 6563 bytes result sent to driver -26/04/01 08:48:01 INFO TaskSetManager: Starting task 127.0 in stage 10.0 (TID 144) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:01 INFO TaskSetManager: Finished task 119.0 in stage 10.0 (TID 136) in 7948 ms on 10.0.0.133 (executor driver) (120/208) -26/04/01 08:48:01 INFO Executor: Running task 127.0 in stage 10.0 (TID 144) -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO Executor: Finished task 120.0 in stage 10.0 (TID 137). 6606 bytes result sent to driver -26/04/01 08:48:09 INFO TaskSetManager: Starting task 128.0 in stage 10.0 (TID 145) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:09 INFO Executor: Running task 128.0 in stage 10.0 (TID 145) -26/04/01 08:48:09 INFO TaskSetManager: Finished task 120.0 in stage 10.0 (TID 137) in 8001 ms on 10.0.0.133 (executor driver) (121/208) -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO Executor: Finished task 121.0 in stage 10.0 (TID 138). 6606 bytes result sent to driver -26/04/01 08:48:09 INFO TaskSetManager: Starting task 129.0 in stage 10.0 (TID 146) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:09 INFO TaskSetManager: Finished task 121.0 in stage 10.0 (TID 138) in 7974 ms on 10.0.0.133 (executor driver) (122/208) -26/04/01 08:48:09 INFO Executor: Running task 129.0 in stage 10.0 (TID 146) -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO Executor: Finished task 122.0 in stage 10.0 (TID 139). 6606 bytes result sent to driver -26/04/01 08:48:09 INFO TaskSetManager: Starting task 130.0 in stage 10.0 (TID 147) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:09 INFO TaskSetManager: Finished task 122.0 in stage 10.0 (TID 139) in 7965 ms on 10.0.0.133 (executor driver) (123/208) -26/04/01 08:48:09 INFO Executor: Running task 130.0 in stage 10.0 (TID 147) -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO Executor: Finished task 123.0 in stage 10.0 (TID 140). 6606 bytes result sent to driver -26/04/01 08:48:09 INFO TaskSetManager: Starting task 131.0 in stage 10.0 (TID 148) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:09 INFO TaskSetManager: Finished task 123.0 in stage 10.0 (TID 140) in 7966 ms on 10.0.0.133 (executor driver) (124/208) -26/04/01 08:48:09 INFO Executor: Running task 131.0 in stage 10.0 (TID 148) -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO Executor: Finished task 124.0 in stage 10.0 (TID 141). 6606 bytes result sent to driver -26/04/01 08:48:09 INFO TaskSetManager: Starting task 132.0 in stage 10.0 (TID 149) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:48:09 INFO TaskSetManager: Finished task 124.0 in stage 10.0 (TID 141) in 7986 ms on 10.0.0.133 (executor driver) (125/208) -26/04/01 08:48:09 INFO Executor: Running task 132.0 in stage 10.0 (TID 149) -26/04/01 08:48:09 INFO Executor: Finished task 125.0 in stage 10.0 (TID 142). 6606 bytes result sent to driver -26/04/01 08:48:09 INFO TaskSetManager: Starting task 133.0 in stage 10.0 (TID 150) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:09 INFO TaskSetManager: Finished task 125.0 in stage 10.0 (TID 142) in 7975 ms on 10.0.0.133 (executor driver) (126/208) -26/04/01 08:48:09 INFO Executor: Running task 133.0 in stage 10.0 (TID 150) -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO Executor: Finished task 126.0 in stage 10.0 (TID 143). 6606 bytes result sent to driver -26/04/01 08:48:09 INFO TaskSetManager: Starting task 134.0 in stage 10.0 (TID 151) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:09 INFO Executor: Running task 134.0 in stage 10.0 (TID 151) -26/04/01 08:48:09 INFO TaskSetManager: Finished task 126.0 in stage 10.0 (TID 143) in 7953 ms on 10.0.0.133 (executor driver) (127/208) -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO Executor: Finished task 127.0 in stage 10.0 (TID 144). 6606 bytes result sent to driver -26/04/01 08:48:09 INFO TaskSetManager: Starting task 135.0 in stage 10.0 (TID 152) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:09 INFO TaskSetManager: Finished task 127.0 in stage 10.0 (TID 144) in 7971 ms on 10.0.0.133 (executor driver) (128/208) -26/04/01 08:48:09 INFO Executor: Running task 135.0 in stage 10.0 (TID 152) -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO Executor: Finished task 129.0 in stage 10.0 (TID 146). 6606 bytes result sent to driver -26/04/01 08:48:17 INFO TaskSetManager: Starting task 136.0 in stage 10.0 (TID 153) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:17 INFO TaskSetManager: Finished task 129.0 in stage 10.0 (TID 146) in 7965 ms on 10.0.0.133 (executor driver) (129/208) -26/04/01 08:48:17 INFO Executor: Running task 136.0 in stage 10.0 (TID 153) -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO Executor: Finished task 130.0 in stage 10.0 (TID 147). 6606 bytes result sent to driver -26/04/01 08:48:17 INFO TaskSetManager: Starting task 137.0 in stage 10.0 (TID 154) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:17 INFO TaskSetManager: Finished task 130.0 in stage 10.0 (TID 147) in 7955 ms on 10.0.0.133 (executor driver) (130/208) -26/04/01 08:48:17 INFO Executor: Running task 137.0 in stage 10.0 (TID 154) -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO Executor: Finished task 131.0 in stage 10.0 (TID 148). 6606 bytes result sent to driver -26/04/01 08:48:17 INFO TaskSetManager: Starting task 138.0 in stage 10.0 (TID 155) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:48:17 INFO TaskSetManager: Finished task 131.0 in stage 10.0 (TID 148) in 7948 ms on 10.0.0.133 (executor driver) (131/208) -26/04/01 08:48:17 INFO Executor: Running task 138.0 in stage 10.0 (TID 155) -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO Executor: Finished task 128.0 in stage 10.0 (TID 145). 6606 bytes result sent to driver -26/04/01 08:48:17 INFO TaskSetManager: Starting task 139.0 in stage 10.0 (TID 156) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:17 INFO TaskSetManager: Finished task 128.0 in stage 10.0 (TID 145) in 8138 ms on 10.0.0.133 (executor driver) (132/208) -26/04/01 08:48:17 INFO Executor: Running task 139.0 in stage 10.0 (TID 156) -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO Executor: Finished task 133.0 in stage 10.0 (TID 150). 6606 bytes result sent to driver -26/04/01 08:48:17 INFO TaskSetManager: Starting task 140.0 in stage 10.0 (TID 157) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:17 INFO TaskSetManager: Finished task 133.0 in stage 10.0 (TID 150) in 7968 ms on 10.0.0.133 (executor driver) (133/208) -26/04/01 08:48:17 INFO Executor: Running task 140.0 in stage 10.0 (TID 157) -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO Executor: Finished task 132.0 in stage 10.0 (TID 149). 6606 bytes result sent to driver -26/04/01 08:48:17 INFO TaskSetManager: Starting task 141.0 in stage 10.0 (TID 158) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:17 INFO Executor: Running task 141.0 in stage 10.0 (TID 158) -26/04/01 08:48:17 INFO TaskSetManager: Finished task 132.0 in stage 10.0 (TID 149) in 7979 ms on 10.0.0.133 (executor driver) (134/208) -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO Executor: Finished task 134.0 in stage 10.0 (TID 151). 6606 bytes result sent to driver -26/04/01 08:48:17 INFO TaskSetManager: Starting task 142.0 in stage 10.0 (TID 159) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:17 INFO TaskSetManager: Finished task 134.0 in stage 10.0 (TID 151) in 7969 ms on 10.0.0.133 (executor driver) (135/208) -26/04/01 08:48:17 INFO Executor: Running task 142.0 in stage 10.0 (TID 159) -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO Executor: Finished task 135.0 in stage 10.0 (TID 152). 6606 bytes result sent to driver -26/04/01 08:48:17 INFO TaskSetManager: Starting task 143.0 in stage 10.0 (TID 160) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:17 INFO TaskSetManager: Finished task 135.0 in stage 10.0 (TID 152) in 7970 ms on 10.0.0.133 (executor driver) (136/208) -26/04/01 08:48:17 INFO Executor: Running task 143.0 in stage 10.0 (TID 160) -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO Executor: Finished task 136.0 in stage 10.0 (TID 153). 6606 bytes result sent to driver -26/04/01 08:48:25 INFO TaskSetManager: Starting task 144.0 in stage 10.0 (TID 161) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:48:25 INFO TaskSetManager: Finished task 136.0 in stage 10.0 (TID 153) in 7954 ms on 10.0.0.133 (executor driver) (137/208) -26/04/01 08:48:25 INFO Executor: Running task 144.0 in stage 10.0 (TID 161) -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO Executor: Finished task 137.0 in stage 10.0 (TID 154). 6606 bytes result sent to driver -26/04/01 08:48:25 INFO TaskSetManager: Starting task 145.0 in stage 10.0 (TID 162) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:48:25 INFO Executor: Running task 145.0 in stage 10.0 (TID 162) -26/04/01 08:48:25 INFO TaskSetManager: Finished task 137.0 in stage 10.0 (TID 154) in 7904 ms on 10.0.0.133 (executor driver) (138/208) -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO Executor: Finished task 138.0 in stage 10.0 (TID 155). 6606 bytes result sent to driver -26/04/01 08:48:25 INFO Executor: Finished task 139.0 in stage 10.0 (TID 156). 6606 bytes result sent to driver -26/04/01 08:48:25 INFO TaskSetManager: Starting task 146.0 in stage 10.0 (TID 163) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:25 INFO Executor: Running task 146.0 in stage 10.0 (TID 163) -26/04/01 08:48:25 INFO TaskSetManager: Starting task 147.0 in stage 10.0 (TID 164) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:25 INFO TaskSetManager: Finished task 138.0 in stage 10.0 (TID 155) in 7886 ms on 10.0.0.133 (executor driver) (139/208) -26/04/01 08:48:25 INFO Executor: Running task 147.0 in stage 10.0 (TID 164) -26/04/01 08:48:25 INFO TaskSetManager: Finished task 139.0 in stage 10.0 (TID 156) in 7882 ms on 10.0.0.133 (executor driver) (140/208) -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO Executor: Finished task 140.0 in stage 10.0 (TID 157). 6606 bytes result sent to driver -26/04/01 08:48:25 INFO TaskSetManager: Starting task 148.0 in stage 10.0 (TID 165) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:25 INFO TaskSetManager: Finished task 140.0 in stage 10.0 (TID 157) in 7917 ms on 10.0.0.133 (executor driver) (141/208) -26/04/01 08:48:25 INFO Executor: Running task 148.0 in stage 10.0 (TID 165) -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO Executor: Finished task 141.0 in stage 10.0 (TID 158). 6606 bytes result sent to driver -26/04/01 08:48:25 INFO TaskSetManager: Starting task 149.0 in stage 10.0 (TID 166) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:25 INFO Executor: Running task 149.0 in stage 10.0 (TID 166) -26/04/01 08:48:25 INFO TaskSetManager: Finished task 141.0 in stage 10.0 (TID 158) in 7924 ms on 10.0.0.133 (executor driver) (142/208) -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO Executor: Finished task 142.0 in stage 10.0 (TID 159). 6606 bytes result sent to driver -26/04/01 08:48:25 INFO TaskSetManager: Starting task 150.0 in stage 10.0 (TID 167) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:48:25 INFO TaskSetManager: Finished task 142.0 in stage 10.0 (TID 159) in 7914 ms on 10.0.0.133 (executor driver) (143/208) -26/04/01 08:48:25 INFO Executor: Running task 150.0 in stage 10.0 (TID 167) -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO Executor: Finished task 143.0 in stage 10.0 (TID 160). 6606 bytes result sent to driver -26/04/01 08:48:25 INFO TaskSetManager: Starting task 151.0 in stage 10.0 (TID 168) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:25 INFO TaskSetManager: Finished task 143.0 in stage 10.0 (TID 160) in 7886 ms on 10.0.0.133 (executor driver) (144/208) -26/04/01 08:48:25 INFO Executor: Running task 151.0 in stage 10.0 (TID 168) -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO Executor: Finished task 144.0 in stage 10.0 (TID 161). 6606 bytes result sent to driver -26/04/01 08:48:33 INFO TaskSetManager: Starting task 152.0 in stage 10.0 (TID 169) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:33 INFO TaskSetManager: Finished task 144.0 in stage 10.0 (TID 161) in 7943 ms on 10.0.0.133 (executor driver) (145/208) -26/04/01 08:48:33 INFO Executor: Running task 152.0 in stage 10.0 (TID 169) -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO Executor: Finished task 145.0 in stage 10.0 (TID 162). 6606 bytes result sent to driver -26/04/01 08:48:33 INFO TaskSetManager: Starting task 153.0 in stage 10.0 (TID 170) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:33 INFO TaskSetManager: Finished task 145.0 in stage 10.0 (TID 162) in 7949 ms on 10.0.0.133 (executor driver) (146/208) -26/04/01 08:48:33 INFO Executor: Running task 153.0 in stage 10.0 (TID 170) -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO Executor: Finished task 147.0 in stage 10.0 (TID 164). 6606 bytes result sent to driver -26/04/01 08:48:33 INFO TaskSetManager: Starting task 154.0 in stage 10.0 (TID 171) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:33 INFO TaskSetManager: Finished task 147.0 in stage 10.0 (TID 164) in 7941 ms on 10.0.0.133 (executor driver) (147/208) -26/04/01 08:48:33 INFO Executor: Running task 154.0 in stage 10.0 (TID 171) -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO Executor: Finished task 146.0 in stage 10.0 (TID 163). 6606 bytes result sent to driver -26/04/01 08:48:33 INFO TaskSetManager: Starting task 155.0 in stage 10.0 (TID 172) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:33 INFO TaskSetManager: Finished task 146.0 in stage 10.0 (TID 163) in 7952 ms on 10.0.0.133 (executor driver) (148/208) -26/04/01 08:48:33 INFO Executor: Running task 155.0 in stage 10.0 (TID 172) -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO Executor: Finished task 148.0 in stage 10.0 (TID 165). 6606 bytes result sent to driver -26/04/01 08:48:33 INFO TaskSetManager: Starting task 156.0 in stage 10.0 (TID 173) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:48:33 INFO TaskSetManager: Finished task 148.0 in stage 10.0 (TID 165) in 7939 ms on 10.0.0.133 (executor driver) (149/208) -26/04/01 08:48:33 INFO Executor: Running task 156.0 in stage 10.0 (TID 173) -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO Executor: Finished task 149.0 in stage 10.0 (TID 166). 6606 bytes result sent to driver -26/04/01 08:48:33 INFO TaskSetManager: Starting task 157.0 in stage 10.0 (TID 174) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:33 INFO Executor: Running task 157.0 in stage 10.0 (TID 174) -26/04/01 08:48:33 INFO TaskSetManager: Finished task 149.0 in stage 10.0 (TID 166) in 7935 ms on 10.0.0.133 (executor driver) (150/208) -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO Executor: Finished task 150.0 in stage 10.0 (TID 167). 6606 bytes result sent to driver -26/04/01 08:48:33 INFO TaskSetManager: Starting task 158.0 in stage 10.0 (TID 175) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:33 INFO TaskSetManager: Finished task 150.0 in stage 10.0 (TID 167) in 7951 ms on 10.0.0.133 (executor driver) (151/208) -26/04/01 08:48:33 INFO Executor: Running task 158.0 in stage 10.0 (TID 175) -26/04/01 08:48:33 INFO Executor: Finished task 151.0 in stage 10.0 (TID 168). 6606 bytes result sent to driver -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO TaskSetManager: Starting task 159.0 in stage 10.0 (TID 176) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:33 INFO TaskSetManager: Finished task 151.0 in stage 10.0 (TID 168) in 7926 ms on 10.0.0.133 (executor driver) (152/208) -26/04/01 08:48:33 INFO Executor: Running task 159.0 in stage 10.0 (TID 176) -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO Executor: Finished task 152.0 in stage 10.0 (TID 169). 6563 bytes result sent to driver -26/04/01 08:48:41 INFO TaskSetManager: Starting task 160.0 in stage 10.0 (TID 177) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:41 INFO TaskSetManager: Finished task 152.0 in stage 10.0 (TID 169) in 8113 ms on 10.0.0.133 (executor driver) (153/208) -26/04/01 08:48:41 INFO Executor: Running task 160.0 in stage 10.0 (TID 177) -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO Executor: Finished task 154.0 in stage 10.0 (TID 171). 6563 bytes result sent to driver -26/04/01 08:48:41 INFO TaskSetManager: Starting task 161.0 in stage 10.0 (TID 178) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:41 INFO TaskSetManager: Finished task 154.0 in stage 10.0 (TID 171) in 8098 ms on 10.0.0.133 (executor driver) (154/208) -26/04/01 08:48:41 INFO Executor: Running task 161.0 in stage 10.0 (TID 178) -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO Executor: Finished task 155.0 in stage 10.0 (TID 172). 6563 bytes result sent to driver -26/04/01 08:48:41 INFO TaskSetManager: Starting task 162.0 in stage 10.0 (TID 179) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:48:41 INFO TaskSetManager: Finished task 155.0 in stage 10.0 (TID 172) in 8097 ms on 10.0.0.133 (executor driver) (155/208) -26/04/01 08:48:41 INFO Executor: Running task 162.0 in stage 10.0 (TID 179) -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO Executor: Finished task 153.0 in stage 10.0 (TID 170). 6563 bytes result sent to driver -26/04/01 08:48:41 INFO TaskSetManager: Starting task 163.0 in stage 10.0 (TID 180) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:48:41 INFO TaskSetManager: Finished task 153.0 in stage 10.0 (TID 170) in 8189 ms on 10.0.0.133 (executor driver) (156/208) -26/04/01 08:48:41 INFO Executor: Running task 163.0 in stage 10.0 (TID 180) -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO Executor: Finished task 157.0 in stage 10.0 (TID 174). 6563 bytes result sent to driver -26/04/01 08:48:41 INFO TaskSetManager: Starting task 164.0 in stage 10.0 (TID 181) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:41 INFO TaskSetManager: Finished task 157.0 in stage 10.0 (TID 174) in 8127 ms on 10.0.0.133 (executor driver) (157/208) -26/04/01 08:48:41 INFO Executor: Running task 164.0 in stage 10.0 (TID 181) -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO Executor: Finished task 156.0 in stage 10.0 (TID 173). 6563 bytes result sent to driver -26/04/01 08:48:41 INFO TaskSetManager: Starting task 165.0 in stage 10.0 (TID 182) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:41 INFO Executor: Running task 165.0 in stage 10.0 (TID 182) -26/04/01 08:48:41 INFO TaskSetManager: Finished task 156.0 in stage 10.0 (TID 173) in 8146 ms on 10.0.0.133 (executor driver) (158/208) -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO Executor: Finished task 158.0 in stage 10.0 (TID 175). 6563 bytes result sent to driver -26/04/01 08:48:41 INFO TaskSetManager: Starting task 166.0 in stage 10.0 (TID 183) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:41 INFO TaskSetManager: Finished task 158.0 in stage 10.0 (TID 175) in 8175 ms on 10.0.0.133 (executor driver) (159/208) -26/04/01 08:48:41 INFO Executor: Running task 166.0 in stage 10.0 (TID 183) -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO Executor: Finished task 159.0 in stage 10.0 (TID 176). 6563 bytes result sent to driver -26/04/01 08:48:41 INFO TaskSetManager: Starting task 167.0 in stage 10.0 (TID 184) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:41 INFO TaskSetManager: Finished task 159.0 in stage 10.0 (TID 176) in 8190 ms on 10.0.0.133 (executor driver) (160/208) -26/04/01 08:48:41 INFO Executor: Running task 167.0 in stage 10.0 (TID 184) -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO Executor: Finished task 160.0 in stage 10.0 (TID 177). 6606 bytes result sent to driver -26/04/01 08:48:49 INFO TaskSetManager: Starting task 168.0 in stage 10.0 (TID 185) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:48:49 INFO TaskSetManager: Finished task 160.0 in stage 10.0 (TID 177) in 7979 ms on 10.0.0.133 (executor driver) (161/208) -26/04/01 08:48:49 INFO Executor: Running task 168.0 in stage 10.0 (TID 185) -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO Executor: Finished task 162.0 in stage 10.0 (TID 179). 6606 bytes result sent to driver -26/04/01 08:48:49 INFO TaskSetManager: Starting task 169.0 in stage 10.0 (TID 186) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:49 INFO TaskSetManager: Finished task 162.0 in stage 10.0 (TID 179) in 7955 ms on 10.0.0.133 (executor driver) (162/208) -26/04/01 08:48:49 INFO Executor: Running task 169.0 in stage 10.0 (TID 186) -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO Executor: Finished task 161.0 in stage 10.0 (TID 178). 6606 bytes result sent to driver -26/04/01 08:48:49 INFO TaskSetManager: Starting task 170.0 in stage 10.0 (TID 187) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:49 INFO TaskSetManager: Finished task 161.0 in stage 10.0 (TID 178) in 7977 ms on 10.0.0.133 (executor driver) (163/208) -26/04/01 08:48:49 INFO Executor: Running task 170.0 in stage 10.0 (TID 187) -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO Executor: Finished task 163.0 in stage 10.0 (TID 180). 6606 bytes result sent to driver -26/04/01 08:48:49 INFO TaskSetManager: Starting task 171.0 in stage 10.0 (TID 188) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:49 INFO TaskSetManager: Finished task 163.0 in stage 10.0 (TID 180) in 7959 ms on 10.0.0.133 (executor driver) (164/208) -26/04/01 08:48:49 INFO Executor: Running task 171.0 in stage 10.0 (TID 188) -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO Executor: Finished task 165.0 in stage 10.0 (TID 182). 6606 bytes result sent to driver -26/04/01 08:48:49 INFO TaskSetManager: Starting task 172.0 in stage 10.0 (TID 189) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:49 INFO TaskSetManager: Finished task 165.0 in stage 10.0 (TID 182) in 7952 ms on 10.0.0.133 (executor driver) (165/208) -26/04/01 08:48:49 INFO Executor: Running task 172.0 in stage 10.0 (TID 189) -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO Executor: Finished task 164.0 in stage 10.0 (TID 181). 6606 bytes result sent to driver -26/04/01 08:48:49 INFO TaskSetManager: Starting task 173.0 in stage 10.0 (TID 190) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:49 INFO TaskSetManager: Finished task 164.0 in stage 10.0 (TID 181) in 7983 ms on 10.0.0.133 (executor driver) (166/208) -26/04/01 08:48:49 INFO Executor: Running task 173.0 in stage 10.0 (TID 190) -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO Executor: Finished task 166.0 in stage 10.0 (TID 183). 6606 bytes result sent to driver -26/04/01 08:48:49 INFO TaskSetManager: Starting task 174.0 in stage 10.0 (TID 191) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9267 bytes) -26/04/01 08:48:49 INFO TaskSetManager: Finished task 166.0 in stage 10.0 (TID 183) in 7961 ms on 10.0.0.133 (executor driver) (167/208) -26/04/01 08:48:49 INFO Executor: Running task 174.0 in stage 10.0 (TID 191) -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO Executor: Finished task 167.0 in stage 10.0 (TID 184). 6606 bytes result sent to driver -26/04/01 08:48:49 INFO TaskSetManager: Starting task 175.0 in stage 10.0 (TID 192) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9272 bytes) -26/04/01 08:48:49 INFO TaskSetManager: Finished task 167.0 in stage 10.0 (TID 184) in 7977 ms on 10.0.0.133 (executor driver) (168/208) -26/04/01 08:48:49 INFO Executor: Running task 175.0 in stage 10.0 (TID 192) -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO Executor: Finished task 168.0 in stage 10.0 (TID 185). 6606 bytes result sent to driver -26/04/01 08:48:57 INFO TaskSetManager: Starting task 176.0 in stage 10.0 (TID 193) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:57 INFO TaskSetManager: Finished task 168.0 in stage 10.0 (TID 185) in 8012 ms on 10.0.0.133 (executor driver) (169/208) -26/04/01 08:48:57 INFO Executor: Running task 176.0 in stage 10.0 (TID 193) -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO Executor: Finished task 170.0 in stage 10.0 (TID 187). 6606 bytes result sent to driver -26/04/01 08:48:57 INFO TaskSetManager: Starting task 177.0 in stage 10.0 (TID 194) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:57 INFO TaskSetManager: Finished task 170.0 in stage 10.0 (TID 187) in 8021 ms on 10.0.0.133 (executor driver) (170/208) -26/04/01 08:48:57 INFO Executor: Running task 177.0 in stage 10.0 (TID 194) -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO Executor: Finished task 169.0 in stage 10.0 (TID 186). 6606 bytes result sent to driver -26/04/01 08:48:57 INFO TaskSetManager: Starting task 178.0 in stage 10.0 (TID 195) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:57 INFO TaskSetManager: Finished task 169.0 in stage 10.0 (TID 186) in 8039 ms on 10.0.0.133 (executor driver) (171/208) -26/04/01 08:48:57 INFO Executor: Running task 178.0 in stage 10.0 (TID 195) -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO Executor: Finished task 171.0 in stage 10.0 (TID 188). 6606 bytes result sent to driver -26/04/01 08:48:57 INFO TaskSetManager: Starting task 179.0 in stage 10.0 (TID 196) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:57 INFO TaskSetManager: Finished task 171.0 in stage 10.0 (TID 188) in 8015 ms on 10.0.0.133 (executor driver) (172/208) -26/04/01 08:48:57 INFO Executor: Running task 179.0 in stage 10.0 (TID 196) -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO Executor: Finished task 173.0 in stage 10.0 (TID 190). 6606 bytes result sent to driver -26/04/01 08:48:57 INFO TaskSetManager: Starting task 180.0 in stage 10.0 (TID 197) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:48:57 INFO TaskSetManager: Finished task 173.0 in stage 10.0 (TID 190) in 8027 ms on 10.0.0.133 (executor driver) (173/208) -26/04/01 08:48:57 INFO Executor: Running task 180.0 in stage 10.0 (TID 197) -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO Executor: Finished task 172.0 in stage 10.0 (TID 189). 6606 bytes result sent to driver -26/04/01 08:48:57 INFO TaskSetManager: Starting task 181.0 in stage 10.0 (TID 198) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:48:57 INFO TaskSetManager: Finished task 172.0 in stage 10.0 (TID 189) in 8107 ms on 10.0.0.133 (executor driver) (174/208) -26/04/01 08:48:57 INFO Executor: Running task 181.0 in stage 10.0 (TID 198) -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO Executor: Finished task 174.0 in stage 10.0 (TID 191). 6606 bytes result sent to driver -26/04/01 08:48:57 INFO TaskSetManager: Starting task 182.0 in stage 10.0 (TID 199) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:57 INFO TaskSetManager: Finished task 174.0 in stage 10.0 (TID 191) in 8038 ms on 10.0.0.133 (executor driver) (175/208) -26/04/01 08:48:57 INFO Executor: Running task 182.0 in stage 10.0 (TID 199) -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO Executor: Finished task 175.0 in stage 10.0 (TID 192). 6606 bytes result sent to driver -26/04/01 08:48:57 INFO TaskSetManager: Starting task 183.0 in stage 10.0 (TID 200) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:48:57 INFO Executor: Running task 183.0 in stage 10.0 (TID 200) -26/04/01 08:48:57 INFO TaskSetManager: Finished task 175.0 in stage 10.0 (TID 192) in 8033 ms on 10.0.0.133 (executor driver) (176/208) -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:48:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO Executor: Finished task 176.0 in stage 10.0 (TID 193). 6606 bytes result sent to driver -26/04/01 08:49:05 INFO TaskSetManager: Starting task 184.0 in stage 10.0 (TID 201) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:49:05 INFO TaskSetManager: Finished task 176.0 in stage 10.0 (TID 193) in 7965 ms on 10.0.0.133 (executor driver) (177/208) -26/04/01 08:49:05 INFO Executor: Running task 184.0 in stage 10.0 (TID 201) -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO Executor: Finished task 178.0 in stage 10.0 (TID 195). 6606 bytes result sent to driver -26/04/01 08:49:05 INFO TaskSetManager: Starting task 185.0 in stage 10.0 (TID 202) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:49:05 INFO TaskSetManager: Finished task 178.0 in stage 10.0 (TID 195) in 7957 ms on 10.0.0.133 (executor driver) (178/208) -26/04/01 08:49:05 INFO Executor: Finished task 177.0 in stage 10.0 (TID 194). 6606 bytes result sent to driver -26/04/01 08:49:05 INFO Executor: Running task 185.0 in stage 10.0 (TID 202) -26/04/01 08:49:05 INFO TaskSetManager: Starting task 186.0 in stage 10.0 (TID 203) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:05 INFO TaskSetManager: Finished task 177.0 in stage 10.0 (TID 194) in 7963 ms on 10.0.0.133 (executor driver) (179/208) -26/04/01 08:49:05 INFO Executor: Running task 186.0 in stage 10.0 (TID 203) -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO Executor: Finished task 179.0 in stage 10.0 (TID 196). 6606 bytes result sent to driver -26/04/01 08:49:05 INFO TaskSetManager: Starting task 187.0 in stage 10.0 (TID 204) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9273 bytes) -26/04/01 08:49:05 INFO TaskSetManager: Finished task 179.0 in stage 10.0 (TID 196) in 7949 ms on 10.0.0.133 (executor driver) (180/208) -26/04/01 08:49:05 INFO Executor: Running task 187.0 in stage 10.0 (TID 204) -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO Executor: Finished task 180.0 in stage 10.0 (TID 197). 6606 bytes result sent to driver -26/04/01 08:49:05 INFO TaskSetManager: Starting task 188.0 in stage 10.0 (TID 205) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:49:05 INFO TaskSetManager: Finished task 180.0 in stage 10.0 (TID 197) in 7949 ms on 10.0.0.133 (executor driver) (181/208) -26/04/01 08:49:05 INFO Executor: Running task 188.0 in stage 10.0 (TID 205) -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO Executor: Finished task 181.0 in stage 10.0 (TID 198). 6606 bytes result sent to driver -26/04/01 08:49:05 INFO TaskSetManager: Starting task 189.0 in stage 10.0 (TID 206) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:49:05 INFO TaskSetManager: Finished task 181.0 in stage 10.0 (TID 198) in 7956 ms on 10.0.0.133 (executor driver) (182/208) -26/04/01 08:49:05 INFO Executor: Running task 189.0 in stage 10.0 (TID 206) -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO Executor: Finished task 182.0 in stage 10.0 (TID 199). 6606 bytes result sent to driver -26/04/01 08:49:05 INFO TaskSetManager: Starting task 190.0 in stage 10.0 (TID 207) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:49:05 INFO TaskSetManager: Finished task 182.0 in stage 10.0 (TID 199) in 7964 ms on 10.0.0.133 (executor driver) (183/208) -26/04/01 08:49:05 INFO Executor: Running task 190.0 in stage 10.0 (TID 207) -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO Executor: Finished task 183.0 in stage 10.0 (TID 200). 6606 bytes result sent to driver -26/04/01 08:49:05 INFO TaskSetManager: Starting task 191.0 in stage 10.0 (TID 208) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9274 bytes) -26/04/01 08:49:05 INFO Executor: Running task 191.0 in stage 10.0 (TID 208) -26/04/01 08:49:05 INFO TaskSetManager: Finished task 183.0 in stage 10.0 (TID 200) in 7952 ms on 10.0.0.133 (executor driver) (184/208) -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO Executor: Finished task 184.0 in stage 10.0 (TID 201). 6606 bytes result sent to driver -26/04/01 08:49:13 INFO TaskSetManager: Starting task 192.0 in stage 10.0 (TID 209) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:49:13 INFO TaskSetManager: Finished task 184.0 in stage 10.0 (TID 201) in 7954 ms on 10.0.0.133 (executor driver) (185/208) -26/04/01 08:49:13 INFO Executor: Running task 192.0 in stage 10.0 (TID 209) -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO Executor: Finished task 186.0 in stage 10.0 (TID 203). 6606 bytes result sent to driver -26/04/01 08:49:13 INFO TaskSetManager: Starting task 193.0 in stage 10.0 (TID 210) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9346 bytes) -26/04/01 08:49:13 INFO Executor: Running task 193.0 in stage 10.0 (TID 210) -26/04/01 08:49:13 INFO TaskSetManager: Finished task 186.0 in stage 10.0 (TID 203) in 7969 ms on 10.0.0.133 (executor driver) (186/208) -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO Executor: Finished task 185.0 in stage 10.0 (TID 202). 6606 bytes result sent to driver -26/04/01 08:49:13 INFO TaskSetManager: Starting task 194.0 in stage 10.0 (TID 211) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:49:13 INFO Executor: Running task 194.0 in stage 10.0 (TID 211) -26/04/01 08:49:13 INFO TaskSetManager: Finished task 185.0 in stage 10.0 (TID 202) in 7972 ms on 10.0.0.133 (executor driver) (187/208) -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO Executor: Finished task 187.0 in stage 10.0 (TID 204). 6606 bytes result sent to driver -26/04/01 08:49:13 INFO TaskSetManager: Starting task 195.0 in stage 10.0 (TID 212) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:49:13 INFO Executor: Running task 195.0 in stage 10.0 (TID 212) -26/04/01 08:49:13 INFO TaskSetManager: Finished task 187.0 in stage 10.0 (TID 204) in 7978 ms on 10.0.0.133 (executor driver) (188/208) -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO Executor: Finished task 188.0 in stage 10.0 (TID 205). 6606 bytes result sent to driver -26/04/01 08:49:13 INFO TaskSetManager: Starting task 196.0 in stage 10.0 (TID 213) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9346 bytes) -26/04/01 08:49:13 INFO TaskSetManager: Finished task 188.0 in stage 10.0 (TID 205) in 7952 ms on 10.0.0.133 (executor driver) (189/208) -26/04/01 08:49:13 INFO Executor: Running task 196.0 in stage 10.0 (TID 213) -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO Executor: Finished task 189.0 in stage 10.0 (TID 206). 6606 bytes result sent to driver -26/04/01 08:49:13 INFO TaskSetManager: Starting task 197.0 in stage 10.0 (TID 214) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9346 bytes) -26/04/01 08:49:13 INFO TaskSetManager: Finished task 189.0 in stage 10.0 (TID 206) in 7951 ms on 10.0.0.133 (executor driver) (190/208) -26/04/01 08:49:13 INFO Executor: Running task 197.0 in stage 10.0 (TID 214) -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO Executor: Finished task 190.0 in stage 10.0 (TID 207). 6606 bytes result sent to driver -26/04/01 08:49:13 INFO TaskSetManager: Starting task 198.0 in stage 10.0 (TID 215) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:49:13 INFO Executor: Running task 198.0 in stage 10.0 (TID 215) -26/04/01 08:49:13 INFO TaskSetManager: Finished task 190.0 in stage 10.0 (TID 207) in 7947 ms on 10.0.0.133 (executor driver) (191/208) -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO Executor: Finished task 191.0 in stage 10.0 (TID 208). 6606 bytes result sent to driver -26/04/01 08:49:13 INFO TaskSetManager: Starting task 199.0 in stage 10.0 (TID 216) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:49:13 INFO TaskSetManager: Finished task 191.0 in stage 10.0 (TID 208) in 7944 ms on 10.0.0.133 (executor driver) (192/208) -26/04/01 08:49:13 INFO Executor: Running task 199.0 in stage 10.0 (TID 216) -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO Executor: Finished task 192.0 in stage 10.0 (TID 209). 6563 bytes result sent to driver -26/04/01 08:49:18 INFO TaskSetManager: Starting task 200.0 in stage 10.0 (TID 217) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9345 bytes) -26/04/01 08:49:18 INFO TaskSetManager: Finished task 192.0 in stage 10.0 (TID 209) in 4962 ms on 10.0.0.133 (executor driver) (193/208) -26/04/01 08:49:18 INFO Executor: Running task 200.0 in stage 10.0 (TID 217) -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO Executor: Finished task 193.0 in stage 10.0 (TID 210). 6563 bytes result sent to driver -26/04/01 08:49:18 INFO TaskSetManager: Starting task 201.0 in stage 10.0 (TID 218) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9346 bytes) -26/04/01 08:49:18 INFO TaskSetManager: Finished task 193.0 in stage 10.0 (TID 210) in 4945 ms on 10.0.0.133 (executor driver) (194/208) -26/04/01 08:49:18 INFO Executor: Running task 201.0 in stage 10.0 (TID 218) -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO Executor: Finished task 194.0 in stage 10.0 (TID 211). 6563 bytes result sent to driver -26/04/01 08:49:18 INFO TaskSetManager: Starting task 202.0 in stage 10.0 (TID 219) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9346 bytes) -26/04/01 08:49:18 INFO TaskSetManager: Finished task 194.0 in stage 10.0 (TID 211) in 4962 ms on 10.0.0.133 (executor driver) (195/208) -26/04/01 08:49:18 INFO Executor: Running task 202.0 in stage 10.0 (TID 219) -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO Executor: Finished task 195.0 in stage 10.0 (TID 212). 6563 bytes result sent to driver -26/04/01 08:49:18 INFO TaskSetManager: Starting task 203.0 in stage 10.0 (TID 220) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:49:18 INFO TaskSetManager: Finished task 195.0 in stage 10.0 (TID 212) in 4938 ms on 10.0.0.133 (executor driver) (196/208) -26/04/01 08:49:18 INFO Executor: Running task 203.0 in stage 10.0 (TID 220) -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO Executor: Finished task 196.0 in stage 10.0 (TID 213). 6563 bytes result sent to driver -26/04/01 08:49:18 INFO TaskSetManager: Starting task 204.0 in stage 10.0 (TID 221) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:49:18 INFO Executor: Running task 204.0 in stage 10.0 (TID 221) -26/04/01 08:49:18 INFO TaskSetManager: Finished task 196.0 in stage 10.0 (TID 213) in 4962 ms on 10.0.0.133 (executor driver) (197/208) -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO Executor: Finished task 197.0 in stage 10.0 (TID 214). 6563 bytes result sent to driver -26/04/01 08:49:18 INFO TaskSetManager: Starting task 205.0 in stage 10.0 (TID 222) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9347 bytes) -26/04/01 08:49:18 INFO TaskSetManager: Finished task 197.0 in stage 10.0 (TID 214) in 4935 ms on 10.0.0.133 (executor driver) (198/208) -26/04/01 08:49:18 INFO Executor: Running task 205.0 in stage 10.0 (TID 222) -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO Executor: Finished task 198.0 in stage 10.0 (TID 215). 6563 bytes result sent to driver -26/04/01 08:49:18 INFO Executor: Finished task 199.0 in stage 10.0 (TID 216). 6563 bytes result sent to driver -26/04/01 08:49:18 INFO TaskSetManager: Starting task 206.0 in stage 10.0 (TID 223) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9346 bytes) -26/04/01 08:49:18 INFO TaskSetManager: Starting task 207.0 in stage 10.0 (TID 224) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9346 bytes) -26/04/01 08:49:18 INFO Executor: Running task 206.0 in stage 10.0 (TID 223) -26/04/01 08:49:18 INFO TaskSetManager: Finished task 198.0 in stage 10.0 (TID 215) in 4944 ms on 10.0.0.133 (executor driver) (199/208) -26/04/01 08:49:18 INFO Executor: Running task 207.0 in stage 10.0 (TID 224) -26/04/01 08:49:18 INFO TaskSetManager: Finished task 199.0 in stage 10.0 (TID 216) in 4925 ms on 10.0.0.133 (executor driver) (200/208) -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO Executor: Finished task 200.0 in stage 10.0 (TID 217). 6606 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 0.0 in stage 11.0 (TID 225) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9463 bytes) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 200.0 in stage 10.0 (TID 217) in 4964 ms on 10.0.0.133 (executor driver) (201/208) -26/04/01 08:49:23 INFO Executor: Running task 0.0 in stage 11.0 (TID 225) -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO Executor: Finished task 201.0 in stage 10.0 (TID 218). 6606 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 1.0 in stage 11.0 (TID 226) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9464 bytes) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 201.0 in stage 10.0 (TID 218) in 4966 ms on 10.0.0.133 (executor driver) (202/208) -26/04/01 08:49:23 INFO Executor: Running task 1.0 in stage 11.0 (TID 226) -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO Executor: Finished task 202.0 in stage 10.0 (TID 219). 6606 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 2.0 in stage 11.0 (TID 227) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9464 bytes) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 202.0 in stage 10.0 (TID 219) in 4976 ms on 10.0.0.133 (executor driver) (203/208) -26/04/01 08:49:23 INFO Executor: Running task 2.0 in stage 11.0 (TID 227) -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO Executor: Finished task 0.0 in stage 11.0 (TID 225). 6563 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 3.0 in stage 11.0 (TID 228) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9465 bytes) -26/04/01 08:49:23 INFO Executor: Running task 3.0 in stage 11.0 (TID 228) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 0.0 in stage 11.0 (TID 225) in 110 ms on 10.0.0.133 (executor driver) (1/8) -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO Executor: Finished task 203.0 in stage 10.0 (TID 220). 6606 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 4.0 in stage 11.0 (TID 229) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9465 bytes) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 203.0 in stage 10.0 (TID 220) in 4986 ms on 10.0.0.133 (executor driver) (204/208) -26/04/01 08:49:23 INFO Executor: Running task 4.0 in stage 11.0 (TID 229) -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO Executor: Finished task 1.0 in stage 11.0 (TID 226). 6563 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 5.0 in stage 11.0 (TID 230) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9464 bytes) -26/04/01 08:49:23 INFO Executor: Running task 5.0 in stage 11.0 (TID 230) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 1.0 in stage 11.0 (TID 226) in 110 ms on 10.0.0.133 (executor driver) (2/8) -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO Executor: Finished task 2.0 in stage 11.0 (TID 227). 6563 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 6.0 in stage 11.0 (TID 231) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9463 bytes) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 2.0 in stage 11.0 (TID 227) in 110 ms on 10.0.0.133 (executor driver) (3/8) -26/04/01 08:49:23 INFO Executor: Running task 6.0 in stage 11.0 (TID 231) -26/04/01 08:49:23 INFO Executor: Finished task 3.0 in stage 11.0 (TID 228). 6563 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 7.0 in stage 11.0 (TID 232) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9463 bytes) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 3.0 in stage 11.0 (TID 228) in 107 ms on 10.0.0.133 (executor driver) (4/8) -26/04/01 08:49:23 INFO Executor: Running task 7.0 in stage 11.0 (TID 232) -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO Executor: Finished task 204.0 in stage 10.0 (TID 221). 6606 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 233) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:23 INFO Executor: Running task 0.0 in stage 12.0 (TID 233) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 204.0 in stage 10.0 (TID 221) in 4956 ms on 10.0.0.133 (executor driver) (205/208) -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO Executor: Finished task 4.0 in stage 11.0 (TID 229). 6563 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 1.0 in stage 12.0 (TID 234) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 4.0 in stage 11.0 (TID 229) in 108 ms on 10.0.0.133 (executor driver) (5/8) -26/04/01 08:49:23 INFO Executor: Running task 1.0 in stage 12.0 (TID 234) -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO Executor: Finished task 5.0 in stage 11.0 (TID 230). 6563 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 2.0 in stage 12.0 (TID 235) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 5.0 in stage 11.0 (TID 230) in 108 ms on 10.0.0.133 (executor driver) (6/8) -26/04/01 08:49:23 INFO Executor: Running task 2.0 in stage 12.0 (TID 235) -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO Executor: Finished task 205.0 in stage 10.0 (TID 222). 6606 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 3.0 in stage 12.0 (TID 236) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 205.0 in stage 10.0 (TID 222) in 5005 ms on 10.0.0.133 (executor driver) (206/208) -26/04/01 08:49:23 INFO Executor: Running task 3.0 in stage 12.0 (TID 236) -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO Executor: Finished task 6.0 in stage 11.0 (TID 231). 6563 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 4.0 in stage 12.0 (TID 237) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 6.0 in stage 11.0 (TID 231) in 114 ms on 10.0.0.133 (executor driver) (7/8) -26/04/01 08:49:23 INFO Executor: Running task 4.0 in stage 12.0 (TID 237) -26/04/01 08:49:23 INFO Executor: Finished task 7.0 in stage 11.0 (TID 232). 6563 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 5.0 in stage 12.0 (TID 238) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 7.0 in stage 11.0 (TID 232) in 113 ms on 10.0.0.133 (executor driver) (8/8) -26/04/01 08:49:23 INFO Executor: Running task 5.0 in stage 12.0 (TID 238) -26/04/01 08:49:23 INFO TaskSchedulerImpl: Removed TaskSet 11.0, whose tasks have all completed, from pool -26/04/01 08:49:23 INFO DAGScheduler: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 203.485 s -26/04/01 08:49:23 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:49:23 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 12, ShuffleMapStage 13, ShuffleMapStage 10) -26/04/01 08:49:23 INFO DAGScheduler: waiting: Set() -26/04/01 08:49:23 INFO DAGScheduler: failed: Set() -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO Executor: Finished task 207.0 in stage 10.0 (TID 224). 6606 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 6.0 in stage 12.0 (TID 239) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 207.0 in stage 10.0 (TID 224) in 4961 ms on 10.0.0.133 (executor driver) (207/208) -26/04/01 08:49:23 INFO Executor: Running task 6.0 in stage 12.0 (TID 239) -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO Executor: Finished task 206.0 in stage 10.0 (TID 223). 6606 bytes result sent to driver -26/04/01 08:49:23 INFO TaskSetManager: Starting task 7.0 in stage 12.0 (TID 240) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:23 INFO TaskSetManager: Finished task 206.0 in stage 10.0 (TID 223) in 4966 ms on 10.0.0.133 (executor driver) (208/208) -26/04/01 08:49:23 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool -26/04/01 08:49:23 INFO Executor: Running task 7.0 in stage 12.0 (TID 240) -26/04/01 08:49:23 INFO DAGScheduler: ShuffleMapStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 203.602 s -26/04/01 08:49:23 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:49:23 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 12, ShuffleMapStage 13) -26/04/01 08:49:23 INFO DAGScheduler: waiting: Set() -26/04/01 08:49:23 INFO DAGScheduler: failed: Set() -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:26 INFO Executor: Finished task 1.0 in stage 12.0 (TID 234). 6563 bytes result sent to driver -26/04/01 08:49:26 INFO TaskSetManager: Starting task 8.0 in stage 12.0 (TID 241) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:26 INFO TaskSetManager: Finished task 1.0 in stage 12.0 (TID 234) in 3309 ms on 10.0.0.133 (executor driver) (1/37) -26/04/01 08:49:26 INFO Executor: Running task 8.0 in stage 12.0 (TID 241) -26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:26 INFO Executor: Finished task 0.0 in stage 12.0 (TID 233). 6563 bytes result sent to driver -26/04/01 08:49:26 INFO TaskSetManager: Starting task 9.0 in stage 12.0 (TID 242) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:26 INFO Executor: Running task 9.0 in stage 12.0 (TID 242) -26/04/01 08:49:26 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 233) in 3341 ms on 10.0.0.133 (executor driver) (2/37) -26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:26 INFO Executor: Finished task 2.0 in stage 12.0 (TID 235). 6563 bytes result sent to driver -26/04/01 08:49:26 INFO TaskSetManager: Starting task 10.0 in stage 12.0 (TID 243) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:26 INFO TaskSetManager: Finished task 2.0 in stage 12.0 (TID 235) in 3306 ms on 10.0.0.133 (executor driver) (3/37) -26/04/01 08:49:26 INFO Executor: Running task 10.0 in stage 12.0 (TID 243) -26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:26 INFO Executor: Finished task 3.0 in stage 12.0 (TID 236). 6563 bytes result sent to driver -26/04/01 08:49:26 INFO TaskSetManager: Starting task 11.0 in stage 12.0 (TID 244) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:26 INFO TaskSetManager: Finished task 3.0 in stage 12.0 (TID 236) in 3318 ms on 10.0.0.133 (executor driver) (4/37) -26/04/01 08:49:26 INFO Executor: Running task 11.0 in stage 12.0 (TID 244) -26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:26 INFO Executor: Finished task 4.0 in stage 12.0 (TID 237). 6563 bytes result sent to driver -26/04/01 08:49:26 INFO TaskSetManager: Starting task 12.0 in stage 12.0 (TID 245) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:26 INFO Executor: Running task 12.0 in stage 12.0 (TID 245) -26/04/01 08:49:26 INFO TaskSetManager: Finished task 4.0 in stage 12.0 (TID 237) in 3320 ms on 10.0.0.133 (executor driver) (5/37) -26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:26 INFO Executor: Finished task 5.0 in stage 12.0 (TID 238). 6606 bytes result sent to driver -26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:26 INFO TaskSetManager: Starting task 13.0 in stage 12.0 (TID 246) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:26 INFO TaskSetManager: Finished task 5.0 in stage 12.0 (TID 238) in 3323 ms on 10.0.0.133 (executor driver) (6/37) -26/04/01 08:49:26 INFO Executor: Running task 13.0 in stage 12.0 (TID 246) -26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:27 INFO Executor: Finished task 6.0 in stage 12.0 (TID 239). 6606 bytes result sent to driver -26/04/01 08:49:27 INFO TaskSetManager: Starting task 14.0 in stage 12.0 (TID 247) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:27 INFO TaskSetManager: Finished task 6.0 in stage 12.0 (TID 239) in 3309 ms on 10.0.0.133 (executor driver) (7/37) -26/04/01 08:49:27 INFO Executor: Running task 14.0 in stage 12.0 (TID 247) -26/04/01 08:49:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:27 INFO Executor: Finished task 7.0 in stage 12.0 (TID 240). 6606 bytes result sent to driver -26/04/01 08:49:27 INFO TaskSetManager: Starting task 15.0 in stage 12.0 (TID 248) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:27 INFO TaskSetManager: Finished task 7.0 in stage 12.0 (TID 240) in 3329 ms on 10.0.0.133 (executor driver) (8/37) -26/04/01 08:49:27 INFO Executor: Running task 15.0 in stage 12.0 (TID 248) -26/04/01 08:49:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO Executor: Finished task 8.0 in stage 12.0 (TID 241). 6606 bytes result sent to driver -26/04/01 08:49:30 INFO TaskSetManager: Starting task 16.0 in stage 12.0 (TID 249) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:30 INFO TaskSetManager: Finished task 8.0 in stage 12.0 (TID 241) in 3299 ms on 10.0.0.133 (executor driver) (9/37) -26/04/01 08:49:30 INFO Executor: Running task 16.0 in stage 12.0 (TID 249) -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO Executor: Finished task 9.0 in stage 12.0 (TID 242). 6606 bytes result sent to driver -26/04/01 08:49:30 INFO TaskSetManager: Starting task 17.0 in stage 12.0 (TID 250) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:30 INFO Executor: Running task 17.0 in stage 12.0 (TID 250) -26/04/01 08:49:30 INFO TaskSetManager: Finished task 9.0 in stage 12.0 (TID 242) in 3296 ms on 10.0.0.133 (executor driver) (10/37) -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO Executor: Finished task 11.0 in stage 12.0 (TID 244). 6606 bytes result sent to driver -26/04/01 08:49:30 INFO Executor: Finished task 10.0 in stage 12.0 (TID 243). 6606 bytes result sent to driver -26/04/01 08:49:30 INFO TaskSetManager: Starting task 18.0 in stage 12.0 (TID 251) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:30 INFO TaskSetManager: Finished task 11.0 in stage 12.0 (TID 244) in 3292 ms on 10.0.0.133 (executor driver) (11/37) -26/04/01 08:49:30 INFO Executor: Running task 18.0 in stage 12.0 (TID 251) -26/04/01 08:49:30 INFO TaskSetManager: Starting task 19.0 in stage 12.0 (TID 252) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:30 INFO TaskSetManager: Finished task 10.0 in stage 12.0 (TID 243) in 3316 ms on 10.0.0.133 (executor driver) (12/37) -26/04/01 08:49:30 INFO Executor: Running task 19.0 in stage 12.0 (TID 252) -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO Executor: Finished task 12.0 in stage 12.0 (TID 245). 6606 bytes result sent to driver -26/04/01 08:49:30 INFO TaskSetManager: Starting task 20.0 in stage 12.0 (TID 253) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:30 INFO TaskSetManager: Finished task 12.0 in stage 12.0 (TID 245) in 3300 ms on 10.0.0.133 (executor driver) (13/37) -26/04/01 08:49:30 INFO Executor: Running task 20.0 in stage 12.0 (TID 253) -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO Executor: Finished task 13.0 in stage 12.0 (TID 246). 6563 bytes result sent to driver -26/04/01 08:49:30 INFO TaskSetManager: Starting task 21.0 in stage 12.0 (TID 254) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:30 INFO TaskSetManager: Finished task 13.0 in stage 12.0 (TID 246) in 3307 ms on 10.0.0.133 (executor driver) (14/37) -26/04/01 08:49:30 INFO Executor: Running task 21.0 in stage 12.0 (TID 254) -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO Executor: Finished task 14.0 in stage 12.0 (TID 247). 6563 bytes result sent to driver -26/04/01 08:49:30 INFO TaskSetManager: Starting task 22.0 in stage 12.0 (TID 255) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:30 INFO TaskSetManager: Finished task 14.0 in stage 12.0 (TID 247) in 3310 ms on 10.0.0.133 (executor driver) (15/37) -26/04/01 08:49:30 INFO Executor: Running task 22.0 in stage 12.0 (TID 255) -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO Executor: Finished task 15.0 in stage 12.0 (TID 248). 6563 bytes result sent to driver -26/04/01 08:49:30 INFO TaskSetManager: Starting task 23.0 in stage 12.0 (TID 256) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:30 INFO Executor: Running task 23.0 in stage 12.0 (TID 256) -26/04/01 08:49:30 INFO TaskSetManager: Finished task 15.0 in stage 12.0 (TID 248) in 3301 ms on 10.0.0.133 (executor driver) (16/37) -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO Executor: Finished task 16.0 in stage 12.0 (TID 249). 6606 bytes result sent to driver -26/04/01 08:49:33 INFO TaskSetManager: Starting task 24.0 in stage 12.0 (TID 257) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:33 INFO TaskSetManager: Finished task 16.0 in stage 12.0 (TID 249) in 3303 ms on 10.0.0.133 (executor driver) (17/37) -26/04/01 08:49:33 INFO Executor: Running task 24.0 in stage 12.0 (TID 257) -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO Executor: Finished task 17.0 in stage 12.0 (TID 250). 6606 bytes result sent to driver -26/04/01 08:49:33 INFO TaskSetManager: Starting task 25.0 in stage 12.0 (TID 258) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:33 INFO TaskSetManager: Finished task 17.0 in stage 12.0 (TID 250) in 3297 ms on 10.0.0.133 (executor driver) (18/37) -26/04/01 08:49:33 INFO Executor: Running task 25.0 in stage 12.0 (TID 258) -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO Executor: Finished task 18.0 in stage 12.0 (TID 251). 6606 bytes result sent to driver -26/04/01 08:49:33 INFO Executor: Finished task 19.0 in stage 12.0 (TID 252). 6606 bytes result sent to driver -26/04/01 08:49:33 INFO TaskSetManager: Starting task 26.0 in stage 12.0 (TID 259) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:33 INFO Executor: Running task 26.0 in stage 12.0 (TID 259) -26/04/01 08:49:33 INFO TaskSetManager: Starting task 27.0 in stage 12.0 (TID 260) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:33 INFO TaskSetManager: Finished task 18.0 in stage 12.0 (TID 251) in 3305 ms on 10.0.0.133 (executor driver) (19/37) -26/04/01 08:49:33 INFO Executor: Running task 27.0 in stage 12.0 (TID 260) -26/04/01 08:49:33 INFO TaskSetManager: Finished task 19.0 in stage 12.0 (TID 252) in 3304 ms on 10.0.0.133 (executor driver) (20/37) -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO Executor: Finished task 20.0 in stage 12.0 (TID 253). 6606 bytes result sent to driver -26/04/01 08:49:33 INFO TaskSetManager: Starting task 28.0 in stage 12.0 (TID 261) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:33 INFO TaskSetManager: Finished task 20.0 in stage 12.0 (TID 253) in 3302 ms on 10.0.0.133 (executor driver) (21/37) -26/04/01 08:49:33 INFO Executor: Running task 28.0 in stage 12.0 (TID 261) -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO Executor: Finished task 21.0 in stage 12.0 (TID 254). 6606 bytes result sent to driver -26/04/01 08:49:33 INFO TaskSetManager: Starting task 29.0 in stage 12.0 (TID 262) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:33 INFO TaskSetManager: Finished task 21.0 in stage 12.0 (TID 254) in 3299 ms on 10.0.0.133 (executor driver) (22/37) -26/04/01 08:49:33 INFO Executor: Running task 29.0 in stage 12.0 (TID 262) -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO Executor: Finished task 22.0 in stage 12.0 (TID 255). 6606 bytes result sent to driver -26/04/01 08:49:33 INFO TaskSetManager: Starting task 30.0 in stage 12.0 (TID 263) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:33 INFO TaskSetManager: Finished task 22.0 in stage 12.0 (TID 255) in 3313 ms on 10.0.0.133 (executor driver) (23/37) -26/04/01 08:49:33 INFO Executor: Running task 30.0 in stage 12.0 (TID 263) -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO Executor: Finished task 23.0 in stage 12.0 (TID 256). 6606 bytes result sent to driver -26/04/01 08:49:33 INFO TaskSetManager: Starting task 31.0 in stage 12.0 (TID 264) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:33 INFO Executor: Running task 31.0 in stage 12.0 (TID 264) -26/04/01 08:49:33 INFO TaskSetManager: Finished task 23.0 in stage 12.0 (TID 256) in 3310 ms on 10.0.0.133 (executor driver) (24/37) -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO Executor: Finished task 24.0 in stage 12.0 (TID 257). 6563 bytes result sent to driver -26/04/01 08:49:36 INFO TaskSetManager: Starting task 32.0 in stage 12.0 (TID 265) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9694 bytes) -26/04/01 08:49:36 INFO TaskSetManager: Finished task 24.0 in stage 12.0 (TID 257) in 3300 ms on 10.0.0.133 (executor driver) (25/37) -26/04/01 08:49:36 INFO Executor: Running task 32.0 in stage 12.0 (TID 265) -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO Executor: Finished task 25.0 in stage 12.0 (TID 258). 6563 bytes result sent to driver -26/04/01 08:49:36 INFO TaskSetManager: Starting task 33.0 in stage 12.0 (TID 266) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9695 bytes) -26/04/01 08:49:36 INFO TaskSetManager: Finished task 25.0 in stage 12.0 (TID 258) in 3314 ms on 10.0.0.133 (executor driver) (26/37) -26/04/01 08:49:36 INFO Executor: Running task 33.0 in stage 12.0 (TID 266) -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO Executor: Finished task 27.0 in stage 12.0 (TID 260). 6563 bytes result sent to driver -26/04/01 08:49:36 INFO TaskSetManager: Starting task 34.0 in stage 12.0 (TID 267) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9691 bytes) -26/04/01 08:49:36 INFO TaskSetManager: Finished task 27.0 in stage 12.0 (TID 260) in 3300 ms on 10.0.0.133 (executor driver) (27/37) -26/04/01 08:49:36 INFO Executor: Running task 34.0 in stage 12.0 (TID 267) -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO Executor: Finished task 26.0 in stage 12.0 (TID 259). 6563 bytes result sent to driver -26/04/01 08:49:36 INFO TaskSetManager: Starting task 35.0 in stage 12.0 (TID 268) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9694 bytes) -26/04/01 08:49:36 INFO Executor: Running task 35.0 in stage 12.0 (TID 268) -26/04/01 08:49:36 INFO TaskSetManager: Finished task 26.0 in stage 12.0 (TID 259) in 3304 ms on 10.0.0.133 (executor driver) (28/37) -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO Executor: Finished task 28.0 in stage 12.0 (TID 261). 6563 bytes result sent to driver -26/04/01 08:49:36 INFO TaskSetManager: Starting task 36.0 in stage 12.0 (TID 269) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9482 bytes) -26/04/01 08:49:36 INFO TaskSetManager: Finished task 28.0 in stage 12.0 (TID 261) in 3310 ms on 10.0.0.133 (executor driver) (29/37) -26/04/01 08:49:36 INFO Executor: Running task 36.0 in stage 12.0 (TID 269) -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO Executor: Finished task 29.0 in stage 12.0 (TID 262). 6563 bytes result sent to driver -26/04/01 08:49:36 INFO TaskSetManager: Starting task 0.0 in stage 13.0 (TID 270) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:36 INFO TaskSetManager: Finished task 29.0 in stage 12.0 (TID 262) in 3312 ms on 10.0.0.133 (executor driver) (30/37) -26/04/01 08:49:36 INFO Executor: Running task 0.0 in stage 13.0 (TID 270) -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO Executor: Finished task 30.0 in stage 12.0 (TID 263). 6563 bytes result sent to driver -26/04/01 08:49:36 INFO TaskSetManager: Starting task 1.0 in stage 13.0 (TID 271) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:49:36 INFO Executor: Running task 1.0 in stage 13.0 (TID 271) -26/04/01 08:49:36 INFO TaskSetManager: Finished task 30.0 in stage 12.0 (TID 263) in 3304 ms on 10.0.0.133 (executor driver) (31/37) -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO Executor: Finished task 31.0 in stage 12.0 (TID 264). 6563 bytes result sent to driver -26/04/01 08:49:36 INFO TaskSetManager: Starting task 2.0 in stage 13.0 (TID 272) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:36 INFO Executor: Running task 2.0 in stage 13.0 (TID 272) -26/04/01 08:49:36 INFO TaskSetManager: Finished task 31.0 in stage 12.0 (TID 264) in 3306 ms on 10.0.0.133 (executor driver) (32/37) -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:38 INFO Executor: Finished task 36.0 in stage 12.0 (TID 269). 6606 bytes result sent to driver -26/04/01 08:49:38 INFO TaskSetManager: Starting task 3.0 in stage 13.0 (TID 273) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:38 INFO TaskSetManager: Finished task 36.0 in stage 12.0 (TID 269) in 1154 ms on 10.0.0.133 (executor driver) (33/37) -26/04/01 08:49:38 INFO Executor: Running task 3.0 in stage 13.0 (TID 273) -26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:38 INFO Executor: Finished task 32.0 in stage 12.0 (TID 265). 6606 bytes result sent to driver -26/04/01 08:49:38 INFO TaskSetManager: Starting task 4.0 in stage 13.0 (TID 274) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:38 INFO Executor: Running task 4.0 in stage 13.0 (TID 274) -26/04/01 08:49:38 INFO TaskSetManager: Finished task 32.0 in stage 12.0 (TID 265) in 2024 ms on 10.0.0.133 (executor driver) (34/37) -26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:38 INFO Executor: Finished task 33.0 in stage 12.0 (TID 266). 6606 bytes result sent to driver -26/04/01 08:49:38 INFO TaskSetManager: Starting task 5.0 in stage 13.0 (TID 275) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:49:38 INFO TaskSetManager: Finished task 33.0 in stage 12.0 (TID 266) in 2026 ms on 10.0.0.133 (executor driver) (35/37) -26/04/01 08:49:38 INFO Executor: Running task 5.0 in stage 13.0 (TID 275) -26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:38 INFO Executor: Finished task 35.0 in stage 12.0 (TID 268). 6606 bytes result sent to driver -26/04/01 08:49:38 INFO TaskSetManager: Starting task 6.0 in stage 13.0 (TID 276) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:38 INFO Executor: Running task 6.0 in stage 13.0 (TID 276) -26/04/01 08:49:38 INFO TaskSetManager: Finished task 35.0 in stage 12.0 (TID 268) in 2027 ms on 10.0.0.133 (executor driver) (36/37) -26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:38 INFO Executor: Finished task 34.0 in stage 12.0 (TID 267). 6606 bytes result sent to driver -26/04/01 08:49:38 INFO TaskSetManager: Starting task 7.0 in stage 13.0 (TID 277) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:38 INFO TaskSetManager: Finished task 34.0 in stage 12.0 (TID 267) in 2032 ms on 10.0.0.133 (executor driver) (37/37) -26/04/01 08:49:38 INFO Executor: Running task 7.0 in stage 13.0 (TID 277) -26/04/01 08:49:38 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool -26/04/01 08:49:38 INFO DAGScheduler: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 218.692 s -26/04/01 08:49:38 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:49:38 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 13) -26/04/01 08:49:38 INFO DAGScheduler: waiting: Set() -26/04/01 08:49:38 INFO DAGScheduler: failed: Set() -26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:39 INFO Executor: Finished task 0.0 in stage 13.0 (TID 270). 6606 bytes result sent to driver -26/04/01 08:49:39 INFO TaskSetManager: Starting task 8.0 in stage 13.0 (TID 278) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:49:39 INFO Executor: Running task 8.0 in stage 13.0 (TID 278) -26/04/01 08:49:39 INFO TaskSetManager: Finished task 0.0 in stage 13.0 (TID 270) in 2250 ms on 10.0.0.133 (executor driver) (1/64) -26/04/01 08:49:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:39 INFO Executor: Finished task 1.0 in stage 13.0 (TID 271). 6606 bytes result sent to driver -26/04/01 08:49:39 INFO TaskSetManager: Starting task 9.0 in stage 13.0 (TID 279) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:39 INFO TaskSetManager: Finished task 1.0 in stage 13.0 (TID 271) in 2233 ms on 10.0.0.133 (executor driver) (2/64) -26/04/01 08:49:39 INFO Executor: Running task 9.0 in stage 13.0 (TID 279) -26/04/01 08:49:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:39 INFO Executor: Finished task 2.0 in stage 13.0 (TID 272). 6606 bytes result sent to driver -26/04/01 08:49:39 INFO TaskSetManager: Starting task 10.0 in stage 13.0 (TID 280) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:49:39 INFO TaskSetManager: Finished task 2.0 in stage 13.0 (TID 272) in 2238 ms on 10.0.0.133 (executor driver) (3/64) -26/04/01 08:49:39 INFO Executor: Running task 10.0 in stage 13.0 (TID 280) -26/04/01 08:49:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:39 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:40 INFO Executor: Finished task 3.0 in stage 13.0 (TID 273). 6563 bytes result sent to driver -26/04/01 08:49:40 INFO TaskSetManager: Starting task 11.0 in stage 13.0 (TID 281) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:40 INFO Executor: Running task 11.0 in stage 13.0 (TID 281) -26/04/01 08:49:40 INFO TaskSetManager: Finished task 3.0 in stage 13.0 (TID 273) in 2214 ms on 10.0.0.133 (executor driver) (4/64) -26/04/01 08:49:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:40 INFO Executor: Finished task 4.0 in stage 13.0 (TID 274). 6563 bytes result sent to driver -26/04/01 08:49:40 INFO TaskSetManager: Starting task 12.0 in stage 13.0 (TID 282) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:40 INFO Executor: Running task 12.0 in stage 13.0 (TID 282) -26/04/01 08:49:40 INFO TaskSetManager: Finished task 4.0 in stage 13.0 (TID 274) in 2240 ms on 10.0.0.133 (executor driver) (5/64) -26/04/01 08:49:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:41 INFO Executor: Finished task 5.0 in stage 13.0 (TID 275). 6563 bytes result sent to driver -26/04/01 08:49:41 INFO TaskSetManager: Starting task 13.0 in stage 13.0 (TID 283) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:41 INFO Executor: Running task 13.0 in stage 13.0 (TID 283) -26/04/01 08:49:41 INFO TaskSetManager: Finished task 5.0 in stage 13.0 (TID 275) in 2243 ms on 10.0.0.133 (executor driver) (6/64) -26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:41 INFO Executor: Finished task 6.0 in stage 13.0 (TID 276). 6563 bytes result sent to driver -26/04/01 08:49:41 INFO TaskSetManager: Starting task 14.0 in stage 13.0 (TID 284) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:41 INFO TaskSetManager: Finished task 6.0 in stage 13.0 (TID 276) in 2246 ms on 10.0.0.133 (executor driver) (7/64) -26/04/01 08:49:41 INFO Executor: Running task 14.0 in stage 13.0 (TID 284) -26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:41 INFO Executor: Finished task 7.0 in stage 13.0 (TID 277). 6563 bytes result sent to driver -26/04/01 08:49:41 INFO TaskSetManager: Starting task 15.0 in stage 13.0 (TID 285) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:41 INFO TaskSetManager: Finished task 7.0 in stage 13.0 (TID 277) in 2253 ms on 10.0.0.133 (executor driver) (8/64) -26/04/01 08:49:41 INFO Executor: Running task 15.0 in stage 13.0 (TID 285) -26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:41 INFO Executor: Finished task 8.0 in stage 13.0 (TID 278). 6606 bytes result sent to driver -26/04/01 08:49:41 INFO TaskSetManager: Starting task 16.0 in stage 13.0 (TID 286) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:49:41 INFO TaskSetManager: Finished task 8.0 in stage 13.0 (TID 278) in 2245 ms on 10.0.0.133 (executor driver) (9/64) -26/04/01 08:49:41 INFO Executor: Running task 16.0 in stage 13.0 (TID 286) -26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:41 INFO Executor: Finished task 9.0 in stage 13.0 (TID 279). 6606 bytes result sent to driver -26/04/01 08:49:41 INFO TaskSetManager: Starting task 17.0 in stage 13.0 (TID 287) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:41 INFO TaskSetManager: Finished task 9.0 in stage 13.0 (TID 279) in 2246 ms on 10.0.0.133 (executor driver) (10/64) -26/04/01 08:49:41 INFO Executor: Running task 17.0 in stage 13.0 (TID 287) -26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:41 INFO Executor: Finished task 10.0 in stage 13.0 (TID 280). 6606 bytes result sent to driver -26/04/01 08:49:41 INFO TaskSetManager: Starting task 18.0 in stage 13.0 (TID 288) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:49:41 INFO TaskSetManager: Finished task 10.0 in stage 13.0 (TID 280) in 2236 ms on 10.0.0.133 (executor driver) (11/64) -26/04/01 08:49:41 INFO Executor: Running task 18.0 in stage 13.0 (TID 288) -26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:41 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:42 INFO Executor: Finished task 11.0 in stage 13.0 (TID 281). 6606 bytes result sent to driver -26/04/01 08:49:42 INFO TaskSetManager: Starting task 19.0 in stage 13.0 (TID 289) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:42 INFO Executor: Running task 19.0 in stage 13.0 (TID 289) -26/04/01 08:49:42 INFO TaskSetManager: Finished task 11.0 in stage 13.0 (TID 281) in 2220 ms on 10.0.0.133 (executor driver) (12/64) -26/04/01 08:49:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:42 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:43 INFO Executor: Finished task 12.0 in stage 13.0 (TID 282). 6606 bytes result sent to driver -26/04/01 08:49:43 INFO TaskSetManager: Starting task 20.0 in stage 13.0 (TID 290) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:43 INFO TaskSetManager: Finished task 12.0 in stage 13.0 (TID 282) in 2252 ms on 10.0.0.133 (executor driver) (13/64) -26/04/01 08:49:43 INFO Executor: Running task 20.0 in stage 13.0 (TID 290) -26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:43 INFO Executor: Finished task 13.0 in stage 13.0 (TID 283). 6606 bytes result sent to driver -26/04/01 08:49:43 INFO TaskSetManager: Starting task 21.0 in stage 13.0 (TID 291) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:49:43 INFO TaskSetManager: Finished task 13.0 in stage 13.0 (TID 283) in 2250 ms on 10.0.0.133 (executor driver) (14/64) -26/04/01 08:49:43 INFO Executor: Running task 21.0 in stage 13.0 (TID 291) -26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:43 INFO Executor: Finished task 14.0 in stage 13.0 (TID 284). 6606 bytes result sent to driver -26/04/01 08:49:43 INFO TaskSetManager: Starting task 22.0 in stage 13.0 (TID 292) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:43 INFO TaskSetManager: Finished task 14.0 in stage 13.0 (TID 284) in 2261 ms on 10.0.0.133 (executor driver) (15/64) -26/04/01 08:49:43 INFO Executor: Running task 22.0 in stage 13.0 (TID 292) -26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:43 INFO Executor: Finished task 15.0 in stage 13.0 (TID 285). 6606 bytes result sent to driver -26/04/01 08:49:43 INFO TaskSetManager: Starting task 23.0 in stage 13.0 (TID 293) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:43 INFO Executor: Running task 23.0 in stage 13.0 (TID 293) -26/04/01 08:49:43 INFO TaskSetManager: Finished task 15.0 in stage 13.0 (TID 285) in 2264 ms on 10.0.0.133 (executor driver) (16/64) -26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:43 INFO Executor: Finished task 16.0 in stage 13.0 (TID 286). 6606 bytes result sent to driver -26/04/01 08:49:43 INFO TaskSetManager: Starting task 24.0 in stage 13.0 (TID 294) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:49:43 INFO TaskSetManager: Finished task 16.0 in stage 13.0 (TID 286) in 2258 ms on 10.0.0.133 (executor driver) (17/64) -26/04/01 08:49:43 INFO Executor: Running task 24.0 in stage 13.0 (TID 294) -26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:43 INFO Executor: Finished task 17.0 in stage 13.0 (TID 287). 6563 bytes result sent to driver -26/04/01 08:49:43 INFO TaskSetManager: Starting task 25.0 in stage 13.0 (TID 295) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:43 INFO TaskSetManager: Finished task 17.0 in stage 13.0 (TID 287) in 2265 ms on 10.0.0.133 (executor driver) (18/64) -26/04/01 08:49:43 INFO Executor: Running task 25.0 in stage 13.0 (TID 295) -26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:43 INFO Executor: Finished task 18.0 in stage 13.0 (TID 288). 6563 bytes result sent to driver -26/04/01 08:49:43 INFO TaskSetManager: Starting task 26.0 in stage 13.0 (TID 296) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:43 INFO TaskSetManager: Finished task 18.0 in stage 13.0 (TID 288) in 2279 ms on 10.0.0.133 (executor driver) (19/64) -26/04/01 08:49:43 INFO Executor: Running task 26.0 in stage 13.0 (TID 296) -26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:43 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:44 INFO Executor: Finished task 19.0 in stage 13.0 (TID 289). 6563 bytes result sent to driver -26/04/01 08:49:44 INFO TaskSetManager: Starting task 27.0 in stage 13.0 (TID 297) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:44 INFO TaskSetManager: Finished task 19.0 in stage 13.0 (TID 289) in 2234 ms on 10.0.0.133 (executor driver) (20/64) -26/04/01 08:49:44 INFO Executor: Running task 27.0 in stage 13.0 (TID 297) -26/04/01 08:49:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:44 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:45 INFO Executor: Finished task 20.0 in stage 13.0 (TID 290). 6563 bytes result sent to driver -26/04/01 08:49:45 INFO TaskSetManager: Starting task 28.0 in stage 13.0 (TID 298) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:45 INFO Executor: Running task 28.0 in stage 13.0 (TID 298) -26/04/01 08:49:45 INFO TaskSetManager: Finished task 20.0 in stage 13.0 (TID 290) in 2258 ms on 10.0.0.133 (executor driver) (21/64) -26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:45 INFO Executor: Finished task 21.0 in stage 13.0 (TID 291). 6563 bytes result sent to driver -26/04/01 08:49:45 INFO TaskSetManager: Starting task 29.0 in stage 13.0 (TID 299) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:45 INFO TaskSetManager: Finished task 21.0 in stage 13.0 (TID 291) in 2247 ms on 10.0.0.133 (executor driver) (22/64) -26/04/01 08:49:45 INFO Executor: Running task 29.0 in stage 13.0 (TID 299) -26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:45 INFO Executor: Finished task 22.0 in stage 13.0 (TID 292). 6563 bytes result sent to driver -26/04/01 08:49:45 INFO TaskSetManager: Starting task 30.0 in stage 13.0 (TID 300) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9264 bytes) -26/04/01 08:49:45 INFO TaskSetManager: Finished task 22.0 in stage 13.0 (TID 292) in 2260 ms on 10.0.0.133 (executor driver) (23/64) -26/04/01 08:49:45 INFO Executor: Running task 30.0 in stage 13.0 (TID 300) -26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:45 INFO Executor: Finished task 23.0 in stage 13.0 (TID 293). 6563 bytes result sent to driver -26/04/01 08:49:45 INFO TaskSetManager: Starting task 31.0 in stage 13.0 (TID 301) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9263 bytes) -26/04/01 08:49:45 INFO TaskSetManager: Finished task 23.0 in stage 13.0 (TID 293) in 2256 ms on 10.0.0.133 (executor driver) (24/64) -26/04/01 08:49:45 INFO Executor: Running task 31.0 in stage 13.0 (TID 301) -26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:45 INFO Executor: Finished task 24.0 in stage 13.0 (TID 294). 6606 bytes result sent to driver -26/04/01 08:49:45 INFO TaskSetManager: Starting task 32.0 in stage 13.0 (TID 302) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:45 INFO TaskSetManager: Finished task 24.0 in stage 13.0 (TID 294) in 2250 ms on 10.0.0.133 (executor driver) (25/64) -26/04/01 08:49:45 INFO Executor: Running task 32.0 in stage 13.0 (TID 302) -26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:45 INFO Executor: Finished task 25.0 in stage 13.0 (TID 295). 6606 bytes result sent to driver -26/04/01 08:49:45 INFO TaskSetManager: Starting task 33.0 in stage 13.0 (TID 303) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:45 INFO TaskSetManager: Finished task 25.0 in stage 13.0 (TID 295) in 2255 ms on 10.0.0.133 (executor driver) (26/64) -26/04/01 08:49:45 INFO Executor: Running task 33.0 in stage 13.0 (TID 303) -26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:45 INFO Executor: Finished task 26.0 in stage 13.0 (TID 296). 6606 bytes result sent to driver -26/04/01 08:49:45 INFO TaskSetManager: Starting task 34.0 in stage 13.0 (TID 304) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:45 INFO TaskSetManager: Finished task 26.0 in stage 13.0 (TID 296) in 2251 ms on 10.0.0.133 (executor driver) (27/64) -26/04/01 08:49:45 INFO Executor: Running task 34.0 in stage 13.0 (TID 304) -26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:46 INFO Executor: Finished task 27.0 in stage 13.0 (TID 297). 6606 bytes result sent to driver -26/04/01 08:49:46 INFO TaskSetManager: Starting task 35.0 in stage 13.0 (TID 305) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:46 INFO TaskSetManager: Finished task 27.0 in stage 13.0 (TID 297) in 2226 ms on 10.0.0.133 (executor driver) (28/64) -26/04/01 08:49:46 INFO Executor: Running task 35.0 in stage 13.0 (TID 305) -26/04/01 08:49:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:46 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:47 INFO Executor: Finished task 32.0 in stage 13.0 (TID 302). 6563 bytes result sent to driver -26/04/01 08:49:47 INFO TaskSetManager: Starting task 36.0 in stage 13.0 (TID 306) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:47 INFO TaskSetManager: Finished task 32.0 in stage 13.0 (TID 302) in 1587 ms on 10.0.0.133 (executor driver) (29/64) -26/04/01 08:49:47 INFO Executor: Running task 36.0 in stage 13.0 (TID 306) -26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:47 INFO Executor: Finished task 33.0 in stage 13.0 (TID 303). 6563 bytes result sent to driver -26/04/01 08:49:47 INFO TaskSetManager: Starting task 37.0 in stage 13.0 (TID 307) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:47 INFO TaskSetManager: Finished task 33.0 in stage 13.0 (TID 303) in 1575 ms on 10.0.0.133 (executor driver) (30/64) -26/04/01 08:49:47 INFO Executor: Running task 37.0 in stage 13.0 (TID 307) -26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:47 INFO Executor: Finished task 34.0 in stage 13.0 (TID 304). 6563 bytes result sent to driver -26/04/01 08:49:47 INFO TaskSetManager: Starting task 38.0 in stage 13.0 (TID 308) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:47 INFO TaskSetManager: Finished task 34.0 in stage 13.0 (TID 304) in 1605 ms on 10.0.0.133 (executor driver) (31/64) -26/04/01 08:49:47 INFO Executor: Running task 38.0 in stage 13.0 (TID 308) -26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:47 INFO Executor: Finished task 28.0 in stage 13.0 (TID 298). 6606 bytes result sent to driver -26/04/01 08:49:47 INFO TaskSetManager: Starting task 39.0 in stage 13.0 (TID 309) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:47 INFO Executor: Running task 39.0 in stage 13.0 (TID 309) -26/04/01 08:49:47 INFO TaskSetManager: Finished task 28.0 in stage 13.0 (TID 298) in 2252 ms on 10.0.0.133 (executor driver) (32/64) -26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:47 INFO Executor: Finished task 29.0 in stage 13.0 (TID 299). 6606 bytes result sent to driver -26/04/01 08:49:47 INFO TaskSetManager: Starting task 40.0 in stage 13.0 (TID 310) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:47 INFO TaskSetManager: Finished task 29.0 in stage 13.0 (TID 299) in 2265 ms on 10.0.0.133 (executor driver) (33/64) -26/04/01 08:49:47 INFO Executor: Running task 40.0 in stage 13.0 (TID 310) -26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:47 INFO Executor: Finished task 30.0 in stage 13.0 (TID 300). 6606 bytes result sent to driver -26/04/01 08:49:47 INFO TaskSetManager: Starting task 41.0 in stage 13.0 (TID 311) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:47 INFO TaskSetManager: Finished task 30.0 in stage 13.0 (TID 300) in 2268 ms on 10.0.0.133 (executor driver) (34/64) -26/04/01 08:49:47 INFO Executor: Running task 41.0 in stage 13.0 (TID 311) -26/04/01 08:49:47 INFO Executor: Finished task 31.0 in stage 13.0 (TID 301). 6606 bytes result sent to driver -26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:47 INFO TaskSetManager: Starting task 42.0 in stage 13.0 (TID 312) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:47 INFO TaskSetManager: Finished task 31.0 in stage 13.0 (TID 301) in 2263 ms on 10.0.0.133 (executor driver) (35/64) -26/04/01 08:49:47 INFO Executor: Running task 42.0 in stage 13.0 (TID 312) -26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:47 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:48 INFO Executor: Finished task 35.0 in stage 13.0 (TID 305). 6563 bytes result sent to driver -26/04/01 08:49:48 INFO TaskSetManager: Starting task 43.0 in stage 13.0 (TID 313) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:48 INFO TaskSetManager: Finished task 35.0 in stage 13.0 (TID 305) in 1575 ms on 10.0.0.133 (executor driver) (36/64) -26/04/01 08:49:48 INFO Executor: Running task 43.0 in stage 13.0 (TID 313) -26/04/01 08:49:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:48 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:49 INFO Executor: Finished task 36.0 in stage 13.0 (TID 306). 6563 bytes result sent to driver -26/04/01 08:49:49 INFO TaskSetManager: Starting task 44.0 in stage 13.0 (TID 314) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:49 INFO TaskSetManager: Finished task 36.0 in stage 13.0 (TID 306) in 1569 ms on 10.0.0.133 (executor driver) (37/64) -26/04/01 08:49:49 INFO Executor: Running task 44.0 in stage 13.0 (TID 314) -26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:49 INFO Executor: Finished task 37.0 in stage 13.0 (TID 307). 6563 bytes result sent to driver -26/04/01 08:49:49 INFO TaskSetManager: Starting task 45.0 in stage 13.0 (TID 315) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:49 INFO Executor: Running task 45.0 in stage 13.0 (TID 315) -26/04/01 08:49:49 INFO TaskSetManager: Finished task 37.0 in stage 13.0 (TID 307) in 1562 ms on 10.0.0.133 (executor driver) (38/64) -26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:49 INFO Executor: Finished task 38.0 in stage 13.0 (TID 308). 6563 bytes result sent to driver -26/04/01 08:49:49 INFO TaskSetManager: Starting task 46.0 in stage 13.0 (TID 316) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:49 INFO Executor: Running task 46.0 in stage 13.0 (TID 316) -26/04/01 08:49:49 INFO TaskSetManager: Finished task 38.0 in stage 13.0 (TID 308) in 1564 ms on 10.0.0.133 (executor driver) (39/64) -26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:49 INFO Executor: Finished task 39.0 in stage 13.0 (TID 309). 6563 bytes result sent to driver -26/04/01 08:49:49 INFO TaskSetManager: Starting task 47.0 in stage 13.0 (TID 317) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:49 INFO TaskSetManager: Finished task 39.0 in stage 13.0 (TID 309) in 1571 ms on 10.0.0.133 (executor driver) (40/64) -26/04/01 08:49:49 INFO Executor: Running task 47.0 in stage 13.0 (TID 317) -26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:49 INFO Executor: Finished task 40.0 in stage 13.0 (TID 310). 6563 bytes result sent to driver -26/04/01 08:49:49 INFO TaskSetManager: Starting task 48.0 in stage 13.0 (TID 318) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:49 INFO TaskSetManager: Finished task 40.0 in stage 13.0 (TID 310) in 1570 ms on 10.0.0.133 (executor driver) (41/64) -26/04/01 08:49:49 INFO Executor: Running task 48.0 in stage 13.0 (TID 318) -26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:49 INFO Executor: Finished task 41.0 in stage 13.0 (TID 311). 6563 bytes result sent to driver -26/04/01 08:49:49 INFO TaskSetManager: Starting task 49.0 in stage 13.0 (TID 319) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:49 INFO TaskSetManager: Finished task 41.0 in stage 13.0 (TID 311) in 1570 ms on 10.0.0.133 (executor driver) (42/64) -26/04/01 08:49:49 INFO Executor: Running task 49.0 in stage 13.0 (TID 319) -26/04/01 08:49:49 INFO Executor: Finished task 42.0 in stage 13.0 (TID 312). 6563 bytes result sent to driver -26/04/01 08:49:49 INFO TaskSetManager: Starting task 50.0 in stage 13.0 (TID 320) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:49 INFO TaskSetManager: Finished task 42.0 in stage 13.0 (TID 312) in 1569 ms on 10.0.0.133 (executor driver) (43/64) -26/04/01 08:49:49 INFO Executor: Running task 50.0 in stage 13.0 (TID 320) -26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:49 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:50 INFO Executor: Finished task 43.0 in stage 13.0 (TID 313). 6563 bytes result sent to driver -26/04/01 08:49:50 INFO TaskSetManager: Starting task 51.0 in stage 13.0 (TID 321) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:50 INFO TaskSetManager: Finished task 43.0 in stage 13.0 (TID 313) in 1547 ms on 10.0.0.133 (executor driver) (44/64) -26/04/01 08:49:50 INFO Executor: Running task 51.0 in stage 13.0 (TID 321) -26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:50 INFO Executor: Finished task 44.0 in stage 13.0 (TID 314). 6563 bytes result sent to driver -26/04/01 08:49:50 INFO TaskSetManager: Starting task 52.0 in stage 13.0 (TID 322) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:50 INFO Executor: Running task 52.0 in stage 13.0 (TID 322) -26/04/01 08:49:50 INFO TaskSetManager: Finished task 44.0 in stage 13.0 (TID 314) in 1552 ms on 10.0.0.133 (executor driver) (45/64) -26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:50 INFO Executor: Finished task 45.0 in stage 13.0 (TID 315). 6563 bytes result sent to driver -26/04/01 08:49:50 INFO TaskSetManager: Starting task 53.0 in stage 13.0 (TID 323) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:50 INFO TaskSetManager: Finished task 45.0 in stage 13.0 (TID 315) in 1560 ms on 10.0.0.133 (executor driver) (46/64) -26/04/01 08:49:50 INFO Executor: Running task 53.0 in stage 13.0 (TID 323) -26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:50 INFO Executor: Finished task 46.0 in stage 13.0 (TID 316). 6563 bytes result sent to driver -26/04/01 08:49:50 INFO TaskSetManager: Starting task 54.0 in stage 13.0 (TID 324) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:50 INFO TaskSetManager: Finished task 46.0 in stage 13.0 (TID 316) in 1556 ms on 10.0.0.133 (executor driver) (47/64) -26/04/01 08:49:50 INFO Executor: Running task 54.0 in stage 13.0 (TID 324) -26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:50 INFO Executor: Finished task 47.0 in stage 13.0 (TID 317). 6563 bytes result sent to driver -26/04/01 08:49:50 INFO TaskSetManager: Starting task 55.0 in stage 13.0 (TID 325) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:50 INFO TaskSetManager: Finished task 47.0 in stage 13.0 (TID 317) in 1564 ms on 10.0.0.133 (executor driver) (48/64) -26/04/01 08:49:50 INFO Executor: Running task 55.0 in stage 13.0 (TID 325) -26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:50 INFO Executor: Finished task 48.0 in stage 13.0 (TID 318). 6563 bytes result sent to driver -26/04/01 08:49:50 INFO TaskSetManager: Starting task 56.0 in stage 13.0 (TID 326) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:50 INFO TaskSetManager: Finished task 48.0 in stage 13.0 (TID 318) in 1562 ms on 10.0.0.133 (executor driver) (49/64) -26/04/01 08:49:50 INFO Executor: Running task 56.0 in stage 13.0 (TID 326) -26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:51 INFO Executor: Finished task 49.0 in stage 13.0 (TID 319). 6563 bytes result sent to driver -26/04/01 08:49:51 INFO TaskSetManager: Starting task 57.0 in stage 13.0 (TID 327) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:51 INFO TaskSetManager: Finished task 49.0 in stage 13.0 (TID 319) in 1564 ms on 10.0.0.133 (executor driver) (50/64) -26/04/01 08:49:51 INFO Executor: Running task 57.0 in stage 13.0 (TID 327) -26/04/01 08:49:51 INFO Executor: Finished task 50.0 in stage 13.0 (TID 320). 6563 bytes result sent to driver -26/04/01 08:49:51 INFO TaskSetManager: Starting task 58.0 in stage 13.0 (TID 328) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:51 INFO TaskSetManager: Finished task 50.0 in stage 13.0 (TID 320) in 1564 ms on 10.0.0.133 (executor driver) (51/64) -26/04/01 08:49:51 INFO Executor: Running task 58.0 in stage 13.0 (TID 328) -26/04/01 08:49:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:51 INFO Executor: Finished task 51.0 in stage 13.0 (TID 321). 6563 bytes result sent to driver -26/04/01 08:49:51 INFO TaskSetManager: Starting task 59.0 in stage 13.0 (TID 329) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:51 INFO TaskSetManager: Finished task 51.0 in stage 13.0 (TID 321) in 1541 ms on 10.0.0.133 (executor driver) (52/64) -26/04/01 08:49:51 INFO Executor: Running task 59.0 in stage 13.0 (TID 329) -26/04/01 08:49:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:51 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:52 INFO Executor: Finished task 52.0 in stage 13.0 (TID 322). 6563 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 60.0 in stage 13.0 (TID 330) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 52.0 in stage 13.0 (TID 322) in 1556 ms on 10.0.0.133 (executor driver) (53/64) -26/04/01 08:49:52 INFO Executor: Running task 60.0 in stage 13.0 (TID 330) -26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:52 INFO Executor: Finished task 53.0 in stage 13.0 (TID 323). 6563 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 61.0 in stage 13.0 (TID 331) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9269 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 53.0 in stage 13.0 (TID 323) in 1567 ms on 10.0.0.133 (executor driver) (54/64) -26/04/01 08:49:52 INFO Executor: Running task 61.0 in stage 13.0 (TID 331) -26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:52 INFO Executor: Finished task 54.0 in stage 13.0 (TID 324). 6563 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 62.0 in stage 13.0 (TID 332) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 54.0 in stage 13.0 (TID 324) in 1570 ms on 10.0.0.133 (executor driver) (55/64) -26/04/01 08:49:52 INFO Executor: Running task 62.0 in stage 13.0 (TID 332) -26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:52 INFO Executor: Finished task 55.0 in stage 13.0 (TID 325). 6563 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 63.0 in stage 13.0 (TID 333) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9268 bytes) -26/04/01 08:49:52 INFO Executor: Running task 63.0 in stage 13.0 (TID 333) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 55.0 in stage 13.0 (TID 325) in 1563 ms on 10.0.0.133 (executor driver) (56/64) -26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:52 INFO Executor: Finished task 56.0 in stage 13.0 (TID 326). 6563 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 334) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 0.0 in stage 15.0 (TID 334) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 56.0 in stage 13.0 (TID 326) in 1567 ms on 10.0.0.133 (executor driver) (57/64) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:49:52 INFO Executor: Finished task 0.0 in stage 15.0 (TID 334). 33895 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 1.0 in stage 15.0 (TID 335) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 1.0 in stage 15.0 (TID 335) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 334) in 28 ms on 10.0.0.133 (executor driver) (1/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 1.0 in stage 15.0 (TID 335). 34070 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 2.0 in stage 15.0 (TID 336) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 2.0 in stage 15.0 (TID 336) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 1.0 in stage 15.0 (TID 335) in 7 ms on 10.0.0.133 (executor driver) (2/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 2.0 in stage 15.0 (TID 336). 33801 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 3.0 in stage 15.0 (TID 337) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 3.0 in stage 15.0 (TID 337) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 2.0 in stage 15.0 (TID 336) in 5 ms on 10.0.0.133 (executor driver) (3/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 3.0 in stage 15.0 (TID 337). 34328 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 4.0 in stage 15.0 (TID 338) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 4.0 in stage 15.0 (TID 338) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 3.0 in stage 15.0 (TID 337) in 5 ms on 10.0.0.133 (executor driver) (4/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 4.0 in stage 15.0 (TID 338). 34473 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 5.0 in stage 15.0 (TID 339) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 5.0 in stage 15.0 (TID 339) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 4.0 in stage 15.0 (TID 338) in 7 ms on 10.0.0.133 (executor driver) (5/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 5.0 in stage 15.0 (TID 339). 34156 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 6.0 in stage 15.0 (TID 340) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 6.0 in stage 15.0 (TID 340) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 5.0 in stage 15.0 (TID 339) in 4 ms on 10.0.0.133 (executor driver) (6/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 6.0 in stage 15.0 (TID 340). 33938 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 7.0 in stage 15.0 (TID 341) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 7.0 in stage 15.0 (TID 341) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 6.0 in stage 15.0 (TID 340) in 5 ms on 10.0.0.133 (executor driver) (7/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 7.0 in stage 15.0 (TID 341). 34062 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 8.0 in stage 15.0 (TID 342) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 8.0 in stage 15.0 (TID 342) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 7.0 in stage 15.0 (TID 341) in 5 ms on 10.0.0.133 (executor driver) (8/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 8.0 in stage 15.0 (TID 342). 33972 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 9.0 in stage 15.0 (TID 343) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 9.0 in stage 15.0 (TID 343) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 8.0 in stage 15.0 (TID 342) in 6 ms on 10.0.0.133 (executor driver) (9/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 9.0 in stage 15.0 (TID 343). 34979 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 10.0 in stage 15.0 (TID 344) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 10.0 in stage 15.0 (TID 344) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 9.0 in stage 15.0 (TID 343) in 6 ms on 10.0.0.133 (executor driver) (10/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 10.0 in stage 15.0 (TID 344). 34438 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 11.0 in stage 15.0 (TID 345) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 11.0 in stage 15.0 (TID 345) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 10.0 in stage 15.0 (TID 344) in 6 ms on 10.0.0.133 (executor driver) (11/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 11.0 in stage 15.0 (TID 345). 33620 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 12.0 in stage 15.0 (TID 346) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Finished task 57.0 in stage 13.0 (TID 327). 6606 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Running task 12.0 in stage 15.0 (TID 346) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 13.0 in stage 15.0 (TID 347) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 11.0 in stage 15.0 (TID 345) in 5 ms on 10.0.0.133 (executor driver) (12/200) -26/04/01 08:49:52 INFO Executor: Running task 13.0 in stage 15.0 (TID 347) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 57.0 in stage 13.0 (TID 327) in 1566 ms on 10.0.0.133 (executor driver) (58/64) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 58.0 in stage 13.0 (TID 328). 6606 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 14.0 in stage 15.0 (TID 348) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 14.0 in stage 15.0 (TID 348) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 58.0 in stage 13.0 (TID 328) in 1570 ms on 10.0.0.133 (executor driver) (59/64) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 12.0 in stage 15.0 (TID 346). 34084 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 15.0 in stage 15.0 (TID 349) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Finished task 13.0 in stage 15.0 (TID 347). 34520 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Running task 15.0 in stage 15.0 (TID 349) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 16.0 in stage 15.0 (TID 350) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 12.0 in stage 15.0 (TID 346) in 5 ms on 10.0.0.133 (executor driver) (13/200) -26/04/01 08:49:52 INFO Executor: Running task 16.0 in stage 15.0 (TID 350) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 13.0 in stage 15.0 (TID 347) in 5 ms on 10.0.0.133 (executor driver) (14/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 14.0 in stage 15.0 (TID 348). 34658 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 17.0 in stage 15.0 (TID 351) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 17.0 in stage 15.0 (TID 351) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 14.0 in stage 15.0 (TID 348) in 5 ms on 10.0.0.133 (executor driver) (15/200) -26/04/01 08:49:52 INFO Executor: Finished task 15.0 in stage 15.0 (TID 349). 34244 bytes result sent to driver -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO TaskSetManager: Starting task 18.0 in stage 15.0 (TID 352) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 18.0 in stage 15.0 (TID 352) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 15.0 in stage 15.0 (TID 349) in 5 ms on 10.0.0.133 (executor driver) (16/200) -26/04/01 08:49:52 INFO Executor: Finished task 16.0 in stage 15.0 (TID 350). 33628 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 19.0 in stage 15.0 (TID 353) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 19.0 in stage 15.0 (TID 353) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 16.0 in stage 15.0 (TID 350) in 6 ms on 10.0.0.133 (executor driver) (17/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 17.0 in stage 15.0 (TID 351). 34240 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 20.0 in stage 15.0 (TID 354) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 20.0 in stage 15.0 (TID 354) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 17.0 in stage 15.0 (TID 351) in 5 ms on 10.0.0.133 (executor driver) (18/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 18.0 in stage 15.0 (TID 352). 34217 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 19.0 in stage 15.0 (TID 353). 35085 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 21.0 in stage 15.0 (TID 355) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 21.0 in stage 15.0 (TID 355) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 22.0 in stage 15.0 (TID 356) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 18.0 in stage 15.0 (TID 352) in 6 ms on 10.0.0.133 (executor driver) (19/200) -26/04/01 08:49:52 INFO Executor: Running task 22.0 in stage 15.0 (TID 356) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 19.0 in stage 15.0 (TID 353) in 6 ms on 10.0.0.133 (executor driver) (20/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms -26/04/01 08:49:52 INFO Executor: Finished task 20.0 in stage 15.0 (TID 354). 34602 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 23.0 in stage 15.0 (TID 357) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 20.0 in stage 15.0 (TID 354) in 5 ms on 10.0.0.133 (executor driver) (21/200) -26/04/01 08:49:52 INFO Executor: Running task 23.0 in stage 15.0 (TID 357) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 22.0 in stage 15.0 (TID 356). 34615 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 21.0 in stage 15.0 (TID 355). 34867 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 24.0 in stage 15.0 (TID 358) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 24.0 in stage 15.0 (TID 358) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 25.0 in stage 15.0 (TID 359) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 22.0 in stage 15.0 (TID 356) in 5 ms on 10.0.0.133 (executor driver) (22/200) -26/04/01 08:49:52 INFO Executor: Running task 25.0 in stage 15.0 (TID 359) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 21.0 in stage 15.0 (TID 355) in 6 ms on 10.0.0.133 (executor driver) (23/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 23.0 in stage 15.0 (TID 357). 34082 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 26.0 in stage 15.0 (TID 360) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 26.0 in stage 15.0 (TID 360) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 23.0 in stage 15.0 (TID 357) in 6 ms on 10.0.0.133 (executor driver) (24/200) -26/04/01 08:49:52 INFO Executor: Finished task 25.0 in stage 15.0 (TID 359). 34085 bytes result sent to driver -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO TaskSetManager: Starting task 27.0 in stage 15.0 (TID 361) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 27.0 in stage 15.0 (TID 361) -26/04/01 08:49:52 INFO Executor: Finished task 24.0 in stage 15.0 (TID 358). 34683 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Finished task 25.0 in stage 15.0 (TID 359) in 5 ms on 10.0.0.133 (executor driver) (25/200) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 28.0 in stage 15.0 (TID 362) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 28.0 in stage 15.0 (TID 362) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 24.0 in stage 15.0 (TID 358) in 5 ms on 10.0.0.133 (executor driver) (26/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 26.0 in stage 15.0 (TID 360). 33825 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 29.0 in stage 15.0 (TID 363) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 29.0 in stage 15.0 (TID 363) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 26.0 in stage 15.0 (TID 360) in 4 ms on 10.0.0.133 (executor driver) (27/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 27.0 in stage 15.0 (TID 361). 34141 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 30.0 in stage 15.0 (TID 364) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Finished task 28.0 in stage 15.0 (TID 362). 34850 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Running task 30.0 in stage 15.0 (TID 364) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 31.0 in stage 15.0 (TID 365) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 27.0 in stage 15.0 (TID 361) in 5 ms on 10.0.0.133 (executor driver) (28/200) -26/04/01 08:49:52 INFO Executor: Running task 31.0 in stage 15.0 (TID 365) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 28.0 in stage 15.0 (TID 362) in 5 ms on 10.0.0.133 (executor driver) (29/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 29.0 in stage 15.0 (TID 363). 33967 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 32.0 in stage 15.0 (TID 366) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Finished task 30.0 in stage 15.0 (TID 364). 33979 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 31.0 in stage 15.0 (TID 365). 34381 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Running task 32.0 in stage 15.0 (TID 366) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 29.0 in stage 15.0 (TID 363) in 7 ms on 10.0.0.133 (executor driver) (30/200) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 33.0 in stage 15.0 (TID 367) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 33.0 in stage 15.0 (TID 367) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 34.0 in stage 15.0 (TID 368) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 30.0 in stage 15.0 (TID 364) in 6 ms on 10.0.0.133 (executor driver) (31/200) -26/04/01 08:49:52 INFO Executor: Running task 34.0 in stage 15.0 (TID 368) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 31.0 in stage 15.0 (TID 365) in 6 ms on 10.0.0.133 (executor driver) (32/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 32.0 in stage 15.0 (TID 366). 34649 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 34.0 in stage 15.0 (TID 368). 34290 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 33.0 in stage 15.0 (TID 367). 34079 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 35.0 in stage 15.0 (TID 369) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 35.0 in stage 15.0 (TID 369) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 36.0 in stage 15.0 (TID 370) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 37.0 in stage 15.0 (TID 371) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 36.0 in stage 15.0 (TID 370) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 34.0 in stage 15.0 (TID 368) in 6 ms on 10.0.0.133 (executor driver) (33/200) -26/04/01 08:49:52 INFO Executor: Running task 37.0 in stage 15.0 (TID 371) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 33.0 in stage 15.0 (TID 367) in 6 ms on 10.0.0.133 (executor driver) (34/200) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 32.0 in stage 15.0 (TID 366) in 6 ms on 10.0.0.133 (executor driver) (35/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 35.0 in stage 15.0 (TID 369). 34960 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 36.0 in stage 15.0 (TID 370). 34927 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 37.0 in stage 15.0 (TID 371). 34242 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 38.0 in stage 15.0 (TID 372) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 38.0 in stage 15.0 (TID 372) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 39.0 in stage 15.0 (TID 373) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 35.0 in stage 15.0 (TID 369) in 5 ms on 10.0.0.133 (executor driver) (36/200) -26/04/01 08:49:52 INFO Executor: Running task 39.0 in stage 15.0 (TID 373) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 40.0 in stage 15.0 (TID 374) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 37.0 in stage 15.0 (TID 371) in 5 ms on 10.0.0.133 (executor driver) (37/200) -26/04/01 08:49:52 INFO Executor: Running task 40.0 in stage 15.0 (TID 374) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 36.0 in stage 15.0 (TID 370) in 5 ms on 10.0.0.133 (executor driver) (38/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 38.0 in stage 15.0 (TID 372). 33723 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 39.0 in stage 15.0 (TID 373). 33734 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 41.0 in stage 15.0 (TID 375) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 41.0 in stage 15.0 (TID 375) -26/04/01 08:49:52 INFO Executor: Finished task 40.0 in stage 15.0 (TID 374). 34627 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 42.0 in stage 15.0 (TID 376) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 39.0 in stage 15.0 (TID 373) in 6 ms on 10.0.0.133 (executor driver) (39/200) -26/04/01 08:49:52 INFO Executor: Running task 42.0 in stage 15.0 (TID 376) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 43.0 in stage 15.0 (TID 377) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO TaskSetManager: Finished task 38.0 in stage 15.0 (TID 372) in 7 ms on 10.0.0.133 (executor driver) (40/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO TaskSetManager: Finished task 40.0 in stage 15.0 (TID 374) in 6 ms on 10.0.0.133 (executor driver) (41/200) -26/04/01 08:49:52 INFO Executor: Running task 43.0 in stage 15.0 (TID 377) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 41.0 in stage 15.0 (TID 375). 34443 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 44.0 in stage 15.0 (TID 378) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 41.0 in stage 15.0 (TID 375) in 4 ms on 10.0.0.133 (executor driver) (42/200) -26/04/01 08:49:52 INFO Executor: Running task 44.0 in stage 15.0 (TID 378) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 42.0 in stage 15.0 (TID 376). 34213 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 43.0 in stage 15.0 (TID 377). 33552 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 45.0 in stage 15.0 (TID 379) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 45.0 in stage 15.0 (TID 379) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 46.0 in stage 15.0 (TID 380) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 42.0 in stage 15.0 (TID 376) in 5 ms on 10.0.0.133 (executor driver) (43/200) -26/04/01 08:49:52 INFO Executor: Running task 46.0 in stage 15.0 (TID 380) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 43.0 in stage 15.0 (TID 377) in 5 ms on 10.0.0.133 (executor driver) (44/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 44.0 in stage 15.0 (TID 378). 33901 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 47.0 in stage 15.0 (TID 381) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 47.0 in stage 15.0 (TID 381) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 44.0 in stage 15.0 (TID 378) in 5 ms on 10.0.0.133 (executor driver) (45/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 45.0 in stage 15.0 (TID 379). 34036 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 48.0 in stage 15.0 (TID 382) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 48.0 in stage 15.0 (TID 382) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 45.0 in stage 15.0 (TID 379) in 5 ms on 10.0.0.133 (executor driver) (46/200) -26/04/01 08:49:52 INFO Executor: Finished task 46.0 in stage 15.0 (TID 380). 34072 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 49.0 in stage 15.0 (TID 383) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 49.0 in stage 15.0 (TID 383) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 46.0 in stage 15.0 (TID 380) in 5 ms on 10.0.0.133 (executor driver) (47/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 47.0 in stage 15.0 (TID 381). 34470 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 50.0 in stage 15.0 (TID 384) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 50.0 in stage 15.0 (TID 384) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 47.0 in stage 15.0 (TID 381) in 4 ms on 10.0.0.133 (executor driver) (48/200) -26/04/01 08:49:52 INFO Executor: Finished task 49.0 in stage 15.0 (TID 383). 33847 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 48.0 in stage 15.0 (TID 382). 33975 bytes result sent to driver -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO TaskSetManager: Starting task 51.0 in stage 15.0 (TID 385) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 51.0 in stage 15.0 (TID 385) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 52.0 in stage 15.0 (TID 386) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 52.0 in stage 15.0 (TID 386) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 49.0 in stage 15.0 (TID 383) in 6 ms on 10.0.0.133 (executor driver) (49/200) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 48.0 in stage 15.0 (TID 382) in 7 ms on 10.0.0.133 (executor driver) (50/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 50.0 in stage 15.0 (TID 384). 34504 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 53.0 in stage 15.0 (TID 387) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 53.0 in stage 15.0 (TID 387) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 50.0 in stage 15.0 (TID 384) in 7 ms on 10.0.0.133 (executor driver) (51/200) -26/04/01 08:49:52 INFO Executor: Finished task 52.0 in stage 15.0 (TID 386). 34119 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 51.0 in stage 15.0 (TID 385). 34243 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 54.0 in stage 15.0 (TID 388) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO Executor: Running task 54.0 in stage 15.0 (TID 388) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 55.0 in stage 15.0 (TID 389) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO TaskSetManager: Finished task 52.0 in stage 15.0 (TID 386) in 6 ms on 10.0.0.133 (executor driver) (52/200) -26/04/01 08:49:52 INFO Executor: Running task 55.0 in stage 15.0 (TID 389) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 51.0 in stage 15.0 (TID 385) in 6 ms on 10.0.0.133 (executor driver) (53/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 53.0 in stage 15.0 (TID 387). 34449 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 56.0 in stage 15.0 (TID 390) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 56.0 in stage 15.0 (TID 390) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 53.0 in stage 15.0 (TID 387) in 6 ms on 10.0.0.133 (executor driver) (54/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 54.0 in stage 15.0 (TID 388). 34702 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 55.0 in stage 15.0 (TID 389). 34244 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 57.0 in stage 15.0 (TID 391) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 57.0 in stage 15.0 (TID 391) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 58.0 in stage 15.0 (TID 392) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 54.0 in stage 15.0 (TID 388) in 6 ms on 10.0.0.133 (executor driver) (55/200) -26/04/01 08:49:52 INFO Executor: Running task 58.0 in stage 15.0 (TID 392) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 55.0 in stage 15.0 (TID 389) in 6 ms on 10.0.0.133 (executor driver) (56/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 56.0 in stage 15.0 (TID 390). 34901 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 59.0 in stage 15.0 (TID 393) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 59.0 in stage 15.0 (TID 393) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 56.0 in stage 15.0 (TID 390) in 6 ms on 10.0.0.133 (executor driver) (57/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 57.0 in stage 15.0 (TID 391). 34181 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 58.0 in stage 15.0 (TID 392). 34726 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 60.0 in stage 15.0 (TID 394) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 60.0 in stage 15.0 (TID 394) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 61.0 in stage 15.0 (TID 395) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 57.0 in stage 15.0 (TID 391) in 7 ms on 10.0.0.133 (executor driver) (58/200) -26/04/01 08:49:52 INFO Executor: Running task 61.0 in stage 15.0 (TID 395) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 58.0 in stage 15.0 (TID 392) in 6 ms on 10.0.0.133 (executor driver) (59/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 59.0 in stage 15.0 (TID 393). 34502 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 62.0 in stage 15.0 (TID 396) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 59.0 in stage 15.0 (TID 393) in 6 ms on 10.0.0.133 (executor driver) (60/200) -26/04/01 08:49:52 INFO Executor: Running task 62.0 in stage 15.0 (TID 396) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 61.0 in stage 15.0 (TID 395). 33977 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 63.0 in stage 15.0 (TID 397) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 61.0 in stage 15.0 (TID 395) in 6 ms on 10.0.0.133 (executor driver) (61/200) -26/04/01 08:49:52 INFO Executor: Finished task 60.0 in stage 15.0 (TID 394). 34580 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Running task 63.0 in stage 15.0 (TID 397) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 64.0 in stage 15.0 (TID 398) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 64.0 in stage 15.0 (TID 398) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 60.0 in stage 15.0 (TID 394) in 6 ms on 10.0.0.133 (executor driver) (62/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 62.0 in stage 15.0 (TID 396). 34073 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 65.0 in stage 15.0 (TID 399) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 65.0 in stage 15.0 (TID 399) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 62.0 in stage 15.0 (TID 396) in 5 ms on 10.0.0.133 (executor driver) (63/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 63.0 in stage 15.0 (TID 397). 33522 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 66.0 in stage 15.0 (TID 400) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 66.0 in stage 15.0 (TID 400) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 63.0 in stage 15.0 (TID 397) in 6 ms on 10.0.0.133 (executor driver) (64/200) -26/04/01 08:49:52 INFO Executor: Finished task 64.0 in stage 15.0 (TID 398). 33929 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 67.0 in stage 15.0 (TID 401) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 67.0 in stage 15.0 (TID 401) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 64.0 in stage 15.0 (TID 398) in 6 ms on 10.0.0.133 (executor driver) (65/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 65.0 in stage 15.0 (TID 399). 34066 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 68.0 in stage 15.0 (TID 402) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 65.0 in stage 15.0 (TID 399) in 6 ms on 10.0.0.133 (executor driver) (66/200) -26/04/01 08:49:52 INFO Executor: Running task 68.0 in stage 15.0 (TID 402) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 67.0 in stage 15.0 (TID 401). 34324 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 66.0 in stage 15.0 (TID 400). 34407 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 69.0 in stage 15.0 (TID 403) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 69.0 in stage 15.0 (TID 403) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 70.0 in stage 15.0 (TID 404) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 67.0 in stage 15.0 (TID 401) in 5 ms on 10.0.0.133 (executor driver) (67/200) -26/04/01 08:49:52 INFO Executor: Running task 70.0 in stage 15.0 (TID 404) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 66.0 in stage 15.0 (TID 400) in 5 ms on 10.0.0.133 (executor driver) (68/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 68.0 in stage 15.0 (TID 402). 33789 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 71.0 in stage 15.0 (TID 405) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 71.0 in stage 15.0 (TID 405) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 68.0 in stage 15.0 (TID 402) in 5 ms on 10.0.0.133 (executor driver) (69/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 70.0 in stage 15.0 (TID 404). 34912 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 72.0 in stage 15.0 (TID 406) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 72.0 in stage 15.0 (TID 406) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 70.0 in stage 15.0 (TID 404) in 5 ms on 10.0.0.133 (executor driver) (70/200) -26/04/01 08:49:52 INFO Executor: Finished task 69.0 in stage 15.0 (TID 403). 33810 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 73.0 in stage 15.0 (TID 407) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 73.0 in stage 15.0 (TID 407) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 69.0 in stage 15.0 (TID 403) in 5 ms on 10.0.0.133 (executor driver) (71/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 71.0 in stage 15.0 (TID 405). 34265 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 74.0 in stage 15.0 (TID 408) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 74.0 in stage 15.0 (TID 408) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 71.0 in stage 15.0 (TID 405) in 5 ms on 10.0.0.133 (executor driver) (72/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 72.0 in stage 15.0 (TID 406). 34272 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 75.0 in stage 15.0 (TID 409) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 75.0 in stage 15.0 (TID 409) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 72.0 in stage 15.0 (TID 406) in 6 ms on 10.0.0.133 (executor driver) (73/200) -26/04/01 08:49:52 INFO Executor: Finished task 73.0 in stage 15.0 (TID 407). 34476 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 76.0 in stage 15.0 (TID 410) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 76.0 in stage 15.0 (TID 410) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 73.0 in stage 15.0 (TID 407) in 5 ms on 10.0.0.133 (executor driver) (74/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 74.0 in stage 15.0 (TID 408). 33908 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 77.0 in stage 15.0 (TID 411) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 77.0 in stage 15.0 (TID 411) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 74.0 in stage 15.0 (TID 408) in 4 ms on 10.0.0.133 (executor driver) (75/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 75.0 in stage 15.0 (TID 409). 34266 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 78.0 in stage 15.0 (TID 412) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Finished task 76.0 in stage 15.0 (TID 410). 34196 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Running task 78.0 in stage 15.0 (TID 412) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 79.0 in stage 15.0 (TID 413) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 75.0 in stage 15.0 (TID 409) in 4 ms on 10.0.0.133 (executor driver) (76/200) -26/04/01 08:49:52 INFO Executor: Running task 79.0 in stage 15.0 (TID 413) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 76.0 in stage 15.0 (TID 410) in 4 ms on 10.0.0.133 (executor driver) (77/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 77.0 in stage 15.0 (TID 411). 34812 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 80.0 in stage 15.0 (TID 414) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 80.0 in stage 15.0 (TID 414) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 77.0 in stage 15.0 (TID 411) in 5 ms on 10.0.0.133 (executor driver) (78/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.6 KiB) non-empty blocks including 8 (31.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 78.0 in stage 15.0 (TID 412). 34141 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 81.0 in stage 15.0 (TID 415) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Finished task 79.0 in stage 15.0 (TID 413). 34085 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Running task 81.0 in stage 15.0 (TID 415) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 82.0 in stage 15.0 (TID 416) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 78.0 in stage 15.0 (TID 412) in 5 ms on 10.0.0.133 (executor driver) (79/200) -26/04/01 08:49:52 INFO Executor: Running task 82.0 in stage 15.0 (TID 416) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 79.0 in stage 15.0 (TID 413) in 5 ms on 10.0.0.133 (executor driver) (80/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 80.0 in stage 15.0 (TID 414). 34593 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 83.0 in stage 15.0 (TID 417) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 83.0 in stage 15.0 (TID 417) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 80.0 in stage 15.0 (TID 414) in 4 ms on 10.0.0.133 (executor driver) (81/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 81.0 in stage 15.0 (TID 415). 33935 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 84.0 in stage 15.0 (TID 418) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Finished task 82.0 in stage 15.0 (TID 416). 34466 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Running task 84.0 in stage 15.0 (TID 418) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 85.0 in stage 15.0 (TID 419) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 81.0 in stage 15.0 (TID 415) in 5 ms on 10.0.0.133 (executor driver) (82/200) -26/04/01 08:49:52 INFO Executor: Running task 85.0 in stage 15.0 (TID 419) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 82.0 in stage 15.0 (TID 416) in 4 ms on 10.0.0.133 (executor driver) (83/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 83.0 in stage 15.0 (TID 417). 34273 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 86.0 in stage 15.0 (TID 420) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 86.0 in stage 15.0 (TID 420) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 83.0 in stage 15.0 (TID 417) in 5 ms on 10.0.0.133 (executor driver) (84/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 84.0 in stage 15.0 (TID 418). 33511 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 85.0 in stage 15.0 (TID 419). 34506 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 87.0 in stage 15.0 (TID 421) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 87.0 in stage 15.0 (TID 421) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 88.0 in stage 15.0 (TID 422) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 84.0 in stage 15.0 (TID 418) in 5 ms on 10.0.0.133 (executor driver) (85/200) -26/04/01 08:49:52 INFO Executor: Running task 88.0 in stage 15.0 (TID 422) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 85.0 in stage 15.0 (TID 419) in 5 ms on 10.0.0.133 (executor driver) (86/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 86.0 in stage 15.0 (TID 420). 34439 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 89.0 in stage 15.0 (TID 423) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 89.0 in stage 15.0 (TID 423) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 86.0 in stage 15.0 (TID 420) in 6 ms on 10.0.0.133 (executor driver) (87/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO Executor: Finished task 88.0 in stage 15.0 (TID 422). 34788 bytes result sent to driver -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 87.0 in stage 15.0 (TID 421). 33774 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 90.0 in stage 15.0 (TID 424) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 90.0 in stage 15.0 (TID 424) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 91.0 in stage 15.0 (TID 425) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 88.0 in stage 15.0 (TID 422) in 5 ms on 10.0.0.133 (executor driver) (88/200) -26/04/01 08:49:52 INFO Executor: Running task 91.0 in stage 15.0 (TID 425) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 87.0 in stage 15.0 (TID 421) in 5 ms on 10.0.0.133 (executor driver) (89/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 89.0 in stage 15.0 (TID 423). 33884 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 92.0 in stage 15.0 (TID 426) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 92.0 in stage 15.0 (TID 426) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 89.0 in stage 15.0 (TID 423) in 5 ms on 10.0.0.133 (executor driver) (90/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 90.0 in stage 15.0 (TID 424). 33756 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 91.0 in stage 15.0 (TID 425). 33835 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 93.0 in stage 15.0 (TID 427) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 93.0 in stage 15.0 (TID 427) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 94.0 in stage 15.0 (TID 428) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 90.0 in stage 15.0 (TID 424) in 5 ms on 10.0.0.133 (executor driver) (91/200) -26/04/01 08:49:52 INFO Executor: Running task 94.0 in stage 15.0 (TID 428) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 91.0 in stage 15.0 (TID 425) in 5 ms on 10.0.0.133 (executor driver) (92/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 92.0 in stage 15.0 (TID 426). 33996 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 95.0 in stage 15.0 (TID 429) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 95.0 in stage 15.0 (TID 429) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 92.0 in stage 15.0 (TID 426) in 6 ms on 10.0.0.133 (executor driver) (93/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 93.0 in stage 15.0 (TID 427). 34038 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 94.0 in stage 15.0 (TID 428). 34594 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 96.0 in stage 15.0 (TID 430) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 96.0 in stage 15.0 (TID 430) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 97.0 in stage 15.0 (TID 431) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 93.0 in stage 15.0 (TID 427) in 6 ms on 10.0.0.133 (executor driver) (94/200) -26/04/01 08:49:52 INFO Executor: Running task 97.0 in stage 15.0 (TID 431) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 94.0 in stage 15.0 (TID 428) in 7 ms on 10.0.0.133 (executor driver) (95/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 95.0 in stage 15.0 (TID 429). 33891 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 98.0 in stage 15.0 (TID 432) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 98.0 in stage 15.0 (TID 432) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 95.0 in stage 15.0 (TID 429) in 4 ms on 10.0.0.133 (executor driver) (96/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 97.0 in stage 15.0 (TID 431). 33883 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 96.0 in stage 15.0 (TID 430). 34205 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 99.0 in stage 15.0 (TID 433) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 99.0 in stage 15.0 (TID 433) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 100.0 in stage 15.0 (TID 434) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 97.0 in stage 15.0 (TID 431) in 4 ms on 10.0.0.133 (executor driver) (97/200) -26/04/01 08:49:52 INFO Executor: Running task 100.0 in stage 15.0 (TID 434) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 96.0 in stage 15.0 (TID 430) in 5 ms on 10.0.0.133 (executor driver) (98/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 98.0 in stage 15.0 (TID 432). 34607 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 101.0 in stage 15.0 (TID 435) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 101.0 in stage 15.0 (TID 435) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 98.0 in stage 15.0 (TID 432) in 4 ms on 10.0.0.133 (executor driver) (99/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 99.0 in stage 15.0 (TID 433). 34229 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 100.0 in stage 15.0 (TID 434). 34078 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 102.0 in stage 15.0 (TID 436) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 102.0 in stage 15.0 (TID 436) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 103.0 in stage 15.0 (TID 437) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 99.0 in stage 15.0 (TID 433) in 5 ms on 10.0.0.133 (executor driver) (100/200) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 100.0 in stage 15.0 (TID 434) in 5 ms on 10.0.0.133 (executor driver) (101/200) -26/04/01 08:49:52 INFO Executor: Running task 103.0 in stage 15.0 (TID 437) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 101.0 in stage 15.0 (TID 435). 33625 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 104.0 in stage 15.0 (TID 438) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 104.0 in stage 15.0 (TID 438) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 101.0 in stage 15.0 (TID 435) in 4 ms on 10.0.0.133 (executor driver) (102/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 102.0 in stage 15.0 (TID 436). 34103 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 105.0 in stage 15.0 (TID 439) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 105.0 in stage 15.0 (TID 439) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 102.0 in stage 15.0 (TID 436) in 4 ms on 10.0.0.133 (executor driver) (103/200) -26/04/01 08:49:52 INFO Executor: Finished task 103.0 in stage 15.0 (TID 437). 34407 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 106.0 in stage 15.0 (TID 440) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 106.0 in stage 15.0 (TID 440) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 103.0 in stage 15.0 (TID 437) in 5 ms on 10.0.0.133 (executor driver) (104/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 104.0 in stage 15.0 (TID 438). 34155 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 107.0 in stage 15.0 (TID 441) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 107.0 in stage 15.0 (TID 441) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 104.0 in stage 15.0 (TID 438) in 4 ms on 10.0.0.133 (executor driver) (105/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 105.0 in stage 15.0 (TID 439). 33868 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 108.0 in stage 15.0 (TID 442) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 108.0 in stage 15.0 (TID 442) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 105.0 in stage 15.0 (TID 439) in 4 ms on 10.0.0.133 (executor driver) (106/200) -26/04/01 08:49:52 INFO Executor: Finished task 106.0 in stage 15.0 (TID 440). 33851 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 109.0 in stage 15.0 (TID 443) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 109.0 in stage 15.0 (TID 443) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 106.0 in stage 15.0 (TID 440) in 4 ms on 10.0.0.133 (executor driver) (107/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 107.0 in stage 15.0 (TID 441). 33898 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 110.0 in stage 15.0 (TID 444) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 110.0 in stage 15.0 (TID 444) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 107.0 in stage 15.0 (TID 441) in 4 ms on 10.0.0.133 (executor driver) (108/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 108.0 in stage 15.0 (TID 442). 34303 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 109.0 in stage 15.0 (TID 443). 34424 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 111.0 in stage 15.0 (TID 445) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 111.0 in stage 15.0 (TID 445) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 112.0 in stage 15.0 (TID 446) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 108.0 in stage 15.0 (TID 442) in 5 ms on 10.0.0.133 (executor driver) (109/200) -26/04/01 08:49:52 INFO Executor: Running task 112.0 in stage 15.0 (TID 446) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 109.0 in stage 15.0 (TID 443) in 4 ms on 10.0.0.133 (executor driver) (110/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 110.0 in stage 15.0 (TID 444). 34141 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 113.0 in stage 15.0 (TID 447) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 113.0 in stage 15.0 (TID 447) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 110.0 in stage 15.0 (TID 444) in 3 ms on 10.0.0.133 (executor driver) (111/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 112.0 in stage 15.0 (TID 446). 34551 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 111.0 in stage 15.0 (TID 445). 34432 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 114.0 in stage 15.0 (TID 448) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 114.0 in stage 15.0 (TID 448) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 115.0 in stage 15.0 (TID 449) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 112.0 in stage 15.0 (TID 446) in 3 ms on 10.0.0.133 (executor driver) (112/200) -26/04/01 08:49:52 INFO Executor: Running task 115.0 in stage 15.0 (TID 449) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 111.0 in stage 15.0 (TID 445) in 3 ms on 10.0.0.133 (executor driver) (113/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 113.0 in stage 15.0 (TID 447). 34235 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 116.0 in stage 15.0 (TID 450) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 116.0 in stage 15.0 (TID 450) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 113.0 in stage 15.0 (TID 447) in 4 ms on 10.0.0.133 (executor driver) (114/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 114.0 in stage 15.0 (TID 448). 34058 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 117.0 in stage 15.0 (TID 451) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 117.0 in stage 15.0 (TID 451) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 114.0 in stage 15.0 (TID 448) in 4 ms on 10.0.0.133 (executor driver) (115/200) -26/04/01 08:49:52 INFO Executor: Finished task 115.0 in stage 15.0 (TID 449). 33531 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 118.0 in stage 15.0 (TID 452) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 118.0 in stage 15.0 (TID 452) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO TaskSetManager: Finished task 115.0 in stage 15.0 (TID 449) in 5 ms on 10.0.0.133 (executor driver) (116/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 116.0 in stage 15.0 (TID 450). 34105 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 119.0 in stage 15.0 (TID 453) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO Executor: Running task 119.0 in stage 15.0 (TID 453) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO TaskSetManager: Finished task 116.0 in stage 15.0 (TID 450) in 4 ms on 10.0.0.133 (executor driver) (117/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 117.0 in stage 15.0 (TID 451). 34192 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 120.0 in stage 15.0 (TID 454) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 117.0 in stage 15.0 (TID 451) in 3 ms on 10.0.0.133 (executor driver) (118/200) -26/04/01 08:49:52 INFO Executor: Running task 120.0 in stage 15.0 (TID 454) -26/04/01 08:49:52 INFO Executor: Finished task 118.0 in stage 15.0 (TID 452). 34647 bytes result sent to driver -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO TaskSetManager: Starting task 121.0 in stage 15.0 (TID 455) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 121.0 in stage 15.0 (TID 455) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 118.0 in stage 15.0 (TID 452) in 4 ms on 10.0.0.133 (executor driver) (119/200) -26/04/01 08:49:52 INFO Executor: Finished task 119.0 in stage 15.0 (TID 453). 33642 bytes result sent to driver -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO TaskSetManager: Starting task 122.0 in stage 15.0 (TID 456) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 122.0 in stage 15.0 (TID 456) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 119.0 in stage 15.0 (TID 453) in 4 ms on 10.0.0.133 (executor driver) (120/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 120.0 in stage 15.0 (TID 454). 33991 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 123.0 in stage 15.0 (TID 457) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 123.0 in stage 15.0 (TID 457) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 120.0 in stage 15.0 (TID 454) in 4 ms on 10.0.0.133 (executor driver) (121/200) -26/04/01 08:49:52 INFO Executor: Finished task 121.0 in stage 15.0 (TID 455). 34107 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 124.0 in stage 15.0 (TID 458) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO Executor: Running task 124.0 in stage 15.0 (TID 458) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 121.0 in stage 15.0 (TID 455) in 4 ms on 10.0.0.133 (executor driver) (122/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 122.0 in stage 15.0 (TID 456). 34345 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 125.0 in stage 15.0 (TID 459) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 125.0 in stage 15.0 (TID 459) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 122.0 in stage 15.0 (TID 456) in 4 ms on 10.0.0.133 (executor driver) (123/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 123.0 in stage 15.0 (TID 457). 34123 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 124.0 in stage 15.0 (TID 458). 34152 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 126.0 in stage 15.0 (TID 460) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 126.0 in stage 15.0 (TID 460) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 127.0 in stage 15.0 (TID 461) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 123.0 in stage 15.0 (TID 457) in 5 ms on 10.0.0.133 (executor driver) (124/200) -26/04/01 08:49:52 INFO Executor: Running task 127.0 in stage 15.0 (TID 461) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 124.0 in stage 15.0 (TID 458) in 4 ms on 10.0.0.133 (executor driver) (125/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 125.0 in stage 15.0 (TID 459). 34554 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 128.0 in stage 15.0 (TID 462) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 128.0 in stage 15.0 (TID 462) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 125.0 in stage 15.0 (TID 459) in 4 ms on 10.0.0.133 (executor driver) (126/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 126.0 in stage 15.0 (TID 460). 33922 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 129.0 in stage 15.0 (TID 463) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 129.0 in stage 15.0 (TID 463) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 126.0 in stage 15.0 (TID 460) in 4 ms on 10.0.0.133 (executor driver) (127/200) -26/04/01 08:49:52 INFO Executor: Finished task 127.0 in stage 15.0 (TID 461). 34650 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 130.0 in stage 15.0 (TID 464) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 127.0 in stage 15.0 (TID 461) in 5 ms on 10.0.0.133 (executor driver) (128/200) -26/04/01 08:49:52 INFO Executor: Running task 130.0 in stage 15.0 (TID 464) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 128.0 in stage 15.0 (TID 462). 34349 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 131.0 in stage 15.0 (TID 465) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 131.0 in stage 15.0 (TID 465) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.6 KiB) non-empty blocks including 8 (31.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO TaskSetManager: Finished task 128.0 in stage 15.0 (TID 462) in 5 ms on 10.0.0.133 (executor driver) (129/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 129.0 in stage 15.0 (TID 463). 34012 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 132.0 in stage 15.0 (TID 466) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 132.0 in stage 15.0 (TID 466) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 129.0 in stage 15.0 (TID 463) in 4 ms on 10.0.0.133 (executor driver) (130/200) -26/04/01 08:49:52 INFO Executor: Finished task 130.0 in stage 15.0 (TID 464). 35090 bytes result sent to driver -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO TaskSetManager: Starting task 133.0 in stage 15.0 (TID 467) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Running task 133.0 in stage 15.0 (TID 467) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 130.0 in stage 15.0 (TID 464) in 4 ms on 10.0.0.133 (executor driver) (131/200) -26/04/01 08:49:52 INFO Executor: Finished task 131.0 in stage 15.0 (TID 465). 33973 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 134.0 in stage 15.0 (TID 468) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 134.0 in stage 15.0 (TID 468) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 131.0 in stage 15.0 (TID 465) in 5 ms on 10.0.0.133 (executor driver) (132/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 132.0 in stage 15.0 (TID 466). 34084 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 135.0 in stage 15.0 (TID 469) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 135.0 in stage 15.0 (TID 469) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 132.0 in stage 15.0 (TID 466) in 7 ms on 10.0.0.133 (executor driver) (133/200) -26/04/01 08:49:52 INFO Executor: Finished task 133.0 in stage 15.0 (TID 467). 33807 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 136.0 in stage 15.0 (TID 470) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 136.0 in stage 15.0 (TID 470) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 133.0 in stage 15.0 (TID 467) in 7 ms on 10.0.0.133 (executor driver) (134/200) -26/04/01 08:49:52 INFO Executor: Finished task 134.0 in stage 15.0 (TID 468). 33755 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 137.0 in stage 15.0 (TID 471) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Running task 137.0 in stage 15.0 (TID 471) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 134.0 in stage 15.0 (TID 468) in 7 ms on 10.0.0.133 (executor driver) (135/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 135.0 in stage 15.0 (TID 469). 34404 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 138.0 in stage 15.0 (TID 472) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 138.0 in stage 15.0 (TID 472) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 135.0 in stage 15.0 (TID 469) in 4 ms on 10.0.0.133 (executor driver) (136/200) -26/04/01 08:49:52 INFO Executor: Finished task 136.0 in stage 15.0 (TID 470). 34430 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 137.0 in stage 15.0 (TID 471). 34756 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 139.0 in stage 15.0 (TID 473) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Running task 139.0 in stage 15.0 (TID 473) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 140.0 in stage 15.0 (TID 474) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 136.0 in stage 15.0 (TID 470) in 4 ms on 10.0.0.133 (executor driver) (137/200) -26/04/01 08:49:52 INFO Executor: Running task 140.0 in stage 15.0 (TID 474) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 137.0 in stage 15.0 (TID 471) in 4 ms on 10.0.0.133 (executor driver) (138/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 138.0 in stage 15.0 (TID 472). 33693 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 141.0 in stage 15.0 (TID 475) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 141.0 in stage 15.0 (TID 475) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 138.0 in stage 15.0 (TID 472) in 4 ms on 10.0.0.133 (executor driver) (139/200) -26/04/01 08:49:52 INFO Executor: Finished task 140.0 in stage 15.0 (TID 474). 33890 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 142.0 in stage 15.0 (TID 476) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Finished task 139.0 in stage 15.0 (TID 473). 34378 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Running task 142.0 in stage 15.0 (TID 476) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 143.0 in stage 15.0 (TID 477) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 140.0 in stage 15.0 (TID 474) in 3 ms on 10.0.0.133 (executor driver) (140/200) -26/04/01 08:49:52 INFO Executor: Running task 143.0 in stage 15.0 (TID 477) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO TaskSetManager: Finished task 139.0 in stage 15.0 (TID 473) in 3 ms on 10.0.0.133 (executor driver) (141/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 141.0 in stage 15.0 (TID 475). 34328 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 144.0 in stage 15.0 (TID 478) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 144.0 in stage 15.0 (TID 478) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 141.0 in stage 15.0 (TID 475) in 4 ms on 10.0.0.133 (executor driver) (142/200) -26/04/01 08:49:52 INFO Executor: Finished task 142.0 in stage 15.0 (TID 476). 34951 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 143.0 in stage 15.0 (TID 477). 34375 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 145.0 in stage 15.0 (TID 479) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Running task 145.0 in stage 15.0 (TID 479) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 146.0 in stage 15.0 (TID 480) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 142.0 in stage 15.0 (TID 476) in 4 ms on 10.0.0.133 (executor driver) (143/200) -26/04/01 08:49:52 INFO Executor: Running task 146.0 in stage 15.0 (TID 480) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 143.0 in stage 15.0 (TID 477) in 4 ms on 10.0.0.133 (executor driver) (144/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 144.0 in stage 15.0 (TID 478). 33655 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 147.0 in stage 15.0 (TID 481) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 147.0 in stage 15.0 (TID 481) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 144.0 in stage 15.0 (TID 478) in 5 ms on 10.0.0.133 (executor driver) (145/200) -26/04/01 08:49:52 INFO Executor: Finished task 145.0 in stage 15.0 (TID 479). 34189 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 148.0 in stage 15.0 (TID 482) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 148.0 in stage 15.0 (TID 482) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 145.0 in stage 15.0 (TID 479) in 4 ms on 10.0.0.133 (executor driver) (146/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 146.0 in stage 15.0 (TID 480). 33726 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 149.0 in stage 15.0 (TID 483) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 149.0 in stage 15.0 (TID 483) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 146.0 in stage 15.0 (TID 480) in 4 ms on 10.0.0.133 (executor driver) (147/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 147.0 in stage 15.0 (TID 481). 33597 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 150.0 in stage 15.0 (TID 484) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Finished task 148.0 in stage 15.0 (TID 482). 34244 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Running task 150.0 in stage 15.0 (TID 484) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 151.0 in stage 15.0 (TID 485) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 147.0 in stage 15.0 (TID 481) in 5 ms on 10.0.0.133 (executor driver) (148/200) -26/04/01 08:49:52 INFO Executor: Running task 151.0 in stage 15.0 (TID 485) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 148.0 in stage 15.0 (TID 482) in 4 ms on 10.0.0.133 (executor driver) (149/200) -26/04/01 08:49:52 INFO Executor: Finished task 149.0 in stage 15.0 (TID 483). 34799 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 152.0 in stage 15.0 (TID 486) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 152.0 in stage 15.0 (TID 486) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 149.0 in stage 15.0 (TID 483) in 4 ms on 10.0.0.133 (executor driver) (150/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 151.0 in stage 15.0 (TID 485). 34392 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 153.0 in stage 15.0 (TID 487) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 153.0 in stage 15.0 (TID 487) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 151.0 in stage 15.0 (TID 485) in 4 ms on 10.0.0.133 (executor driver) (151/200) -26/04/01 08:49:52 INFO Executor: Finished task 152.0 in stage 15.0 (TID 486). 34441 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 154.0 in stage 15.0 (TID 488) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 154.0 in stage 15.0 (TID 488) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 152.0 in stage 15.0 (TID 486) in 4 ms on 10.0.0.133 (executor driver) (152/200) -26/04/01 08:49:52 INFO Executor: Finished task 150.0 in stage 15.0 (TID 484). 34597 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 155.0 in stage 15.0 (TID 489) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 155.0 in stage 15.0 (TID 489) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 150.0 in stage 15.0 (TID 484) in 5 ms on 10.0.0.133 (executor driver) (153/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 153.0 in stage 15.0 (TID 487). 34127 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 156.0 in stage 15.0 (TID 490) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 156.0 in stage 15.0 (TID 490) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 153.0 in stage 15.0 (TID 487) in 4 ms on 10.0.0.133 (executor driver) (154/200) -26/04/01 08:49:52 INFO Executor: Finished task 154.0 in stage 15.0 (TID 488). 33726 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 155.0 in stage 15.0 (TID 489). 33738 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 157.0 in stage 15.0 (TID 491) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 157.0 in stage 15.0 (TID 491) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 158.0 in stage 15.0 (TID 492) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 158.0 in stage 15.0 (TID 492) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 154.0 in stage 15.0 (TID 488) in 4 ms on 10.0.0.133 (executor driver) (155/200) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 155.0 in stage 15.0 (TID 489) in 4 ms on 10.0.0.133 (executor driver) (156/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 157.0 in stage 15.0 (TID 491). 33697 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 159.0 in stage 15.0 (TID 493) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Finished task 156.0 in stage 15.0 (TID 490). 34636 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 158.0 in stage 15.0 (TID 492). 34759 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Running task 159.0 in stage 15.0 (TID 493) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 160.0 in stage 15.0 (TID 494) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 157.0 in stage 15.0 (TID 491) in 4 ms on 10.0.0.133 (executor driver) (157/200) -26/04/01 08:49:52 INFO Executor: Running task 160.0 in stage 15.0 (TID 494) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 161.0 in stage 15.0 (TID 495) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 156.0 in stage 15.0 (TID 490) in 4 ms on 10.0.0.133 (executor driver) (158/200) -26/04/01 08:49:52 INFO Executor: Running task 161.0 in stage 15.0 (TID 495) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 158.0 in stage 15.0 (TID 492) in 4 ms on 10.0.0.133 (executor driver) (159/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 159.0 in stage 15.0 (TID 493). 34519 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 161.0 in stage 15.0 (TID 495). 33835 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 162.0 in stage 15.0 (TID 496) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 162.0 in stage 15.0 (TID 496) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 163.0 in stage 15.0 (TID 497) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 159.0 in stage 15.0 (TID 493) in 5 ms on 10.0.0.133 (executor driver) (160/200) -26/04/01 08:49:52 INFO Executor: Running task 163.0 in stage 15.0 (TID 497) -26/04/01 08:49:52 INFO Executor: Finished task 160.0 in stage 15.0 (TID 494). 34867 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Finished task 161.0 in stage 15.0 (TID 495) in 5 ms on 10.0.0.133 (executor driver) (161/200) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 164.0 in stage 15.0 (TID 498) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 164.0 in stage 15.0 (TID 498) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 160.0 in stage 15.0 (TID 494) in 5 ms on 10.0.0.133 (executor driver) (162/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 162.0 in stage 15.0 (TID 496). 34295 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 165.0 in stage 15.0 (TID 499) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 162.0 in stage 15.0 (TID 496) in 4 ms on 10.0.0.133 (executor driver) (163/200) -26/04/01 08:49:52 INFO Executor: Running task 165.0 in stage 15.0 (TID 499) -26/04/01 08:49:52 INFO Executor: Finished task 164.0 in stage 15.0 (TID 498). 33851 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 163.0 in stage 15.0 (TID 497). 34082 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 166.0 in stage 15.0 (TID 500) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 166.0 in stage 15.0 (TID 500) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 167.0 in stage 15.0 (TID 501) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 164.0 in stage 15.0 (TID 498) in 4 ms on 10.0.0.133 (executor driver) (164/200) -26/04/01 08:49:52 INFO Executor: Running task 167.0 in stage 15.0 (TID 501) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 163.0 in stage 15.0 (TID 497) in 4 ms on 10.0.0.133 (executor driver) (165/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.2 KiB) non-empty blocks including 8 (31.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 165.0 in stage 15.0 (TID 499). 34396 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 168.0 in stage 15.0 (TID 502) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 168.0 in stage 15.0 (TID 502) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 165.0 in stage 15.0 (TID 499) in 4 ms on 10.0.0.133 (executor driver) (166/200) -26/04/01 08:49:52 INFO Executor: Finished task 167.0 in stage 15.0 (TID 501). 33887 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 166.0 in stage 15.0 (TID 500). 34848 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 169.0 in stage 15.0 (TID 503) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 169.0 in stage 15.0 (TID 503) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 170.0 in stage 15.0 (TID 504) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 167.0 in stage 15.0 (TID 501) in 4 ms on 10.0.0.133 (executor driver) (167/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO Executor: Running task 170.0 in stage 15.0 (TID 504) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO TaskSetManager: Finished task 166.0 in stage 15.0 (TID 500) in 4 ms on 10.0.0.133 (executor driver) (168/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 168.0 in stage 15.0 (TID 502). 33588 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 171.0 in stage 15.0 (TID 505) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 171.0 in stage 15.0 (TID 505) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 168.0 in stage 15.0 (TID 502) in 4 ms on 10.0.0.133 (executor driver) (169/200) -26/04/01 08:49:52 INFO Executor: Finished task 169.0 in stage 15.0 (TID 503). 34471 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 170.0 in stage 15.0 (TID 504). 34026 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 172.0 in stage 15.0 (TID 506) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Running task 172.0 in stage 15.0 (TID 506) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 173.0 in stage 15.0 (TID 507) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 169.0 in stage 15.0 (TID 503) in 4 ms on 10.0.0.133 (executor driver) (170/200) -26/04/01 08:49:52 INFO Executor: Running task 173.0 in stage 15.0 (TID 507) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 170.0 in stage 15.0 (TID 504) in 4 ms on 10.0.0.133 (executor driver) (171/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 171.0 in stage 15.0 (TID 505). 34099 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 174.0 in stage 15.0 (TID 508) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 174.0 in stage 15.0 (TID 508) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 171.0 in stage 15.0 (TID 505) in 4 ms on 10.0.0.133 (executor driver) (172/200) -26/04/01 08:49:52 INFO Executor: Finished task 172.0 in stage 15.0 (TID 506). 33919 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 173.0 in stage 15.0 (TID 507). 34566 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 175.0 in stage 15.0 (TID 509) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 175.0 in stage 15.0 (TID 509) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO TaskSetManager: Starting task 176.0 in stage 15.0 (TID 510) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO TaskSetManager: Finished task 172.0 in stage 15.0 (TID 506) in 4 ms on 10.0.0.133 (executor driver) (173/200) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 173.0 in stage 15.0 (TID 507) in 4 ms on 10.0.0.133 (executor driver) (174/200) -26/04/01 08:49:52 INFO Executor: Running task 176.0 in stage 15.0 (TID 510) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 174.0 in stage 15.0 (TID 508). 33575 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 177.0 in stage 15.0 (TID 511) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 174.0 in stage 15.0 (TID 508) in 7 ms on 10.0.0.133 (executor driver) (175/200) -26/04/01 08:49:52 INFO Executor: Finished task 175.0 in stage 15.0 (TID 509). 34385 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 178.0 in stage 15.0 (TID 512) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 175.0 in stage 15.0 (TID 509) in 6 ms on 10.0.0.133 (executor driver) (176/200) -26/04/01 08:49:52 INFO Executor: Running task 177.0 in stage 15.0 (TID 511) -26/04/01 08:49:52 INFO Executor: Running task 178.0 in stage 15.0 (TID 512) -26/04/01 08:49:52 INFO Executor: Finished task 176.0 in stage 15.0 (TID 510). 33718 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 179.0 in stage 15.0 (TID 513) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 179.0 in stage 15.0 (TID 513) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 176.0 in stage 15.0 (TID 510) in 7 ms on 10.0.0.133 (executor driver) (177/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 177.0 in stage 15.0 (TID 511). 34399 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 180.0 in stage 15.0 (TID 514) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Finished task 178.0 in stage 15.0 (TID 512). 34255 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 179.0 in stage 15.0 (TID 513). 34030 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Finished task 177.0 in stage 15.0 (TID 511) in 5 ms on 10.0.0.133 (executor driver) (178/200) -26/04/01 08:49:52 INFO Executor: Running task 180.0 in stage 15.0 (TID 514) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 181.0 in stage 15.0 (TID 515) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 181.0 in stage 15.0 (TID 515) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 182.0 in stage 15.0 (TID 516) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 178.0 in stage 15.0 (TID 512) in 5 ms on 10.0.0.133 (executor driver) (179/200) -26/04/01 08:49:52 INFO Executor: Running task 182.0 in stage 15.0 (TID 516) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 179.0 in stage 15.0 (TID 513) in 5 ms on 10.0.0.133 (executor driver) (180/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 180.0 in stage 15.0 (TID 514). 34531 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 182.0 in stage 15.0 (TID 516). 34252 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 183.0 in stage 15.0 (TID 517) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 183.0 in stage 15.0 (TID 517) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 184.0 in stage 15.0 (TID 518) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 180.0 in stage 15.0 (TID 514) in 4 ms on 10.0.0.133 (executor driver) (181/200) -26/04/01 08:49:52 INFO Executor: Running task 184.0 in stage 15.0 (TID 518) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 182.0 in stage 15.0 (TID 516) in 4 ms on 10.0.0.133 (executor driver) (182/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (31.6 KiB) non-empty blocks including 8 (31.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 181.0 in stage 15.0 (TID 515). 34196 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 185.0 in stage 15.0 (TID 519) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 185.0 in stage 15.0 (TID 519) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 181.0 in stage 15.0 (TID 515) in 5 ms on 10.0.0.133 (executor driver) (183/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 183.0 in stage 15.0 (TID 517). 34150 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 184.0 in stage 15.0 (TID 518). 34956 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 186.0 in stage 15.0 (TID 520) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 183.0 in stage 15.0 (TID 517) in 4 ms on 10.0.0.133 (executor driver) (184/200) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 187.0 in stage 15.0 (TID 521) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 186.0 in stage 15.0 (TID 520) -26/04/01 08:49:52 INFO Executor: Running task 187.0 in stage 15.0 (TID 521) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 184.0 in stage 15.0 (TID 518) in 4 ms on 10.0.0.133 (executor driver) (185/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO Executor: Finished task 185.0 in stage 15.0 (TID 519). 34506 bytes result sent to driver -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO TaskSetManager: Starting task 188.0 in stage 15.0 (TID 522) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO Executor: Running task 188.0 in stage 15.0 (TID 522) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO TaskSetManager: Finished task 185.0 in stage 15.0 (TID 519) in 4 ms on 10.0.0.133 (executor driver) (186/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 187.0 in stage 15.0 (TID 521). 34650 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 189.0 in stage 15.0 (TID 523) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 189.0 in stage 15.0 (TID 523) -26/04/01 08:49:52 INFO Executor: Finished task 186.0 in stage 15.0 (TID 520). 33801 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Finished task 187.0 in stage 15.0 (TID 521) in 5 ms on 10.0.0.133 (executor driver) (187/200) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 190.0 in stage 15.0 (TID 524) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Finished task 188.0 in stage 15.0 (TID 522). 34848 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Running task 190.0 in stage 15.0 (TID 524) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 191.0 in stage 15.0 (TID 525) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 186.0 in stage 15.0 (TID 520) in 5 ms on 10.0.0.133 (executor driver) (188/200) -26/04/01 08:49:52 INFO Executor: Running task 191.0 in stage 15.0 (TID 525) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 188.0 in stage 15.0 (TID 522) in 4 ms on 10.0.0.133 (executor driver) (189/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.9 KiB) non-empty blocks including 8 (30.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 189.0 in stage 15.0 (TID 523). 34333 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 192.0 in stage 15.0 (TID 526) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 192.0 in stage 15.0 (TID 526) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 189.0 in stage 15.0 (TID 523) in 5 ms on 10.0.0.133 (executor driver) (190/200) -26/04/01 08:49:52 INFO Executor: Finished task 191.0 in stage 15.0 (TID 525). 34559 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 193.0 in stage 15.0 (TID 527) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Finished task 190.0 in stage 15.0 (TID 524). 34397 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Running task 193.0 in stage 15.0 (TID 527) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 194.0 in stage 15.0 (TID 528) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 191.0 in stage 15.0 (TID 525) in 4 ms on 10.0.0.133 (executor driver) (191/200) -26/04/01 08:49:52 INFO Executor: Running task 194.0 in stage 15.0 (TID 528) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 190.0 in stage 15.0 (TID 524) in 4 ms on 10.0.0.133 (executor driver) (192/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 192.0 in stage 15.0 (TID 526). 33410 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 195.0 in stage 15.0 (TID 529) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 195.0 in stage 15.0 (TID 529) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 192.0 in stage 15.0 (TID 526) in 4 ms on 10.0.0.133 (executor driver) (193/200) -26/04/01 08:49:52 INFO Executor: Finished task 194.0 in stage 15.0 (TID 528). 34348 bytes result sent to driver -26/04/01 08:49:52 INFO Executor: Finished task 193.0 in stage 15.0 (TID 527). 34499 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 196.0 in stage 15.0 (TID 530) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Running task 196.0 in stage 15.0 (TID 530) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 197.0 in stage 15.0 (TID 531) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 194.0 in stage 15.0 (TID 528) in 4 ms on 10.0.0.133 (executor driver) (194/200) -26/04/01 08:49:52 INFO Executor: Running task 197.0 in stage 15.0 (TID 531) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 193.0 in stage 15.0 (TID 527) in 4 ms on 10.0.0.133 (executor driver) (195/200) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO Executor: Finished task 195.0 in stage 15.0 (TID 529). 34725 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 198.0 in stage 15.0 (TID 532) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 198.0 in stage 15.0 (TID 532) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 195.0 in stage 15.0 (TID 529) in 4 ms on 10.0.0.133 (executor driver) (196/200) -26/04/01 08:49:52 INFO Executor: Finished task 196.0 in stage 15.0 (TID 530). 34709 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Starting task 199.0 in stage 15.0 (TID 533) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9215 bytes) -26/04/01 08:49:52 INFO Executor: Running task 199.0 in stage 15.0 (TID 533) -26/04/01 08:49:52 INFO TaskSetManager: Finished task 196.0 in stage 15.0 (TID 530) in 3 ms on 10.0.0.133 (executor driver) (197/200) -26/04/01 08:49:52 INFO Executor: Finished task 197.0 in stage 15.0 (TID 531). 34348 bytes result sent to driver -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.8 KiB) non-empty blocks including 8 (30.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO TaskSetManager: Finished task 197.0 in stage 15.0 (TID 531) in 4 ms on 10.0.0.133 (executor driver) (198/200) -26/04/01 08:49:52 INFO Executor: Finished task 199.0 in stage 15.0 (TID 533). 34766 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Finished task 199.0 in stage 15.0 (TID 533) in 4 ms on 10.0.0.133 (executor driver) (199/200) -26/04/01 08:49:52 INFO Executor: Finished task 198.0 in stage 15.0 (TID 532). 34063 bytes result sent to driver -26/04/01 08:49:52 INFO TaskSetManager: Finished task 198.0 in stage 15.0 (TID 532) in 4 ms on 10.0.0.133 (executor driver) (200/200) -26/04/01 08:49:52 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool -26/04/01 08:49:52 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 230.196 s -26/04/01 08:49:52 INFO DAGScheduler: Job 14 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:49:52 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished -26/04/01 08:49:52 INFO DAGScheduler: Job 14 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 230.211398 s -26/04/01 08:49:52 INFO Utils: Coalesced 1600 broadcast batches into 1 (1088125 rows) -26/04/01 08:49:52 INFO MemoryStore: Block broadcast_22 stored as values in memory (estimated size 5.2 MiB, free 12.4 GiB) -26/04/01 08:49:52 INFO MemoryStore: Block broadcast_22_piece0 stored as bytes in memory (estimated size 4.0 MiB, free 12.4 GiB) -26/04/01 08:49:52 INFO BlockManagerInfo: Added broadcast_22_piece0 in memory on 10.0.0.133:59333 (size: 4.0 MiB, free: 12.6 GiB) -26/04/01 08:49:52 INFO MemoryStore: Block broadcast_22_piece1 stored as bytes in memory (estimated size 357.8 KiB, free 12.4 GiB) -26/04/01 08:49:52 INFO BlockManagerInfo: Added broadcast_22_piece1 in memory on 10.0.0.133:59333 (size: 357.8 KiB, free: 12.6 GiB) -26/04/01 08:49:52 INFO SparkContext: Created broadcast 22 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:49:52 INFO ShufflePartitionsUtil: For shuffle(1), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 08:49:52 INFO DAGScheduler: Registering RDD 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 5 -26/04/01 08:49:52 INFO DAGScheduler: Got map stage job 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions -26/04/01 08:49:52 INFO DAGScheduler: Final stage: ShuffleMapStage 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:49:52 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 16) -26/04/01 08:49:52 INFO DAGScheduler: Missing parents: List() -26/04/01 08:49:52 INFO DAGScheduler: Submitting ShuffleMapStage 17 (MapPartitionsRDD[39] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:49:52 INFO MemoryStore: Block broadcast_23 stored as values in memory (estimated size 21.7 KiB, free 12.4 GiB) -26/04/01 08:49:52 INFO MemoryStore: Block broadcast_23_piece0 stored as bytes in memory (estimated size 11.0 KiB, free 12.4 GiB) -26/04/01 08:49:52 INFO BlockManagerInfo: Added broadcast_23_piece0 in memory on 10.0.0.133:59333 (size: 11.0 KiB, free: 12.6 GiB) -26/04/01 08:49:52 INFO SparkContext: Created broadcast 23 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:49:52 INFO DAGScheduler: Submitting 200 missing tasks from ShuffleMapStage 17 (MapPartitionsRDD[39] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:49:52 INFO TaskSchedulerImpl: Adding task set 17.0 with 200 tasks resource profile 0 -26/04/01 08:49:52 INFO TaskSetManager: Starting task 0.0 in stage 17.0 (TID 534) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 1.0 in stage 17.0 (TID 535) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:52 INFO TaskSetManager: Starting task 2.0 in stage 17.0 (TID 536) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:52 INFO Executor: Running task 0.0 in stage 17.0 (TID 534) -26/04/01 08:49:52 INFO Executor: Running task 2.0 in stage 17.0 (TID 536) -26/04/01 08:49:52 INFO Executor: Running task 1.0 in stage 17.0 (TID 535) -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:53 INFO Executor: Finished task 59.0 in stage 13.0 (TID 329). 6606 bytes result sent to driver -26/04/01 08:49:53 INFO TaskSetManager: Starting task 3.0 in stage 17.0 (TID 537) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:53 INFO TaskSetManager: Finished task 59.0 in stage 13.0 (TID 329) in 1583 ms on 10.0.0.133 (executor driver) (60/64) -26/04/01 08:49:53 INFO Executor: Running task 3.0 in stage 17.0 (TID 537) -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:53 INFO Executor: Finished task 60.0 in stage 13.0 (TID 330). 6606 bytes result sent to driver -26/04/01 08:49:53 INFO TaskSetManager: Starting task 4.0 in stage 17.0 (TID 538) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:53 INFO TaskSetManager: Finished task 60.0 in stage 13.0 (TID 330) in 1589 ms on 10.0.0.133 (executor driver) (61/64) -26/04/01 08:49:53 INFO Executor: Running task 4.0 in stage 17.0 (TID 538) -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:53 INFO Executor: Finished task 61.0 in stage 13.0 (TID 331). 6606 bytes result sent to driver -26/04/01 08:49:53 INFO TaskSetManager: Starting task 5.0 in stage 17.0 (TID 539) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:53 INFO Executor: Running task 5.0 in stage 17.0 (TID 539) -26/04/01 08:49:53 INFO TaskSetManager: Finished task 61.0 in stage 13.0 (TID 331) in 1581 ms on 10.0.0.133 (executor driver) (62/64) -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:53 INFO Executor: Finished task 62.0 in stage 13.0 (TID 332). 6606 bytes result sent to driver -26/04/01 08:49:53 INFO TaskSetManager: Starting task 6.0 in stage 17.0 (TID 540) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:53 INFO Executor: Running task 6.0 in stage 17.0 (TID 540) -26/04/01 08:49:53 INFO TaskSetManager: Finished task 62.0 in stage 13.0 (TID 332) in 1578 ms on 10.0.0.133 (executor driver) (63/64) -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:53 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:54 INFO Executor: Finished task 63.0 in stage 13.0 (TID 333). 6606 bytes result sent to driver -26/04/01 08:49:54 INFO TaskSetManager: Starting task 7.0 in stage 17.0 (TID 541) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:54 INFO TaskSetManager: Finished task 63.0 in stage 13.0 (TID 333) in 1582 ms on 10.0.0.133 (executor driver) (64/64) -26/04/01 08:49:54 INFO Executor: Running task 7.0 in stage 17.0 (TID 541) -26/04/01 08:49:54 INFO TaskSchedulerImpl: Removed TaskSet 13.0, whose tasks have all completed, from pool -26/04/01 08:49:54 INFO DAGScheduler: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 233.877 s -26/04/01 08:49:54 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:49:54 INFO DAGScheduler: running: Set(ShuffleMapStage 17) -26/04/01 08:49:54 INFO DAGScheduler: waiting: Set() -26/04/01 08:49:54 INFO DAGScheduler: failed: Set() -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:54 INFO Executor: Finished task 0.0 in stage 17.0 (TID 534). 6401 bytes result sent to driver -26/04/01 08:49:54 INFO TaskSetManager: Starting task 8.0 in stage 17.0 (TID 542) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:54 INFO Executor: Running task 8.0 in stage 17.0 (TID 542) -26/04/01 08:49:54 INFO TaskSetManager: Finished task 0.0 in stage 17.0 (TID 534) in 2021 ms on 10.0.0.133 (executor driver) (1/200) -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:54 INFO Executor: Finished task 1.0 in stage 17.0 (TID 535). 6401 bytes result sent to driver -26/04/01 08:49:54 INFO TaskSetManager: Starting task 9.0 in stage 17.0 (TID 543) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:54 INFO TaskSetManager: Finished task 1.0 in stage 17.0 (TID 535) in 2033 ms on 10.0.0.133 (executor driver) (2/200) -26/04/01 08:49:54 INFO Executor: Running task 9.0 in stage 17.0 (TID 543) -26/04/01 08:49:54 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:54 INFO Executor: Finished task 2.0 in stage 17.0 (TID 536). 6401 bytes result sent to driver -26/04/01 08:49:54 INFO TaskSetManager: Starting task 10.0 in stage 17.0 (TID 544) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:54 INFO Executor: Running task 10.0 in stage 17.0 (TID 544) -26/04/01 08:49:54 INFO TaskSetManager: Finished task 2.0 in stage 17.0 (TID 536) in 2035 ms on 10.0.0.133 (executor driver) (3/200) -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:55 INFO Executor: Finished task 3.0 in stage 17.0 (TID 537). 6401 bytes result sent to driver -26/04/01 08:49:55 INFO TaskSetManager: Starting task 11.0 in stage 17.0 (TID 545) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:55 INFO TaskSetManager: Finished task 3.0 in stage 17.0 (TID 537) in 1950 ms on 10.0.0.133 (executor driver) (4/200) -26/04/01 08:49:55 INFO Executor: Running task 11.0 in stage 17.0 (TID 545) -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:55 INFO Executor: Finished task 4.0 in stage 17.0 (TID 538). 6401 bytes result sent to driver -26/04/01 08:49:55 INFO TaskSetManager: Starting task 12.0 in stage 17.0 (TID 546) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:55 INFO Executor: Running task 12.0 in stage 17.0 (TID 546) -26/04/01 08:49:55 INFO TaskSetManager: Finished task 4.0 in stage 17.0 (TID 538) in 1864 ms on 10.0.0.133 (executor driver) (5/200) -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:55 INFO Executor: Finished task 5.0 in stage 17.0 (TID 539). 6401 bytes result sent to driver -26/04/01 08:49:55 INFO TaskSetManager: Starting task 13.0 in stage 17.0 (TID 547) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:55 INFO Executor: Running task 13.0 in stage 17.0 (TID 547) -26/04/01 08:49:55 INFO TaskSetManager: Finished task 5.0 in stage 17.0 (TID 539) in 1863 ms on 10.0.0.133 (executor driver) (6/200) -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:55 INFO Executor: Finished task 6.0 in stage 17.0 (TID 540). 6401 bytes result sent to driver -26/04/01 08:49:55 INFO TaskSetManager: Starting task 14.0 in stage 17.0 (TID 548) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:55 INFO TaskSetManager: Finished task 6.0 in stage 17.0 (TID 540) in 1854 ms on 10.0.0.133 (executor driver) (7/200) -26/04/01 08:49:55 INFO Executor: Running task 14.0 in stage 17.0 (TID 548) -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:55 INFO Executor: Finished task 7.0 in stage 17.0 (TID 541). 6401 bytes result sent to driver -26/04/01 08:49:55 INFO TaskSetManager: Starting task 15.0 in stage 17.0 (TID 549) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:55 INFO Executor: Running task 15.0 in stage 17.0 (TID 549) -26/04/01 08:49:55 INFO TaskSetManager: Finished task 7.0 in stage 17.0 (TID 541) in 1863 ms on 10.0.0.133 (executor driver) (8/200) -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:55 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:56 INFO Executor: Finished task 8.0 in stage 17.0 (TID 542). 6401 bytes result sent to driver -26/04/01 08:49:56 INFO TaskSetManager: Starting task 16.0 in stage 17.0 (TID 550) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:56 INFO Executor: Running task 16.0 in stage 17.0 (TID 550) -26/04/01 08:49:56 INFO TaskSetManager: Finished task 8.0 in stage 17.0 (TID 542) in 1831 ms on 10.0.0.133 (executor driver) (9/200) -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:56 INFO Executor: Finished task 10.0 in stage 17.0 (TID 544). 6401 bytes result sent to driver -26/04/01 08:49:56 INFO TaskSetManager: Starting task 17.0 in stage 17.0 (TID 551) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:56 INFO TaskSetManager: Finished task 10.0 in stage 17.0 (TID 544) in 1839 ms on 10.0.0.133 (executor driver) (10/200) -26/04/01 08:49:56 INFO Executor: Running task 17.0 in stage 17.0 (TID 551) -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:56 INFO Executor: Finished task 9.0 in stage 17.0 (TID 543). 6401 bytes result sent to driver -26/04/01 08:49:56 INFO TaskSetManager: Starting task 18.0 in stage 17.0 (TID 552) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:56 INFO Executor: Running task 18.0 in stage 17.0 (TID 552) -26/04/01 08:49:56 INFO TaskSetManager: Finished task 9.0 in stage 17.0 (TID 543) in 1854 ms on 10.0.0.133 (executor driver) (11/200) -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:56 INFO Executor: Finished task 11.0 in stage 17.0 (TID 545). 6401 bytes result sent to driver -26/04/01 08:49:56 INFO TaskSetManager: Starting task 19.0 in stage 17.0 (TID 553) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:56 INFO TaskSetManager: Finished task 11.0 in stage 17.0 (TID 545) in 1840 ms on 10.0.0.133 (executor driver) (12/200) -26/04/01 08:49:56 INFO Executor: Running task 19.0 in stage 17.0 (TID 553) -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:56 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:57 INFO Executor: Finished task 12.0 in stage 17.0 (TID 546). 6401 bytes result sent to driver -26/04/01 08:49:57 INFO TaskSetManager: Starting task 20.0 in stage 17.0 (TID 554) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:57 INFO TaskSetManager: Finished task 12.0 in stage 17.0 (TID 546) in 1841 ms on 10.0.0.133 (executor driver) (13/200) -26/04/01 08:49:57 INFO Executor: Running task 20.0 in stage 17.0 (TID 554) -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:57 INFO Executor: Finished task 13.0 in stage 17.0 (TID 547). 6401 bytes result sent to driver -26/04/01 08:49:57 INFO TaskSetManager: Starting task 21.0 in stage 17.0 (TID 555) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:57 INFO Executor: Running task 21.0 in stage 17.0 (TID 555) -26/04/01 08:49:57 INFO TaskSetManager: Finished task 13.0 in stage 17.0 (TID 547) in 1911 ms on 10.0.0.133 (executor driver) (14/200) -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:57 INFO Executor: Finished task 14.0 in stage 17.0 (TID 548). 6401 bytes result sent to driver -26/04/01 08:49:57 INFO TaskSetManager: Starting task 22.0 in stage 17.0 (TID 556) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:57 INFO Executor: Running task 22.0 in stage 17.0 (TID 556) -26/04/01 08:49:57 INFO TaskSetManager: Finished task 14.0 in stage 17.0 (TID 548) in 1926 ms on 10.0.0.133 (executor driver) (15/200) -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (76.4 MiB) non-empty blocks including 208 (76.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (76.4 MiB) non-empty blocks including 208 (76.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:57 INFO Executor: Finished task 15.0 in stage 17.0 (TID 549). 6401 bytes result sent to driver -26/04/01 08:49:57 INFO TaskSetManager: Starting task 23.0 in stage 17.0 (TID 557) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:57 INFO TaskSetManager: Finished task 15.0 in stage 17.0 (TID 549) in 1851 ms on 10.0.0.133 (executor driver) (16/200) -26/04/01 08:49:57 INFO Executor: Running task 23.0 in stage 17.0 (TID 557) -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:57 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:58 INFO Executor: Finished task 16.0 in stage 17.0 (TID 550). 6401 bytes result sent to driver -26/04/01 08:49:58 INFO TaskSetManager: Starting task 24.0 in stage 17.0 (TID 558) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:58 INFO TaskSetManager: Finished task 16.0 in stage 17.0 (TID 550) in 1843 ms on 10.0.0.133 (executor driver) (17/200) -26/04/01 08:49:58 INFO Executor: Running task 24.0 in stage 17.0 (TID 558) -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:58 INFO Executor: Finished task 17.0 in stage 17.0 (TID 551). 6401 bytes result sent to driver -26/04/01 08:49:58 INFO TaskSetManager: Starting task 25.0 in stage 17.0 (TID 559) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:58 INFO TaskSetManager: Finished task 17.0 in stage 17.0 (TID 551) in 1847 ms on 10.0.0.133 (executor driver) (18/200) -26/04/01 08:49:58 INFO Executor: Running task 25.0 in stage 17.0 (TID 559) -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:58 INFO Executor: Finished task 18.0 in stage 17.0 (TID 552). 6401 bytes result sent to driver -26/04/01 08:49:58 INFO TaskSetManager: Starting task 26.0 in stage 17.0 (TID 560) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:58 INFO TaskSetManager: Finished task 18.0 in stage 17.0 (TID 552) in 1853 ms on 10.0.0.133 (executor driver) (19/200) -26/04/01 08:49:58 INFO Executor: Running task 26.0 in stage 17.0 (TID 560) -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:58 INFO Executor: Finished task 19.0 in stage 17.0 (TID 553). 6401 bytes result sent to driver -26/04/01 08:49:58 INFO TaskSetManager: Starting task 27.0 in stage 17.0 (TID 561) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:58 INFO Executor: Running task 27.0 in stage 17.0 (TID 561) -26/04/01 08:49:58 INFO TaskSetManager: Finished task 19.0 in stage 17.0 (TID 553) in 1860 ms on 10.0.0.133 (executor driver) (20/200) -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:58 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:59 INFO Executor: Finished task 20.0 in stage 17.0 (TID 554). 6401 bytes result sent to driver -26/04/01 08:49:59 INFO TaskSetManager: Starting task 28.0 in stage 17.0 (TID 562) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:59 INFO Executor: Running task 28.0 in stage 17.0 (TID 562) -26/04/01 08:49:59 INFO TaskSetManager: Finished task 20.0 in stage 17.0 (TID 554) in 1847 ms on 10.0.0.133 (executor driver) (21/200) -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:59 INFO Executor: Finished task 21.0 in stage 17.0 (TID 555). 6401 bytes result sent to driver -26/04/01 08:49:59 INFO TaskSetManager: Starting task 29.0 in stage 17.0 (TID 563) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:59 INFO TaskSetManager: Finished task 21.0 in stage 17.0 (TID 555) in 1860 ms on 10.0.0.133 (executor driver) (22/200) -26/04/01 08:49:59 INFO Executor: Running task 29.0 in stage 17.0 (TID 563) -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:59 INFO Executor: Finished task 22.0 in stage 17.0 (TID 556). 6401 bytes result sent to driver -26/04/01 08:49:59 INFO TaskSetManager: Starting task 30.0 in stage 17.0 (TID 564) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:59 INFO Executor: Running task 30.0 in stage 17.0 (TID 564) -26/04/01 08:49:59 INFO TaskSetManager: Finished task 22.0 in stage 17.0 (TID 556) in 1856 ms on 10.0.0.133 (executor driver) (23/200) -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:59 INFO Executor: Finished task 23.0 in stage 17.0 (TID 557). 6401 bytes result sent to driver -26/04/01 08:49:59 INFO TaskSetManager: Starting task 31.0 in stage 17.0 (TID 565) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9624 bytes) -26/04/01 08:49:59 INFO Executor: Running task 31.0 in stage 17.0 (TID 565) -26/04/01 08:49:59 INFO TaskSetManager: Finished task 23.0 in stage 17.0 (TID 557) in 1855 ms on 10.0.0.133 (executor driver) (24/200) -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:49:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:49:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:00 INFO Executor: Finished task 24.0 in stage 17.0 (TID 558). 6401 bytes result sent to driver -26/04/01 08:50:00 INFO TaskSetManager: Starting task 32.0 in stage 17.0 (TID 566) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:00 INFO TaskSetManager: Finished task 24.0 in stage 17.0 (TID 558) in 1828 ms on 10.0.0.133 (executor driver) (25/200) -26/04/01 08:50:00 INFO Executor: Running task 32.0 in stage 17.0 (TID 566) -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:00 INFO Executor: Finished task 26.0 in stage 17.0 (TID 560). 6401 bytes result sent to driver -26/04/01 08:50:00 INFO TaskSetManager: Starting task 33.0 in stage 17.0 (TID 567) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:00 INFO Executor: Running task 33.0 in stage 17.0 (TID 567) -26/04/01 08:50:00 INFO TaskSetManager: Finished task 26.0 in stage 17.0 (TID 560) in 1802 ms on 10.0.0.133 (executor driver) (26/200) -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:00 INFO Executor: Finished task 25.0 in stage 17.0 (TID 559). 6401 bytes result sent to driver -26/04/01 08:50:00 INFO TaskSetManager: Starting task 34.0 in stage 17.0 (TID 568) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:00 INFO Executor: Running task 34.0 in stage 17.0 (TID 568) -26/04/01 08:50:00 INFO TaskSetManager: Finished task 25.0 in stage 17.0 (TID 559) in 1839 ms on 10.0.0.133 (executor driver) (27/200) -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:00 INFO Executor: Finished task 27.0 in stage 17.0 (TID 561). 6401 bytes result sent to driver -26/04/01 08:50:00 INFO TaskSetManager: Starting task 35.0 in stage 17.0 (TID 569) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:00 INFO Executor: Running task 35.0 in stage 17.0 (TID 569) -26/04/01 08:50:00 INFO TaskSetManager: Finished task 27.0 in stage 17.0 (TID 561) in 1845 ms on 10.0.0.133 (executor driver) (28/200) -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:00 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:01 INFO Executor: Finished task 28.0 in stage 17.0 (TID 562). 7132 bytes result sent to driver -26/04/01 08:50:01 INFO TaskSetManager: Starting task 36.0 in stage 17.0 (TID 570) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:01 INFO TaskSetManager: Finished task 28.0 in stage 17.0 (TID 562) in 1840 ms on 10.0.0.133 (executor driver) (29/200) -26/04/01 08:50:01 INFO Executor: Running task 36.0 in stage 17.0 (TID 570) -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:01 INFO Executor: Finished task 29.0 in stage 17.0 (TID 563). 7132 bytes result sent to driver -26/04/01 08:50:01 INFO TaskSetManager: Starting task 37.0 in stage 17.0 (TID 571) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:01 INFO Executor: Running task 37.0 in stage 17.0 (TID 571) -26/04/01 08:50:01 INFO TaskSetManager: Finished task 29.0 in stage 17.0 (TID 563) in 1835 ms on 10.0.0.133 (executor driver) (30/200) -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:01 INFO Executor: Finished task 30.0 in stage 17.0 (TID 564). 7132 bytes result sent to driver -26/04/01 08:50:01 INFO TaskSetManager: Starting task 38.0 in stage 17.0 (TID 572) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:01 INFO TaskSetManager: Finished task 30.0 in stage 17.0 (TID 564) in 1834 ms on 10.0.0.133 (executor driver) (31/200) -26/04/01 08:50:01 INFO Executor: Running task 38.0 in stage 17.0 (TID 572) -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:01 INFO Executor: Finished task 31.0 in stage 17.0 (TID 565). 7132 bytes result sent to driver -26/04/01 08:50:01 INFO TaskSetManager: Starting task 39.0 in stage 17.0 (TID 573) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:01 INFO Executor: Running task 39.0 in stage 17.0 (TID 573) -26/04/01 08:50:01 INFO TaskSetManager: Finished task 31.0 in stage 17.0 (TID 565) in 1851 ms on 10.0.0.133 (executor driver) (32/200) -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:01 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:02 INFO Executor: Finished task 32.0 in stage 17.0 (TID 566). 7132 bytes result sent to driver -26/04/01 08:50:02 INFO TaskSetManager: Starting task 40.0 in stage 17.0 (TID 574) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:02 INFO TaskSetManager: Finished task 32.0 in stage 17.0 (TID 566) in 1857 ms on 10.0.0.133 (executor driver) (33/200) -26/04/01 08:50:02 INFO Executor: Running task 40.0 in stage 17.0 (TID 574) -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:02 INFO Executor: Finished task 33.0 in stage 17.0 (TID 567). 7132 bytes result sent to driver -26/04/01 08:50:02 INFO TaskSetManager: Starting task 41.0 in stage 17.0 (TID 575) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:02 INFO Executor: Running task 41.0 in stage 17.0 (TID 575) -26/04/01 08:50:02 INFO TaskSetManager: Finished task 33.0 in stage 17.0 (TID 567) in 1850 ms on 10.0.0.133 (executor driver) (34/200) -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:02 INFO Executor: Finished task 34.0 in stage 17.0 (TID 568). 7132 bytes result sent to driver -26/04/01 08:50:02 INFO TaskSetManager: Starting task 42.0 in stage 17.0 (TID 576) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:02 INFO TaskSetManager: Finished task 34.0 in stage 17.0 (TID 568) in 1853 ms on 10.0.0.133 (executor driver) (35/200) -26/04/01 08:50:02 INFO Executor: Running task 42.0 in stage 17.0 (TID 576) -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:02 INFO Executor: Finished task 35.0 in stage 17.0 (TID 569). 7132 bytes result sent to driver -26/04/01 08:50:02 INFO TaskSetManager: Starting task 43.0 in stage 17.0 (TID 577) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:02 INFO TaskSetManager: Finished task 35.0 in stage 17.0 (TID 569) in 1909 ms on 10.0.0.133 (executor driver) (36/200) -26/04/01 08:50:02 INFO Executor: Running task 43.0 in stage 17.0 (TID 577) -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:02 INFO Executor: Finished task 36.0 in stage 17.0 (TID 570). 6401 bytes result sent to driver -26/04/01 08:50:02 INFO TaskSetManager: Starting task 44.0 in stage 17.0 (TID 578) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:02 INFO TaskSetManager: Finished task 36.0 in stage 17.0 (TID 570) in 1876 ms on 10.0.0.133 (executor driver) (37/200) -26/04/01 08:50:02 INFO Executor: Running task 44.0 in stage 17.0 (TID 578) -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:02 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:03 INFO Executor: Finished task 37.0 in stage 17.0 (TID 571). 6401 bytes result sent to driver -26/04/01 08:50:03 INFO TaskSetManager: Starting task 45.0 in stage 17.0 (TID 579) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:03 INFO TaskSetManager: Finished task 37.0 in stage 17.0 (TID 571) in 1861 ms on 10.0.0.133 (executor driver) (38/200) -26/04/01 08:50:03 INFO Executor: Running task 45.0 in stage 17.0 (TID 579) -26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:03 INFO Executor: Finished task 38.0 in stage 17.0 (TID 572). 6401 bytes result sent to driver -26/04/01 08:50:03 INFO TaskSetManager: Starting task 46.0 in stage 17.0 (TID 580) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:03 INFO TaskSetManager: Finished task 38.0 in stage 17.0 (TID 572) in 1836 ms on 10.0.0.133 (executor driver) (39/200) -26/04/01 08:50:03 INFO Executor: Running task 46.0 in stage 17.0 (TID 580) -26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:03 INFO Executor: Finished task 39.0 in stage 17.0 (TID 573). 6487 bytes result sent to driver -26/04/01 08:50:03 INFO TaskSetManager: Starting task 47.0 in stage 17.0 (TID 581) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:03 INFO Executor: Running task 47.0 in stage 17.0 (TID 581) -26/04/01 08:50:03 INFO TaskSetManager: Finished task 39.0 in stage 17.0 (TID 573) in 1845 ms on 10.0.0.133 (executor driver) (40/200) -26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:03 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:04 INFO Executor: Finished task 40.0 in stage 17.0 (TID 574). 6444 bytes result sent to driver -26/04/01 08:50:04 INFO Executor: Finished task 41.0 in stage 17.0 (TID 575). 6444 bytes result sent to driver -26/04/01 08:50:04 INFO TaskSetManager: Starting task 48.0 in stage 17.0 (TID 582) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:04 INFO TaskSetManager: Finished task 40.0 in stage 17.0 (TID 574) in 1855 ms on 10.0.0.133 (executor driver) (41/200) -26/04/01 08:50:04 INFO Executor: Running task 48.0 in stage 17.0 (TID 582) -26/04/01 08:50:04 INFO TaskSetManager: Starting task 49.0 in stage 17.0 (TID 583) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:04 INFO Executor: Running task 49.0 in stage 17.0 (TID 583) -26/04/01 08:50:04 INFO TaskSetManager: Finished task 41.0 in stage 17.0 (TID 575) in 1844 ms on 10.0.0.133 (executor driver) (42/200) -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:04 INFO Executor: Finished task 42.0 in stage 17.0 (TID 576). 6444 bytes result sent to driver -26/04/01 08:50:04 INFO TaskSetManager: Starting task 50.0 in stage 17.0 (TID 584) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:04 INFO TaskSetManager: Finished task 42.0 in stage 17.0 (TID 576) in 1840 ms on 10.0.0.133 (executor driver) (43/200) -26/04/01 08:50:04 INFO Executor: Running task 50.0 in stage 17.0 (TID 584) -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:04 INFO Executor: Finished task 43.0 in stage 17.0 (TID 577). 6444 bytes result sent to driver -26/04/01 08:50:04 INFO TaskSetManager: Starting task 51.0 in stage 17.0 (TID 585) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:04 INFO TaskSetManager: Finished task 43.0 in stage 17.0 (TID 577) in 1845 ms on 10.0.0.133 (executor driver) (44/200) -26/04/01 08:50:04 INFO Executor: Running task 51.0 in stage 17.0 (TID 585) -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:04 INFO Executor: Finished task 44.0 in stage 17.0 (TID 578). 6444 bytes result sent to driver -26/04/01 08:50:04 INFO TaskSetManager: Starting task 52.0 in stage 17.0 (TID 586) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:04 INFO TaskSetManager: Finished task 44.0 in stage 17.0 (TID 578) in 1869 ms on 10.0.0.133 (executor driver) (45/200) -26/04/01 08:50:04 INFO Executor: Running task 52.0 in stage 17.0 (TID 586) -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:04 INFO Executor: Finished task 45.0 in stage 17.0 (TID 579). 6444 bytes result sent to driver -26/04/01 08:50:04 INFO TaskSetManager: Starting task 53.0 in stage 17.0 (TID 587) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:04 INFO TaskSetManager: Finished task 45.0 in stage 17.0 (TID 579) in 1857 ms on 10.0.0.133 (executor driver) (46/200) -26/04/01 08:50:04 INFO Executor: Running task 53.0 in stage 17.0 (TID 587) -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:04 INFO Executor: Finished task 46.0 in stage 17.0 (TID 580). 6444 bytes result sent to driver -26/04/01 08:50:04 INFO TaskSetManager: Starting task 54.0 in stage 17.0 (TID 588) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:04 INFO Executor: Running task 54.0 in stage 17.0 (TID 588) -26/04/01 08:50:04 INFO TaskSetManager: Finished task 46.0 in stage 17.0 (TID 580) in 1837 ms on 10.0.0.133 (executor driver) (47/200) -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:04 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:05 INFO Executor: Finished task 47.0 in stage 17.0 (TID 581). 6401 bytes result sent to driver -26/04/01 08:50:05 INFO TaskSetManager: Starting task 55.0 in stage 17.0 (TID 589) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:05 INFO TaskSetManager: Finished task 47.0 in stage 17.0 (TID 581) in 1852 ms on 10.0.0.133 (executor driver) (48/200) -26/04/01 08:50:05 INFO Executor: Running task 55.0 in stage 17.0 (TID 589) -26/04/01 08:50:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:05 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:05 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:06 INFO Executor: Finished task 48.0 in stage 17.0 (TID 582). 6401 bytes result sent to driver -26/04/01 08:50:06 INFO TaskSetManager: Starting task 56.0 in stage 17.0 (TID 590) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:06 INFO Executor: Running task 56.0 in stage 17.0 (TID 590) -26/04/01 08:50:06 INFO TaskSetManager: Finished task 48.0 in stage 17.0 (TID 582) in 1836 ms on 10.0.0.133 (executor driver) (49/200) -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:06 INFO Executor: Finished task 49.0 in stage 17.0 (TID 583). 6401 bytes result sent to driver -26/04/01 08:50:06 INFO TaskSetManager: Starting task 57.0 in stage 17.0 (TID 591) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:06 INFO TaskSetManager: Finished task 49.0 in stage 17.0 (TID 583) in 1840 ms on 10.0.0.133 (executor driver) (50/200) -26/04/01 08:50:06 INFO Executor: Running task 57.0 in stage 17.0 (TID 591) -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:06 INFO Executor: Finished task 50.0 in stage 17.0 (TID 584). 6401 bytes result sent to driver -26/04/01 08:50:06 INFO TaskSetManager: Starting task 58.0 in stage 17.0 (TID 592) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:06 INFO TaskSetManager: Finished task 50.0 in stage 17.0 (TID 584) in 1833 ms on 10.0.0.133 (executor driver) (51/200) -26/04/01 08:50:06 INFO Executor: Running task 58.0 in stage 17.0 (TID 592) -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:06 INFO Executor: Finished task 51.0 in stage 17.0 (TID 585). 6401 bytes result sent to driver -26/04/01 08:50:06 INFO TaskSetManager: Starting task 59.0 in stage 17.0 (TID 593) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:06 INFO Executor: Running task 59.0 in stage 17.0 (TID 593) -26/04/01 08:50:06 INFO TaskSetManager: Finished task 51.0 in stage 17.0 (TID 585) in 1849 ms on 10.0.0.133 (executor driver) (52/200) -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:06 INFO Executor: Finished task 52.0 in stage 17.0 (TID 586). 6401 bytes result sent to driver -26/04/01 08:50:06 INFO TaskSetManager: Starting task 60.0 in stage 17.0 (TID 594) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:06 INFO TaskSetManager: Finished task 52.0 in stage 17.0 (TID 586) in 1844 ms on 10.0.0.133 (executor driver) (53/200) -26/04/01 08:50:06 INFO Executor: Running task 60.0 in stage 17.0 (TID 594) -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:06 INFO Executor: Finished task 53.0 in stage 17.0 (TID 587). 6401 bytes result sent to driver -26/04/01 08:50:06 INFO TaskSetManager: Starting task 61.0 in stage 17.0 (TID 595) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:06 INFO TaskSetManager: Finished task 53.0 in stage 17.0 (TID 587) in 1862 ms on 10.0.0.133 (executor driver) (54/200) -26/04/01 08:50:06 INFO Executor: Running task 61.0 in stage 17.0 (TID 595) -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:06 INFO Executor: Finished task 54.0 in stage 17.0 (TID 588). 6401 bytes result sent to driver -26/04/01 08:50:06 INFO TaskSetManager: Starting task 62.0 in stage 17.0 (TID 596) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:06 INFO TaskSetManager: Finished task 54.0 in stage 17.0 (TID 588) in 1860 ms on 10.0.0.133 (executor driver) (55/200) -26/04/01 08:50:06 INFO Executor: Running task 62.0 in stage 17.0 (TID 596) -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:07 INFO Executor: Finished task 55.0 in stage 17.0 (TID 589). 6401 bytes result sent to driver -26/04/01 08:50:07 INFO TaskSetManager: Starting task 63.0 in stage 17.0 (TID 597) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:07 INFO TaskSetManager: Finished task 55.0 in stage 17.0 (TID 589) in 1858 ms on 10.0.0.133 (executor driver) (56/200) -26/04/01 08:50:07 INFO Executor: Running task 63.0 in stage 17.0 (TID 597) -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:07 INFO Executor: Finished task 56.0 in stage 17.0 (TID 590). 6401 bytes result sent to driver -26/04/01 08:50:07 INFO TaskSetManager: Starting task 64.0 in stage 17.0 (TID 598) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:07 INFO Executor: Running task 64.0 in stage 17.0 (TID 598) -26/04/01 08:50:07 INFO TaskSetManager: Finished task 56.0 in stage 17.0 (TID 590) in 1884 ms on 10.0.0.133 (executor driver) (57/200) -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:07 INFO Executor: Finished task 57.0 in stage 17.0 (TID 591). 6401 bytes result sent to driver -26/04/01 08:50:07 INFO TaskSetManager: Starting task 65.0 in stage 17.0 (TID 599) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:07 INFO TaskSetManager: Finished task 57.0 in stage 17.0 (TID 591) in 1919 ms on 10.0.0.133 (executor driver) (58/200) -26/04/01 08:50:07 INFO Executor: Running task 65.0 in stage 17.0 (TID 599) -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:07 INFO Executor: Finished task 58.0 in stage 17.0 (TID 592). 6401 bytes result sent to driver -26/04/01 08:50:07 INFO TaskSetManager: Starting task 66.0 in stage 17.0 (TID 600) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:07 INFO Executor: Running task 66.0 in stage 17.0 (TID 600) -26/04/01 08:50:07 INFO TaskSetManager: Finished task 58.0 in stage 17.0 (TID 592) in 1920 ms on 10.0.0.133 (executor driver) (59/200) -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:07 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:08 INFO Executor: Finished task 59.0 in stage 17.0 (TID 593). 6401 bytes result sent to driver -26/04/01 08:50:08 INFO TaskSetManager: Starting task 67.0 in stage 17.0 (TID 601) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:08 INFO TaskSetManager: Finished task 59.0 in stage 17.0 (TID 593) in 1897 ms on 10.0.0.133 (executor driver) (60/200) -26/04/01 08:50:08 INFO Executor: Running task 67.0 in stage 17.0 (TID 601) -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:08 INFO Executor: Finished task 60.0 in stage 17.0 (TID 594). 6401 bytes result sent to driver -26/04/01 08:50:08 INFO TaskSetManager: Starting task 68.0 in stage 17.0 (TID 602) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:08 INFO Executor: Running task 68.0 in stage 17.0 (TID 602) -26/04/01 08:50:08 INFO TaskSetManager: Finished task 60.0 in stage 17.0 (TID 594) in 1865 ms on 10.0.0.133 (executor driver) (61/200) -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:08 INFO Executor: Finished task 61.0 in stage 17.0 (TID 595). 6401 bytes result sent to driver -26/04/01 08:50:08 INFO Executor: Finished task 62.0 in stage 17.0 (TID 596). 6401 bytes result sent to driver -26/04/01 08:50:08 INFO TaskSetManager: Starting task 69.0 in stage 17.0 (TID 603) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:08 INFO Executor: Running task 69.0 in stage 17.0 (TID 603) -26/04/01 08:50:08 INFO TaskSetManager: Starting task 70.0 in stage 17.0 (TID 604) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:08 INFO Executor: Running task 70.0 in stage 17.0 (TID 604) -26/04/01 08:50:08 INFO TaskSetManager: Finished task 61.0 in stage 17.0 (TID 595) in 1862 ms on 10.0.0.133 (executor driver) (62/200) -26/04/01 08:50:08 INFO TaskSetManager: Finished task 62.0 in stage 17.0 (TID 596) in 1860 ms on 10.0.0.133 (executor driver) (63/200) -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:08 INFO Executor: Finished task 63.0 in stage 17.0 (TID 597). 6401 bytes result sent to driver -26/04/01 08:50:08 INFO TaskSetManager: Starting task 71.0 in stage 17.0 (TID 605) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:08 INFO TaskSetManager: Finished task 63.0 in stage 17.0 (TID 597) in 1859 ms on 10.0.0.133 (executor driver) (64/200) -26/04/01 08:50:08 INFO Executor: Running task 71.0 in stage 17.0 (TID 605) -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:08 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:09 INFO Executor: Finished task 64.0 in stage 17.0 (TID 598). 6401 bytes result sent to driver -26/04/01 08:50:09 INFO TaskSetManager: Starting task 72.0 in stage 17.0 (TID 606) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:09 INFO Executor: Running task 72.0 in stage 17.0 (TID 606) -26/04/01 08:50:09 INFO TaskSetManager: Finished task 64.0 in stage 17.0 (TID 598) in 1859 ms on 10.0.0.133 (executor driver) (65/200) -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:09 INFO Executor: Finished task 65.0 in stage 17.0 (TID 599). 6401 bytes result sent to driver -26/04/01 08:50:09 INFO TaskSetManager: Starting task 73.0 in stage 17.0 (TID 607) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:09 INFO TaskSetManager: Finished task 65.0 in stage 17.0 (TID 599) in 1845 ms on 10.0.0.133 (executor driver) (66/200) -26/04/01 08:50:09 INFO Executor: Running task 73.0 in stage 17.0 (TID 607) -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:09 INFO Executor: Finished task 66.0 in stage 17.0 (TID 600). 6401 bytes result sent to driver -26/04/01 08:50:09 INFO TaskSetManager: Starting task 74.0 in stage 17.0 (TID 608) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:09 INFO TaskSetManager: Finished task 66.0 in stage 17.0 (TID 600) in 1857 ms on 10.0.0.133 (executor driver) (67/200) -26/04/01 08:50:09 INFO Executor: Running task 74.0 in stage 17.0 (TID 608) -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:09 INFO Executor: Finished task 67.0 in stage 17.0 (TID 601). 6401 bytes result sent to driver -26/04/01 08:50:09 INFO TaskSetManager: Starting task 75.0 in stage 17.0 (TID 609) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:09 INFO TaskSetManager: Finished task 67.0 in stage 17.0 (TID 601) in 1858 ms on 10.0.0.133 (executor driver) (68/200) -26/04/01 08:50:09 INFO Executor: Running task 75.0 in stage 17.0 (TID 609) -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:10 INFO Executor: Finished task 68.0 in stage 17.0 (TID 602). 6401 bytes result sent to driver -26/04/01 08:50:10 INFO TaskSetManager: Starting task 76.0 in stage 17.0 (TID 610) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:10 INFO TaskSetManager: Finished task 68.0 in stage 17.0 (TID 602) in 1840 ms on 10.0.0.133 (executor driver) (69/200) -26/04/01 08:50:10 INFO Executor: Running task 76.0 in stage 17.0 (TID 610) -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:10 INFO Executor: Finished task 69.0 in stage 17.0 (TID 603). 6401 bytes result sent to driver -26/04/01 08:50:10 INFO TaskSetManager: Starting task 77.0 in stage 17.0 (TID 611) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:10 INFO TaskSetManager: Finished task 69.0 in stage 17.0 (TID 603) in 1861 ms on 10.0.0.133 (executor driver) (70/200) -26/04/01 08:50:10 INFO Executor: Running task 77.0 in stage 17.0 (TID 611) -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:10 INFO Executor: Finished task 70.0 in stage 17.0 (TID 604). 6401 bytes result sent to driver -26/04/01 08:50:10 INFO TaskSetManager: Starting task 78.0 in stage 17.0 (TID 612) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:10 INFO TaskSetManager: Finished task 70.0 in stage 17.0 (TID 604) in 1877 ms on 10.0.0.133 (executor driver) (71/200) -26/04/01 08:50:10 INFO Executor: Running task 78.0 in stage 17.0 (TID 612) -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:10 INFO Executor: Finished task 71.0 in stage 17.0 (TID 605). 6401 bytes result sent to driver -26/04/01 08:50:10 INFO TaskSetManager: Starting task 79.0 in stage 17.0 (TID 613) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:10 INFO TaskSetManager: Finished task 71.0 in stage 17.0 (TID 605) in 1846 ms on 10.0.0.133 (executor driver) (72/200) -26/04/01 08:50:10 INFO Executor: Running task 79.0 in stage 17.0 (TID 613) -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:11 INFO Executor: Finished task 72.0 in stage 17.0 (TID 606). 7132 bytes result sent to driver -26/04/01 08:50:11 INFO TaskSetManager: Starting task 80.0 in stage 17.0 (TID 614) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:11 INFO TaskSetManager: Finished task 72.0 in stage 17.0 (TID 606) in 1843 ms on 10.0.0.133 (executor driver) (73/200) -26/04/01 08:50:11 INFO Executor: Running task 80.0 in stage 17.0 (TID 614) -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:11 INFO Executor: Finished task 73.0 in stage 17.0 (TID 607). 7132 bytes result sent to driver -26/04/01 08:50:11 INFO TaskSetManager: Starting task 81.0 in stage 17.0 (TID 615) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:11 INFO Executor: Running task 81.0 in stage 17.0 (TID 615) -26/04/01 08:50:11 INFO TaskSetManager: Finished task 73.0 in stage 17.0 (TID 607) in 1839 ms on 10.0.0.133 (executor driver) (74/200) -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:11 INFO Executor: Finished task 74.0 in stage 17.0 (TID 608). 7132 bytes result sent to driver -26/04/01 08:50:11 INFO TaskSetManager: Starting task 82.0 in stage 17.0 (TID 616) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:11 INFO TaskSetManager: Finished task 74.0 in stage 17.0 (TID 608) in 1863 ms on 10.0.0.133 (executor driver) (75/200) -26/04/01 08:50:11 INFO Executor: Running task 82.0 in stage 17.0 (TID 616) -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:11 INFO Executor: Finished task 75.0 in stage 17.0 (TID 609). 7132 bytes result sent to driver -26/04/01 08:50:11 INFO TaskSetManager: Starting task 83.0 in stage 17.0 (TID 617) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:11 INFO TaskSetManager: Finished task 75.0 in stage 17.0 (TID 609) in 1857 ms on 10.0.0.133 (executor driver) (76/200) -26/04/01 08:50:11 INFO Executor: Running task 83.0 in stage 17.0 (TID 617) -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:11 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:12 INFO Executor: Finished task 76.0 in stage 17.0 (TID 610). 7132 bytes result sent to driver -26/04/01 08:50:12 INFO TaskSetManager: Starting task 84.0 in stage 17.0 (TID 618) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:12 INFO Executor: Running task 84.0 in stage 17.0 (TID 618) -26/04/01 08:50:12 INFO TaskSetManager: Finished task 76.0 in stage 17.0 (TID 610) in 1854 ms on 10.0.0.133 (executor driver) (77/200) -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:12 INFO Executor: Finished task 77.0 in stage 17.0 (TID 611). 7132 bytes result sent to driver -26/04/01 08:50:12 INFO TaskSetManager: Starting task 85.0 in stage 17.0 (TID 619) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:12 INFO Executor: Running task 85.0 in stage 17.0 (TID 619) -26/04/01 08:50:12 INFO TaskSetManager: Finished task 77.0 in stage 17.0 (TID 611) in 1868 ms on 10.0.0.133 (executor driver) (78/200) -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:12 INFO Executor: Finished task 78.0 in stage 17.0 (TID 612). 7132 bytes result sent to driver -26/04/01 08:50:12 INFO TaskSetManager: Starting task 86.0 in stage 17.0 (TID 620) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:12 INFO Executor: Running task 86.0 in stage 17.0 (TID 620) -26/04/01 08:50:12 INFO TaskSetManager: Finished task 78.0 in stage 17.0 (TID 612) in 1858 ms on 10.0.0.133 (executor driver) (79/200) -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:12 INFO Executor: Finished task 79.0 in stage 17.0 (TID 613). 7132 bytes result sent to driver -26/04/01 08:50:12 INFO TaskSetManager: Starting task 87.0 in stage 17.0 (TID 621) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:12 INFO Executor: Running task 87.0 in stage 17.0 (TID 621) -26/04/01 08:50:12 INFO TaskSetManager: Finished task 79.0 in stage 17.0 (TID 613) in 1882 ms on 10.0.0.133 (executor driver) (80/200) -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:13 INFO Executor: Finished task 81.0 in stage 17.0 (TID 615). 6401 bytes result sent to driver -26/04/01 08:50:13 INFO TaskSetManager: Starting task 88.0 in stage 17.0 (TID 622) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:13 INFO TaskSetManager: Finished task 81.0 in stage 17.0 (TID 615) in 1838 ms on 10.0.0.133 (executor driver) (81/200) -26/04/01 08:50:13 INFO Executor: Running task 88.0 in stage 17.0 (TID 622) -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:13 INFO Executor: Finished task 80.0 in stage 17.0 (TID 614). 6401 bytes result sent to driver -26/04/01 08:50:13 INFO TaskSetManager: Starting task 89.0 in stage 17.0 (TID 623) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:13 INFO TaskSetManager: Finished task 80.0 in stage 17.0 (TID 614) in 1863 ms on 10.0.0.133 (executor driver) (82/200) -26/04/01 08:50:13 INFO Executor: Running task 89.0 in stage 17.0 (TID 623) -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:13 INFO Executor: Finished task 82.0 in stage 17.0 (TID 616). 6401 bytes result sent to driver -26/04/01 08:50:13 INFO TaskSetManager: Starting task 90.0 in stage 17.0 (TID 624) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:13 INFO Executor: Running task 90.0 in stage 17.0 (TID 624) -26/04/01 08:50:13 INFO TaskSetManager: Finished task 82.0 in stage 17.0 (TID 616) in 1845 ms on 10.0.0.133 (executor driver) (83/200) -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:13 INFO Executor: Finished task 83.0 in stage 17.0 (TID 617). 6401 bytes result sent to driver -26/04/01 08:50:13 INFO TaskSetManager: Starting task 91.0 in stage 17.0 (TID 625) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:13 INFO TaskSetManager: Finished task 83.0 in stage 17.0 (TID 617) in 1850 ms on 10.0.0.133 (executor driver) (84/200) -26/04/01 08:50:13 INFO Executor: Running task 91.0 in stage 17.0 (TID 625) -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:14 INFO Executor: Finished task 84.0 in stage 17.0 (TID 618). 6401 bytes result sent to driver -26/04/01 08:50:14 INFO TaskSetManager: Starting task 92.0 in stage 17.0 (TID 626) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:14 INFO Executor: Running task 92.0 in stage 17.0 (TID 626) -26/04/01 08:50:14 INFO TaskSetManager: Finished task 84.0 in stage 17.0 (TID 618) in 1842 ms on 10.0.0.133 (executor driver) (85/200) -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (76.1 MiB) non-empty blocks including 208 (76.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:14 INFO Executor: Finished task 85.0 in stage 17.0 (TID 619). 6401 bytes result sent to driver -26/04/01 08:50:14 INFO TaskSetManager: Starting task 93.0 in stage 17.0 (TID 627) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:14 INFO Executor: Running task 93.0 in stage 17.0 (TID 627) -26/04/01 08:50:14 INFO TaskSetManager: Finished task 85.0 in stage 17.0 (TID 619) in 1846 ms on 10.0.0.133 (executor driver) (86/200) -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:14 INFO Executor: Finished task 86.0 in stage 17.0 (TID 620). 6401 bytes result sent to driver -26/04/01 08:50:14 INFO TaskSetManager: Starting task 94.0 in stage 17.0 (TID 628) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:14 INFO Executor: Running task 94.0 in stage 17.0 (TID 628) -26/04/01 08:50:14 INFO TaskSetManager: Finished task 86.0 in stage 17.0 (TID 620) in 1843 ms on 10.0.0.133 (executor driver) (87/200) -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:14 INFO Executor: Finished task 87.0 in stage 17.0 (TID 621). 6401 bytes result sent to driver -26/04/01 08:50:14 INFO TaskSetManager: Starting task 95.0 in stage 17.0 (TID 629) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:14 INFO TaskSetManager: Finished task 87.0 in stage 17.0 (TID 621) in 1835 ms on 10.0.0.133 (executor driver) (88/200) -26/04/01 08:50:14 INFO Executor: Running task 95.0 in stage 17.0 (TID 629) -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:14 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:15 INFO Executor: Finished task 89.0 in stage 17.0 (TID 623). 6401 bytes result sent to driver -26/04/01 08:50:15 INFO TaskSetManager: Starting task 96.0 in stage 17.0 (TID 630) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:15 INFO Executor: Running task 96.0 in stage 17.0 (TID 630) -26/04/01 08:50:15 INFO TaskSetManager: Finished task 89.0 in stage 17.0 (TID 623) in 1840 ms on 10.0.0.133 (executor driver) (89/200) -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:15 INFO Executor: Finished task 88.0 in stage 17.0 (TID 622). 6401 bytes result sent to driver -26/04/01 08:50:15 INFO TaskSetManager: Starting task 97.0 in stage 17.0 (TID 631) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:15 INFO TaskSetManager: Finished task 88.0 in stage 17.0 (TID 622) in 1851 ms on 10.0.0.133 (executor driver) (90/200) -26/04/01 08:50:15 INFO Executor: Running task 97.0 in stage 17.0 (TID 631) -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:15 INFO Executor: Finished task 90.0 in stage 17.0 (TID 624). 6401 bytes result sent to driver -26/04/01 08:50:15 INFO TaskSetManager: Starting task 98.0 in stage 17.0 (TID 632) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:15 INFO Executor: Running task 98.0 in stage 17.0 (TID 632) -26/04/01 08:50:15 INFO TaskSetManager: Finished task 90.0 in stage 17.0 (TID 624) in 1883 ms on 10.0.0.133 (executor driver) (91/200) -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:15 INFO Executor: Finished task 91.0 in stage 17.0 (TID 625). 6401 bytes result sent to driver -26/04/01 08:50:15 INFO TaskSetManager: Starting task 99.0 in stage 17.0 (TID 633) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:15 INFO TaskSetManager: Finished task 91.0 in stage 17.0 (TID 625) in 1860 ms on 10.0.0.133 (executor driver) (92/200) -26/04/01 08:50:15 INFO Executor: Running task 99.0 in stage 17.0 (TID 633) -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:15 INFO Executor: Finished task 92.0 in stage 17.0 (TID 626). 6401 bytes result sent to driver -26/04/01 08:50:15 INFO TaskSetManager: Starting task 100.0 in stage 17.0 (TID 634) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:15 INFO TaskSetManager: Finished task 92.0 in stage 17.0 (TID 626) in 1860 ms on 10.0.0.133 (executor driver) (93/200) -26/04/01 08:50:15 INFO Executor: Running task 100.0 in stage 17.0 (TID 634) -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:15 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:16 INFO Executor: Finished task 93.0 in stage 17.0 (TID 627). 6401 bytes result sent to driver -26/04/01 08:50:16 INFO TaskSetManager: Starting task 101.0 in stage 17.0 (TID 635) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:16 INFO TaskSetManager: Finished task 93.0 in stage 17.0 (TID 627) in 1853 ms on 10.0.0.133 (executor driver) (94/200) -26/04/01 08:50:16 INFO Executor: Running task 101.0 in stage 17.0 (TID 635) -26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:16 INFO Executor: Finished task 94.0 in stage 17.0 (TID 628). 6401 bytes result sent to driver -26/04/01 08:50:16 INFO TaskSetManager: Starting task 102.0 in stage 17.0 (TID 636) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:16 INFO TaskSetManager: Finished task 94.0 in stage 17.0 (TID 628) in 1858 ms on 10.0.0.133 (executor driver) (95/200) -26/04/01 08:50:16 INFO Executor: Running task 102.0 in stage 17.0 (TID 636) -26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:16 INFO Executor: Finished task 95.0 in stage 17.0 (TID 629). 6401 bytes result sent to driver -26/04/01 08:50:16 INFO TaskSetManager: Starting task 103.0 in stage 17.0 (TID 637) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:16 INFO Executor: Running task 103.0 in stage 17.0 (TID 637) -26/04/01 08:50:16 INFO TaskSetManager: Finished task 95.0 in stage 17.0 (TID 629) in 1882 ms on 10.0.0.133 (executor driver) (96/200) -26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:16 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:17 INFO Executor: Finished task 97.0 in stage 17.0 (TID 631). 6401 bytes result sent to driver -26/04/01 08:50:17 INFO TaskSetManager: Starting task 104.0 in stage 17.0 (TID 638) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:17 INFO Executor: Running task 104.0 in stage 17.0 (TID 638) -26/04/01 08:50:17 INFO TaskSetManager: Finished task 97.0 in stage 17.0 (TID 631) in 1891 ms on 10.0.0.133 (executor driver) (97/200) -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:17 INFO Executor: Finished task 96.0 in stage 17.0 (TID 630). 6401 bytes result sent to driver -26/04/01 08:50:17 INFO TaskSetManager: Starting task 105.0 in stage 17.0 (TID 639) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:17 INFO TaskSetManager: Finished task 96.0 in stage 17.0 (TID 630) in 1901 ms on 10.0.0.133 (executor driver) (98/200) -26/04/01 08:50:17 INFO Executor: Running task 105.0 in stage 17.0 (TID 639) -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:17 INFO Executor: Finished task 98.0 in stage 17.0 (TID 632). 6401 bytes result sent to driver -26/04/01 08:50:17 INFO TaskSetManager: Starting task 106.0 in stage 17.0 (TID 640) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:17 INFO TaskSetManager: Finished task 98.0 in stage 17.0 (TID 632) in 1873 ms on 10.0.0.133 (executor driver) (99/200) -26/04/01 08:50:17 INFO Executor: Running task 106.0 in stage 17.0 (TID 640) -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:17 INFO Executor: Finished task 99.0 in stage 17.0 (TID 633). 6401 bytes result sent to driver -26/04/01 08:50:17 INFO TaskSetManager: Starting task 107.0 in stage 17.0 (TID 641) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:17 INFO TaskSetManager: Finished task 99.0 in stage 17.0 (TID 633) in 1878 ms on 10.0.0.133 (executor driver) (100/200) -26/04/01 08:50:17 INFO Executor: Running task 107.0 in stage 17.0 (TID 641) -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:17 INFO Executor: Finished task 100.0 in stage 17.0 (TID 634). 6401 bytes result sent to driver -26/04/01 08:50:17 INFO TaskSetManager: Starting task 108.0 in stage 17.0 (TID 642) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:17 INFO TaskSetManager: Finished task 100.0 in stage 17.0 (TID 634) in 1879 ms on 10.0.0.133 (executor driver) (101/200) -26/04/01 08:50:17 INFO Executor: Running task 108.0 in stage 17.0 (TID 642) -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:18 INFO Executor: Finished task 101.0 in stage 17.0 (TID 635). 6401 bytes result sent to driver -26/04/01 08:50:18 INFO TaskSetManager: Starting task 109.0 in stage 17.0 (TID 643) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:18 INFO TaskSetManager: Finished task 101.0 in stage 17.0 (TID 635) in 1883 ms on 10.0.0.133 (executor driver) (102/200) -26/04/01 08:50:18 INFO Executor: Running task 109.0 in stage 17.0 (TID 643) -26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:18 INFO Executor: Finished task 102.0 in stage 17.0 (TID 636). 6401 bytes result sent to driver -26/04/01 08:50:18 INFO TaskSetManager: Starting task 110.0 in stage 17.0 (TID 644) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:18 INFO Executor: Running task 110.0 in stage 17.0 (TID 644) -26/04/01 08:50:18 INFO TaskSetManager: Finished task 102.0 in stage 17.0 (TID 636) in 1891 ms on 10.0.0.133 (executor driver) (103/200) -26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:18 INFO Executor: Finished task 103.0 in stage 17.0 (TID 637). 6401 bytes result sent to driver -26/04/01 08:50:18 INFO TaskSetManager: Starting task 111.0 in stage 17.0 (TID 645) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:18 INFO TaskSetManager: Finished task 103.0 in stage 17.0 (TID 637) in 1857 ms on 10.0.0.133 (executor driver) (104/200) -26/04/01 08:50:18 INFO Executor: Running task 111.0 in stage 17.0 (TID 645) -26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:18 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:19 INFO Executor: Finished task 105.0 in stage 17.0 (TID 639). 6401 bytes result sent to driver -26/04/01 08:50:19 INFO TaskSetManager: Starting task 112.0 in stage 17.0 (TID 646) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:19 INFO TaskSetManager: Finished task 105.0 in stage 17.0 (TID 639) in 1853 ms on 10.0.0.133 (executor driver) (105/200) -26/04/01 08:50:19 INFO Executor: Running task 112.0 in stage 17.0 (TID 646) -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:19 INFO Executor: Finished task 104.0 in stage 17.0 (TID 638). 6401 bytes result sent to driver -26/04/01 08:50:19 INFO TaskSetManager: Starting task 113.0 in stage 17.0 (TID 647) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:19 INFO Executor: Running task 113.0 in stage 17.0 (TID 647) -26/04/01 08:50:19 INFO TaskSetManager: Finished task 104.0 in stage 17.0 (TID 638) in 1863 ms on 10.0.0.133 (executor driver) (106/200) -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:19 INFO Executor: Finished task 106.0 in stage 17.0 (TID 640). 6401 bytes result sent to driver -26/04/01 08:50:19 INFO TaskSetManager: Starting task 114.0 in stage 17.0 (TID 648) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:19 INFO Executor: Running task 114.0 in stage 17.0 (TID 648) -26/04/01 08:50:19 INFO TaskSetManager: Finished task 106.0 in stage 17.0 (TID 640) in 1873 ms on 10.0.0.133 (executor driver) (107/200) -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:19 INFO Executor: Finished task 107.0 in stage 17.0 (TID 641). 6401 bytes result sent to driver -26/04/01 08:50:19 INFO TaskSetManager: Starting task 115.0 in stage 17.0 (TID 649) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:19 INFO TaskSetManager: Finished task 107.0 in stage 17.0 (TID 641) in 1878 ms on 10.0.0.133 (executor driver) (108/200) -26/04/01 08:50:19 INFO Executor: Running task 115.0 in stage 17.0 (TID 649) -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:19 INFO Executor: Finished task 108.0 in stage 17.0 (TID 642). 6401 bytes result sent to driver -26/04/01 08:50:19 INFO TaskSetManager: Starting task 116.0 in stage 17.0 (TID 650) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:19 INFO TaskSetManager: Finished task 108.0 in stage 17.0 (TID 642) in 1895 ms on 10.0.0.133 (executor driver) (109/200) -26/04/01 08:50:19 INFO Executor: Running task 116.0 in stage 17.0 (TID 650) -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:19 INFO Executor: Finished task 109.0 in stage 17.0 (TID 643). 6401 bytes result sent to driver -26/04/01 08:50:19 INFO TaskSetManager: Starting task 117.0 in stage 17.0 (TID 651) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:19 INFO TaskSetManager: Finished task 109.0 in stage 17.0 (TID 643) in 1892 ms on 10.0.0.133 (executor driver) (110/200) -26/04/01 08:50:19 INFO Executor: Running task 117.0 in stage 17.0 (TID 651) -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:19 INFO Executor: Finished task 110.0 in stage 17.0 (TID 644). 6401 bytes result sent to driver -26/04/01 08:50:19 INFO TaskSetManager: Starting task 118.0 in stage 17.0 (TID 652) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:19 INFO Executor: Running task 118.0 in stage 17.0 (TID 652) -26/04/01 08:50:19 INFO TaskSetManager: Finished task 110.0 in stage 17.0 (TID 644) in 1881 ms on 10.0.0.133 (executor driver) (111/200) -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (76.0 MiB) non-empty blocks including 208 (76.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Getting 208 (76.0 MiB) non-empty blocks including 208 (76.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:20 INFO Executor: Finished task 111.0 in stage 17.0 (TID 645). 6401 bytes result sent to driver -26/04/01 08:50:20 INFO TaskSetManager: Starting task 119.0 in stage 17.0 (TID 653) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:20 INFO TaskSetManager: Finished task 111.0 in stage 17.0 (TID 645) in 1871 ms on 10.0.0.133 (executor driver) (112/200) -26/04/01 08:50:20 INFO Executor: Running task 119.0 in stage 17.0 (TID 653) -26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:20 INFO Executor: Finished task 112.0 in stage 17.0 (TID 646). 6401 bytes result sent to driver -26/04/01 08:50:20 INFO TaskSetManager: Starting task 120.0 in stage 17.0 (TID 654) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:20 INFO TaskSetManager: Finished task 112.0 in stage 17.0 (TID 646) in 1867 ms on 10.0.0.133 (executor driver) (113/200) -26/04/01 08:50:20 INFO Executor: Running task 120.0 in stage 17.0 (TID 654) -26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:20 INFO Executor: Finished task 113.0 in stage 17.0 (TID 647). 6401 bytes result sent to driver -26/04/01 08:50:20 INFO TaskSetManager: Starting task 121.0 in stage 17.0 (TID 655) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:20 INFO Executor: Running task 121.0 in stage 17.0 (TID 655) -26/04/01 08:50:20 INFO TaskSetManager: Finished task 113.0 in stage 17.0 (TID 647) in 1865 ms on 10.0.0.133 (executor driver) (114/200) -26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:20 INFO Executor: Finished task 114.0 in stage 17.0 (TID 648). 6401 bytes result sent to driver -26/04/01 08:50:20 INFO TaskSetManager: Starting task 122.0 in stage 17.0 (TID 656) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:21 INFO TaskSetManager: Finished task 114.0 in stage 17.0 (TID 648) in 1847 ms on 10.0.0.133 (executor driver) (115/200) -26/04/01 08:50:21 INFO Executor: Running task 122.0 in stage 17.0 (TID 656) -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:21 INFO Executor: Finished task 115.0 in stage 17.0 (TID 649). 7132 bytes result sent to driver -26/04/01 08:50:21 INFO TaskSetManager: Starting task 123.0 in stage 17.0 (TID 657) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:21 INFO TaskSetManager: Finished task 115.0 in stage 17.0 (TID 649) in 1864 ms on 10.0.0.133 (executor driver) (116/200) -26/04/01 08:50:21 INFO Executor: Running task 123.0 in stage 17.0 (TID 657) -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:21 INFO Executor: Finished task 116.0 in stage 17.0 (TID 650). 7132 bytes result sent to driver -26/04/01 08:50:21 INFO TaskSetManager: Starting task 124.0 in stage 17.0 (TID 658) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:21 INFO TaskSetManager: Finished task 116.0 in stage 17.0 (TID 650) in 1855 ms on 10.0.0.133 (executor driver) (117/200) -26/04/01 08:50:21 INFO Executor: Running task 124.0 in stage 17.0 (TID 658) -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:21 INFO Executor: Finished task 117.0 in stage 17.0 (TID 651). 7132 bytes result sent to driver -26/04/01 08:50:21 INFO TaskSetManager: Starting task 125.0 in stage 17.0 (TID 659) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:21 INFO TaskSetManager: Finished task 117.0 in stage 17.0 (TID 651) in 1853 ms on 10.0.0.133 (executor driver) (118/200) -26/04/01 08:50:21 INFO Executor: Running task 125.0 in stage 17.0 (TID 659) -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:21 INFO Executor: Finished task 118.0 in stage 17.0 (TID 652). 7132 bytes result sent to driver -26/04/01 08:50:21 INFO TaskSetManager: Starting task 126.0 in stage 17.0 (TID 660) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:21 INFO TaskSetManager: Finished task 118.0 in stage 17.0 (TID 652) in 1860 ms on 10.0.0.133 (executor driver) (119/200) -26/04/01 08:50:21 INFO Executor: Running task 126.0 in stage 17.0 (TID 660) -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:21 INFO Executor: Finished task 119.0 in stage 17.0 (TID 653). 7132 bytes result sent to driver -26/04/01 08:50:21 INFO TaskSetManager: Starting task 127.0 in stage 17.0 (TID 661) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:21 INFO TaskSetManager: Finished task 119.0 in stage 17.0 (TID 653) in 1837 ms on 10.0.0.133 (executor driver) (120/200) -26/04/01 08:50:21 INFO Executor: Running task 127.0 in stage 17.0 (TID 661) -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:21 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:22 INFO Executor: Finished task 120.0 in stage 17.0 (TID 654). 7132 bytes result sent to driver -26/04/01 08:50:22 INFO TaskSetManager: Starting task 128.0 in stage 17.0 (TID 662) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:22 INFO Executor: Running task 128.0 in stage 17.0 (TID 662) -26/04/01 08:50:22 INFO TaskSetManager: Finished task 120.0 in stage 17.0 (TID 654) in 1835 ms on 10.0.0.133 (executor driver) (121/200) -26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:22 INFO Executor: Finished task 121.0 in stage 17.0 (TID 655). 7132 bytes result sent to driver -26/04/01 08:50:22 INFO TaskSetManager: Starting task 129.0 in stage 17.0 (TID 663) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:22 INFO Executor: Running task 129.0 in stage 17.0 (TID 663) -26/04/01 08:50:22 INFO TaskSetManager: Finished task 121.0 in stage 17.0 (TID 655) in 1833 ms on 10.0.0.133 (executor driver) (122/200) -26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:22 INFO Executor: Finished task 122.0 in stage 17.0 (TID 656). 7132 bytes result sent to driver -26/04/01 08:50:22 INFO TaskSetManager: Starting task 130.0 in stage 17.0 (TID 664) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:22 INFO Executor: Running task 130.0 in stage 17.0 (TID 664) -26/04/01 08:50:22 INFO TaskSetManager: Finished task 122.0 in stage 17.0 (TID 656) in 1835 ms on 10.0.0.133 (executor driver) (123/200) -26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:22 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:23 INFO Executor: Finished task 123.0 in stage 17.0 (TID 657). 6401 bytes result sent to driver -26/04/01 08:50:23 INFO TaskSetManager: Starting task 131.0 in stage 17.0 (TID 665) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:23 INFO TaskSetManager: Finished task 123.0 in stage 17.0 (TID 657) in 1835 ms on 10.0.0.133 (executor driver) (124/200) -26/04/01 08:50:23 INFO Executor: Running task 131.0 in stage 17.0 (TID 665) -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:23 INFO Executor: Finished task 124.0 in stage 17.0 (TID 658). 6401 bytes result sent to driver -26/04/01 08:50:23 INFO TaskSetManager: Starting task 132.0 in stage 17.0 (TID 666) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:23 INFO Executor: Running task 132.0 in stage 17.0 (TID 666) -26/04/01 08:50:23 INFO TaskSetManager: Finished task 124.0 in stage 17.0 (TID 658) in 1837 ms on 10.0.0.133 (executor driver) (125/200) -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:23 INFO Executor: Finished task 126.0 in stage 17.0 (TID 660). 6401 bytes result sent to driver -26/04/01 08:50:23 INFO TaskSetManager: Starting task 133.0 in stage 17.0 (TID 667) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:23 INFO TaskSetManager: Finished task 126.0 in stage 17.0 (TID 660) in 1835 ms on 10.0.0.133 (executor driver) (126/200) -26/04/01 08:50:23 INFO Executor: Running task 133.0 in stage 17.0 (TID 667) -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:23 INFO Executor: Finished task 125.0 in stage 17.0 (TID 659). 6401 bytes result sent to driver -26/04/01 08:50:23 INFO TaskSetManager: Starting task 134.0 in stage 17.0 (TID 668) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:23 INFO TaskSetManager: Finished task 125.0 in stage 17.0 (TID 659) in 1853 ms on 10.0.0.133 (executor driver) (127/200) -26/04/01 08:50:23 INFO Executor: Running task 134.0 in stage 17.0 (TID 668) -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:23 INFO Executor: Finished task 127.0 in stage 17.0 (TID 661). 6401 bytes result sent to driver -26/04/01 08:50:23 INFO TaskSetManager: Starting task 135.0 in stage 17.0 (TID 669) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:23 INFO TaskSetManager: Finished task 127.0 in stage 17.0 (TID 661) in 1847 ms on 10.0.0.133 (executor driver) (128/200) -26/04/01 08:50:23 INFO Executor: Running task 135.0 in stage 17.0 (TID 669) -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:23 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:24 INFO Executor: Finished task 129.0 in stage 17.0 (TID 663). 6401 bytes result sent to driver -26/04/01 08:50:24 INFO TaskSetManager: Starting task 136.0 in stage 17.0 (TID 670) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:24 INFO Executor: Running task 136.0 in stage 17.0 (TID 670) -26/04/01 08:50:24 INFO TaskSetManager: Finished task 129.0 in stage 17.0 (TID 663) in 1869 ms on 10.0.0.133 (executor driver) (129/200) -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:24 INFO Executor: Finished task 128.0 in stage 17.0 (TID 662). 6401 bytes result sent to driver -26/04/01 08:50:24 INFO TaskSetManager: Starting task 137.0 in stage 17.0 (TID 671) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:24 INFO Executor: Running task 137.0 in stage 17.0 (TID 671) -26/04/01 08:50:24 INFO TaskSetManager: Finished task 128.0 in stage 17.0 (TID 662) in 1878 ms on 10.0.0.133 (executor driver) (130/200) -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:24 INFO Executor: Finished task 130.0 in stage 17.0 (TID 664). 6401 bytes result sent to driver -26/04/01 08:50:24 INFO TaskSetManager: Starting task 138.0 in stage 17.0 (TID 672) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:24 INFO Executor: Running task 138.0 in stage 17.0 (TID 672) -26/04/01 08:50:24 INFO TaskSetManager: Finished task 130.0 in stage 17.0 (TID 664) in 1865 ms on 10.0.0.133 (executor driver) (131/200) -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:24 INFO Executor: Finished task 131.0 in stage 17.0 (TID 665). 6401 bytes result sent to driver -26/04/01 08:50:24 INFO TaskSetManager: Starting task 139.0 in stage 17.0 (TID 673) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:24 INFO Executor: Running task 139.0 in stage 17.0 (TID 673) -26/04/01 08:50:24 INFO TaskSetManager: Finished task 131.0 in stage 17.0 (TID 665) in 1853 ms on 10.0.0.133 (executor driver) (132/200) -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:24 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:25 INFO Executor: Finished task 132.0 in stage 17.0 (TID 666). 6401 bytes result sent to driver -26/04/01 08:50:25 INFO TaskSetManager: Starting task 140.0 in stage 17.0 (TID 674) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:25 INFO Executor: Running task 140.0 in stage 17.0 (TID 674) -26/04/01 08:50:25 INFO TaskSetManager: Finished task 132.0 in stage 17.0 (TID 666) in 1832 ms on 10.0.0.133 (executor driver) (133/200) -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:25 INFO Executor: Finished task 134.0 in stage 17.0 (TID 668). 6401 bytes result sent to driver -26/04/01 08:50:25 INFO Executor: Finished task 133.0 in stage 17.0 (TID 667). 6401 bytes result sent to driver -26/04/01 08:50:25 INFO TaskSetManager: Starting task 141.0 in stage 17.0 (TID 675) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:25 INFO Executor: Running task 141.0 in stage 17.0 (TID 675) -26/04/01 08:50:25 INFO TaskSetManager: Starting task 142.0 in stage 17.0 (TID 676) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:25 INFO TaskSetManager: Finished task 134.0 in stage 17.0 (TID 668) in 1822 ms on 10.0.0.133 (executor driver) (134/200) -26/04/01 08:50:25 INFO Executor: Running task 142.0 in stage 17.0 (TID 676) -26/04/01 08:50:25 INFO TaskSetManager: Finished task 133.0 in stage 17.0 (TID 667) in 1828 ms on 10.0.0.133 (executor driver) (135/200) -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:25 INFO Executor: Finished task 135.0 in stage 17.0 (TID 669). 6401 bytes result sent to driver -26/04/01 08:50:25 INFO TaskSetManager: Starting task 143.0 in stage 17.0 (TID 677) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:25 INFO Executor: Running task 143.0 in stage 17.0 (TID 677) -26/04/01 08:50:25 INFO TaskSetManager: Finished task 135.0 in stage 17.0 (TID 669) in 1833 ms on 10.0.0.133 (executor driver) (136/200) -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:25 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:26 INFO Executor: Finished task 136.0 in stage 17.0 (TID 670). 6401 bytes result sent to driver -26/04/01 08:50:26 INFO TaskSetManager: Starting task 144.0 in stage 17.0 (TID 678) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:26 INFO Executor: Running task 144.0 in stage 17.0 (TID 678) -26/04/01 08:50:26 INFO TaskSetManager: Finished task 136.0 in stage 17.0 (TID 670) in 1829 ms on 10.0.0.133 (executor driver) (137/200) -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:26 INFO Executor: Finished task 137.0 in stage 17.0 (TID 671). 6401 bytes result sent to driver -26/04/01 08:50:26 INFO TaskSetManager: Starting task 145.0 in stage 17.0 (TID 679) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:26 INFO TaskSetManager: Finished task 137.0 in stage 17.0 (TID 671) in 1836 ms on 10.0.0.133 (executor driver) (138/200) -26/04/01 08:50:26 INFO Executor: Running task 145.0 in stage 17.0 (TID 679) -26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:26 INFO Executor: Finished task 138.0 in stage 17.0 (TID 672). 6401 bytes result sent to driver -26/04/01 08:50:26 INFO TaskSetManager: Starting task 146.0 in stage 17.0 (TID 680) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:26 INFO TaskSetManager: Finished task 138.0 in stage 17.0 (TID 672) in 1836 ms on 10.0.0.133 (executor driver) (139/200) -26/04/01 08:50:26 INFO Executor: Running task 146.0 in stage 17.0 (TID 680) -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:26 INFO Executor: Finished task 139.0 in stage 17.0 (TID 673). 6401 bytes result sent to driver -26/04/01 08:50:26 INFO TaskSetManager: Starting task 147.0 in stage 17.0 (TID 681) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:26 INFO TaskSetManager: Finished task 139.0 in stage 17.0 (TID 673) in 1851 ms on 10.0.0.133 (executor driver) (140/200) -26/04/01 08:50:26 INFO Executor: Running task 147.0 in stage 17.0 (TID 681) -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:27 INFO Executor: Finished task 140.0 in stage 17.0 (TID 674). 6401 bytes result sent to driver -26/04/01 08:50:27 INFO TaskSetManager: Starting task 148.0 in stage 17.0 (TID 682) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:27 INFO TaskSetManager: Finished task 140.0 in stage 17.0 (TID 674) in 1835 ms on 10.0.0.133 (executor driver) (141/200) -26/04/01 08:50:27 INFO Executor: Running task 148.0 in stage 17.0 (TID 682) -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:27 INFO Executor: Finished task 141.0 in stage 17.0 (TID 675). 6401 bytes result sent to driver -26/04/01 08:50:27 INFO TaskSetManager: Starting task 149.0 in stage 17.0 (TID 683) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:27 INFO TaskSetManager: Finished task 141.0 in stage 17.0 (TID 675) in 1845 ms on 10.0.0.133 (executor driver) (142/200) -26/04/01 08:50:27 INFO Executor: Running task 149.0 in stage 17.0 (TID 683) -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:27 INFO Executor: Finished task 142.0 in stage 17.0 (TID 676). 6401 bytes result sent to driver -26/04/01 08:50:27 INFO TaskSetManager: Starting task 150.0 in stage 17.0 (TID 684) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:27 INFO TaskSetManager: Finished task 142.0 in stage 17.0 (TID 676) in 1847 ms on 10.0.0.133 (executor driver) (143/200) -26/04/01 08:50:27 INFO Executor: Running task 150.0 in stage 17.0 (TID 684) -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:27 INFO Executor: Finished task 143.0 in stage 17.0 (TID 677). 6401 bytes result sent to driver -26/04/01 08:50:27 INFO TaskSetManager: Starting task 151.0 in stage 17.0 (TID 685) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:27 INFO TaskSetManager: Finished task 143.0 in stage 17.0 (TID 677) in 1849 ms on 10.0.0.133 (executor driver) (144/200) -26/04/01 08:50:27 INFO Executor: Running task 151.0 in stage 17.0 (TID 685) -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:28 INFO Executor: Finished task 144.0 in stage 17.0 (TID 678). 6401 bytes result sent to driver -26/04/01 08:50:28 INFO TaskSetManager: Starting task 152.0 in stage 17.0 (TID 686) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:28 INFO TaskSetManager: Finished task 144.0 in stage 17.0 (TID 678) in 1846 ms on 10.0.0.133 (executor driver) (145/200) -26/04/01 08:50:28 INFO Executor: Running task 152.0 in stage 17.0 (TID 686) -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:28 INFO Executor: Finished task 145.0 in stage 17.0 (TID 679). 6401 bytes result sent to driver -26/04/01 08:50:28 INFO TaskSetManager: Starting task 153.0 in stage 17.0 (TID 687) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:28 INFO TaskSetManager: Finished task 145.0 in stage 17.0 (TID 679) in 1839 ms on 10.0.0.133 (executor driver) (146/200) -26/04/01 08:50:28 INFO Executor: Running task 153.0 in stage 17.0 (TID 687) -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:28 INFO Executor: Finished task 146.0 in stage 17.0 (TID 680). 6401 bytes result sent to driver -26/04/01 08:50:28 INFO TaskSetManager: Starting task 154.0 in stage 17.0 (TID 688) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:28 INFO TaskSetManager: Finished task 146.0 in stage 17.0 (TID 680) in 1846 ms on 10.0.0.133 (executor driver) (147/200) -26/04/01 08:50:28 INFO Executor: Running task 154.0 in stage 17.0 (TID 688) -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:28 INFO Executor: Finished task 147.0 in stage 17.0 (TID 681). 6401 bytes result sent to driver -26/04/01 08:50:28 INFO TaskSetManager: Starting task 155.0 in stage 17.0 (TID 689) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:28 INFO TaskSetManager: Finished task 147.0 in stage 17.0 (TID 681) in 1875 ms on 10.0.0.133 (executor driver) (148/200) -26/04/01 08:50:28 INFO Executor: Running task 155.0 in stage 17.0 (TID 689) -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:28 INFO Executor: Finished task 148.0 in stage 17.0 (TID 682). 6401 bytes result sent to driver -26/04/01 08:50:28 INFO TaskSetManager: Starting task 156.0 in stage 17.0 (TID 690) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:28 INFO TaskSetManager: Finished task 148.0 in stage 17.0 (TID 682) in 1874 ms on 10.0.0.133 (executor driver) (149/200) -26/04/01 08:50:28 INFO Executor: Running task 156.0 in stage 17.0 (TID 690) -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:29 INFO Executor: Finished task 150.0 in stage 17.0 (TID 684). 6401 bytes result sent to driver -26/04/01 08:50:29 INFO TaskSetManager: Starting task 157.0 in stage 17.0 (TID 691) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:29 INFO Executor: Finished task 149.0 in stage 17.0 (TID 683). 6401 bytes result sent to driver -26/04/01 08:50:29 INFO Executor: Running task 157.0 in stage 17.0 (TID 691) -26/04/01 08:50:29 INFO TaskSetManager: Starting task 158.0 in stage 17.0 (TID 692) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:29 INFO TaskSetManager: Finished task 150.0 in stage 17.0 (TID 684) in 1871 ms on 10.0.0.133 (executor driver) (150/200) -26/04/01 08:50:29 INFO Executor: Running task 158.0 in stage 17.0 (TID 692) -26/04/01 08:50:29 INFO TaskSetManager: Finished task 149.0 in stage 17.0 (TID 683) in 1874 ms on 10.0.0.133 (executor driver) (151/200) -26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:29 INFO Executor: Finished task 151.0 in stage 17.0 (TID 685). 6401 bytes result sent to driver -26/04/01 08:50:29 INFO TaskSetManager: Starting task 159.0 in stage 17.0 (TID 693) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:29 INFO Executor: Running task 159.0 in stage 17.0 (TID 693) -26/04/01 08:50:29 INFO TaskSetManager: Finished task 151.0 in stage 17.0 (TID 685) in 1858 ms on 10.0.0.133 (executor driver) (152/200) -26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:30 INFO Executor: Finished task 153.0 in stage 17.0 (TID 687). 6401 bytes result sent to driver -26/04/01 08:50:30 INFO TaskSetManager: Starting task 160.0 in stage 17.0 (TID 694) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:30 INFO TaskSetManager: Finished task 153.0 in stage 17.0 (TID 687) in 1899 ms on 10.0.0.133 (executor driver) (153/200) -26/04/01 08:50:30 INFO Executor: Running task 160.0 in stage 17.0 (TID 694) -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:30 INFO Executor: Finished task 152.0 in stage 17.0 (TID 686). 6401 bytes result sent to driver -26/04/01 08:50:30 INFO TaskSetManager: Starting task 161.0 in stage 17.0 (TID 695) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:30 INFO Executor: Running task 161.0 in stage 17.0 (TID 695) -26/04/01 08:50:30 INFO TaskSetManager: Finished task 152.0 in stage 17.0 (TID 686) in 1907 ms on 10.0.0.133 (executor driver) (154/200) -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:30 INFO Executor: Finished task 154.0 in stage 17.0 (TID 688). 6401 bytes result sent to driver -26/04/01 08:50:30 INFO TaskSetManager: Starting task 162.0 in stage 17.0 (TID 696) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:30 INFO TaskSetManager: Finished task 154.0 in stage 17.0 (TID 688) in 1871 ms on 10.0.0.133 (executor driver) (155/200) -26/04/01 08:50:30 INFO Executor: Running task 162.0 in stage 17.0 (TID 696) -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:30 INFO Executor: Finished task 155.0 in stage 17.0 (TID 689). 6401 bytes result sent to driver -26/04/01 08:50:30 INFO TaskSetManager: Starting task 163.0 in stage 17.0 (TID 697) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:30 INFO Executor: Running task 163.0 in stage 17.0 (TID 697) -26/04/01 08:50:30 INFO TaskSetManager: Finished task 155.0 in stage 17.0 (TID 689) in 1851 ms on 10.0.0.133 (executor driver) (156/200) -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:30 INFO Executor: Finished task 156.0 in stage 17.0 (TID 690). 6401 bytes result sent to driver -26/04/01 08:50:30 INFO TaskSetManager: Starting task 164.0 in stage 17.0 (TID 698) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:30 INFO Executor: Running task 164.0 in stage 17.0 (TID 698) -26/04/01 08:50:30 INFO TaskSetManager: Finished task 156.0 in stage 17.0 (TID 690) in 1867 ms on 10.0.0.133 (executor driver) (157/200) -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:30 INFO Executor: Finished task 157.0 in stage 17.0 (TID 691). 6401 bytes result sent to driver -26/04/01 08:50:30 INFO TaskSetManager: Starting task 165.0 in stage 17.0 (TID 699) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:30 INFO TaskSetManager: Finished task 157.0 in stage 17.0 (TID 691) in 1840 ms on 10.0.0.133 (executor driver) (158/200) -26/04/01 08:50:30 INFO Executor: Running task 165.0 in stage 17.0 (TID 699) -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:30 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:30 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:31 INFO Executor: Finished task 158.0 in stage 17.0 (TID 692). 6401 bytes result sent to driver -26/04/01 08:50:31 INFO TaskSetManager: Starting task 166.0 in stage 17.0 (TID 700) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:31 INFO TaskSetManager: Finished task 158.0 in stage 17.0 (TID 692) in 1863 ms on 10.0.0.133 (executor driver) (159/200) -26/04/01 08:50:31 INFO Executor: Running task 166.0 in stage 17.0 (TID 700) -26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:31 INFO Executor: Finished task 159.0 in stage 17.0 (TID 693). 7132 bytes result sent to driver -26/04/01 08:50:31 INFO TaskSetManager: Starting task 167.0 in stage 17.0 (TID 701) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:31 INFO TaskSetManager: Finished task 159.0 in stage 17.0 (TID 693) in 1868 ms on 10.0.0.133 (executor driver) (160/200) -26/04/01 08:50:31 INFO Executor: Running task 167.0 in stage 17.0 (TID 701) -26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:31 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO Executor: Finished task 161.0 in stage 17.0 (TID 695). 7132 bytes result sent to driver -26/04/01 08:50:32 INFO TaskSetManager: Starting task 168.0 in stage 17.0 (TID 702) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:32 INFO TaskSetManager: Finished task 161.0 in stage 17.0 (TID 695) in 1859 ms on 10.0.0.133 (executor driver) (161/200) -26/04/01 08:50:32 INFO Executor: Running task 168.0 in stage 17.0 (TID 702) -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO Executor: Finished task 160.0 in stage 17.0 (TID 694). 7132 bytes result sent to driver -26/04/01 08:50:32 INFO TaskSetManager: Starting task 169.0 in stage 17.0 (TID 703) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:32 INFO TaskSetManager: Finished task 160.0 in stage 17.0 (TID 694) in 1871 ms on 10.0.0.133 (executor driver) (162/200) -26/04/01 08:50:32 INFO Executor: Running task 169.0 in stage 17.0 (TID 703) -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO Executor: Finished task 162.0 in stage 17.0 (TID 696). 7132 bytes result sent to driver -26/04/01 08:50:32 INFO TaskSetManager: Starting task 170.0 in stage 17.0 (TID 704) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:32 INFO Executor: Running task 170.0 in stage 17.0 (TID 704) -26/04/01 08:50:32 INFO TaskSetManager: Finished task 162.0 in stage 17.0 (TID 696) in 1866 ms on 10.0.0.133 (executor driver) (163/200) -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.4 MiB) non-empty blocks including 208 (75.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO Executor: Finished task 163.0 in stage 17.0 (TID 697). 7132 bytes result sent to driver -26/04/01 08:50:32 INFO TaskSetManager: Starting task 171.0 in stage 17.0 (TID 705) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:32 INFO Executor: Running task 171.0 in stage 17.0 (TID 705) -26/04/01 08:50:32 INFO TaskSetManager: Finished task 163.0 in stage 17.0 (TID 697) in 1842 ms on 10.0.0.133 (executor driver) (164/200) -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO Executor: Finished task 164.0 in stage 17.0 (TID 698). 7132 bytes result sent to driver -26/04/01 08:50:32 INFO TaskSetManager: Starting task 172.0 in stage 17.0 (TID 706) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:32 INFO Executor: Running task 172.0 in stage 17.0 (TID 706) -26/04/01 08:50:32 INFO TaskSetManager: Finished task 164.0 in stage 17.0 (TID 698) in 1848 ms on 10.0.0.133 (executor driver) (165/200) -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO Executor: Finished task 165.0 in stage 17.0 (TID 699). 7132 bytes result sent to driver -26/04/01 08:50:32 INFO TaskSetManager: Starting task 173.0 in stage 17.0 (TID 707) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:32 INFO TaskSetManager: Finished task 165.0 in stage 17.0 (TID 699) in 1832 ms on 10.0.0.133 (executor driver) (166/200) -26/04/01 08:50:32 INFO Executor: Running task 173.0 in stage 17.0 (TID 707) -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO Executor: Finished task 166.0 in stage 17.0 (TID 700). 7132 bytes result sent to driver -26/04/01 08:50:32 INFO TaskSetManager: Starting task 174.0 in stage 17.0 (TID 708) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:32 INFO TaskSetManager: Finished task 166.0 in stage 17.0 (TID 700) in 1850 ms on 10.0.0.133 (executor driver) (167/200) -26/04/01 08:50:32 INFO Executor: Running task 174.0 in stage 17.0 (TID 708) -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO Executor: Finished task 167.0 in stage 17.0 (TID 701). 6401 bytes result sent to driver -26/04/01 08:50:32 INFO TaskSetManager: Starting task 175.0 in stage 17.0 (TID 709) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:32 INFO TaskSetManager: Finished task 167.0 in stage 17.0 (TID 701) in 1827 ms on 10.0.0.133 (executor driver) (168/200) -26/04/01 08:50:32 INFO Executor: Running task 175.0 in stage 17.0 (TID 709) -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:32 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:33 INFO Executor: Finished task 168.0 in stage 17.0 (TID 702). 6401 bytes result sent to driver -26/04/01 08:50:33 INFO TaskSetManager: Starting task 176.0 in stage 17.0 (TID 710) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:33 INFO TaskSetManager: Finished task 168.0 in stage 17.0 (TID 702) in 1819 ms on 10.0.0.133 (executor driver) (169/200) -26/04/01 08:50:33 INFO Executor: Running task 176.0 in stage 17.0 (TID 710) -26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:33 INFO Executor: Finished task 169.0 in stage 17.0 (TID 703). 6401 bytes result sent to driver -26/04/01 08:50:33 INFO TaskSetManager: Starting task 177.0 in stage 17.0 (TID 711) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:33 INFO Executor: Running task 177.0 in stage 17.0 (TID 711) -26/04/01 08:50:33 INFO TaskSetManager: Finished task 169.0 in stage 17.0 (TID 703) in 1839 ms on 10.0.0.133 (executor driver) (170/200) -26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:33 INFO Executor: Finished task 170.0 in stage 17.0 (TID 704). 6401 bytes result sent to driver -26/04/01 08:50:33 INFO TaskSetManager: Starting task 178.0 in stage 17.0 (TID 712) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:33 INFO Executor: Running task 178.0 in stage 17.0 (TID 712) -26/04/01 08:50:33 INFO TaskSetManager: Finished task 170.0 in stage 17.0 (TID 704) in 1836 ms on 10.0.0.133 (executor driver) (171/200) -26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:33 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:33 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:34 INFO Executor: Finished task 171.0 in stage 17.0 (TID 705). 6401 bytes result sent to driver -26/04/01 08:50:34 INFO TaskSetManager: Starting task 179.0 in stage 17.0 (TID 713) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:34 INFO Executor: Running task 179.0 in stage 17.0 (TID 713) -26/04/01 08:50:34 INFO TaskSetManager: Finished task 171.0 in stage 17.0 (TID 705) in 1848 ms on 10.0.0.133 (executor driver) (172/200) -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:34 INFO Executor: Finished task 172.0 in stage 17.0 (TID 706). 6401 bytes result sent to driver -26/04/01 08:50:34 INFO TaskSetManager: Starting task 180.0 in stage 17.0 (TID 714) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:34 INFO TaskSetManager: Finished task 172.0 in stage 17.0 (TID 706) in 1836 ms on 10.0.0.133 (executor driver) (173/200) -26/04/01 08:50:34 INFO Executor: Running task 180.0 in stage 17.0 (TID 714) -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:34 INFO Executor: Finished task 173.0 in stage 17.0 (TID 707). 6401 bytes result sent to driver -26/04/01 08:50:34 INFO TaskSetManager: Starting task 181.0 in stage 17.0 (TID 715) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:34 INFO TaskSetManager: Finished task 173.0 in stage 17.0 (TID 707) in 1848 ms on 10.0.0.133 (executor driver) (174/200) -26/04/01 08:50:34 INFO Executor: Running task 181.0 in stage 17.0 (TID 715) -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:34 INFO Executor: Finished task 174.0 in stage 17.0 (TID 708). 6401 bytes result sent to driver -26/04/01 08:50:34 INFO TaskSetManager: Starting task 182.0 in stage 17.0 (TID 716) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:34 INFO TaskSetManager: Finished task 174.0 in stage 17.0 (TID 708) in 1831 ms on 10.0.0.133 (executor driver) (175/200) -26/04/01 08:50:34 INFO Executor: Running task 182.0 in stage 17.0 (TID 716) -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:34 INFO Executor: Finished task 175.0 in stage 17.0 (TID 709). 6401 bytes result sent to driver -26/04/01 08:50:34 INFO TaskSetManager: Starting task 183.0 in stage 17.0 (TID 717) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:34 INFO TaskSetManager: Finished task 175.0 in stage 17.0 (TID 709) in 1830 ms on 10.0.0.133 (executor driver) (176/200) -26/04/01 08:50:34 INFO Executor: Running task 183.0 in stage 17.0 (TID 717) -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:34 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:35 INFO Executor: Finished task 176.0 in stage 17.0 (TID 710). 6401 bytes result sent to driver -26/04/01 08:50:35 INFO TaskSetManager: Starting task 184.0 in stage 17.0 (TID 718) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:35 INFO Executor: Running task 184.0 in stage 17.0 (TID 718) -26/04/01 08:50:35 INFO TaskSetManager: Finished task 176.0 in stage 17.0 (TID 710) in 1818 ms on 10.0.0.133 (executor driver) (177/200) -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.7 MiB) non-empty blocks including 208 (74.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:35 INFO Executor: Finished task 177.0 in stage 17.0 (TID 711). 6401 bytes result sent to driver -26/04/01 08:50:35 INFO TaskSetManager: Starting task 185.0 in stage 17.0 (TID 719) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:35 INFO Executor: Running task 185.0 in stage 17.0 (TID 719) -26/04/01 08:50:35 INFO TaskSetManager: Finished task 177.0 in stage 17.0 (TID 711) in 1831 ms on 10.0.0.133 (executor driver) (178/200) -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (77.0 MiB) non-empty blocks including 208 (77.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (77.0 MiB) non-empty blocks including 208 (77.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:35 INFO Executor: Finished task 178.0 in stage 17.0 (TID 712). 6401 bytes result sent to driver -26/04/01 08:50:35 INFO TaskSetManager: Starting task 186.0 in stage 17.0 (TID 720) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:35 INFO Executor: Running task 186.0 in stage 17.0 (TID 720) -26/04/01 08:50:35 INFO TaskSetManager: Finished task 178.0 in stage 17.0 (TID 712) in 1836 ms on 10.0.0.133 (executor driver) (179/200) -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:35 INFO Executor: Finished task 179.0 in stage 17.0 (TID 713). 6401 bytes result sent to driver -26/04/01 08:50:35 INFO TaskSetManager: Starting task 187.0 in stage 17.0 (TID 721) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:35 INFO TaskSetManager: Finished task 179.0 in stage 17.0 (TID 713) in 1823 ms on 10.0.0.133 (executor driver) (180/200) -26/04/01 08:50:35 INFO Executor: Running task 187.0 in stage 17.0 (TID 721) -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Getting 208 (75.5 MiB) non-empty blocks including 208 (75.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:35 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:36 INFO Executor: Finished task 180.0 in stage 17.0 (TID 714). 6401 bytes result sent to driver -26/04/01 08:50:36 INFO TaskSetManager: Starting task 188.0 in stage 17.0 (TID 722) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:36 INFO Executor: Running task 188.0 in stage 17.0 (TID 722) -26/04/01 08:50:36 INFO TaskSetManager: Finished task 180.0 in stage 17.0 (TID 714) in 1814 ms on 10.0.0.133 (executor driver) (181/200) -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.6 MiB) non-empty blocks including 208 (75.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:36 INFO Executor: Finished task 181.0 in stage 17.0 (TID 715). 6401 bytes result sent to driver -26/04/01 08:50:36 INFO TaskSetManager: Starting task 189.0 in stage 17.0 (TID 723) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:36 INFO TaskSetManager: Finished task 181.0 in stage 17.0 (TID 715) in 1828 ms on 10.0.0.133 (executor driver) (182/200) -26/04/01 08:50:36 INFO Executor: Running task 189.0 in stage 17.0 (TID 723) -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.1 MiB) non-empty blocks including 208 (75.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:36 INFO Executor: Finished task 182.0 in stage 17.0 (TID 716). 6401 bytes result sent to driver -26/04/01 08:50:36 INFO TaskSetManager: Starting task 190.0 in stage 17.0 (TID 724) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:36 INFO TaskSetManager: Finished task 182.0 in stage 17.0 (TID 716) in 1825 ms on 10.0.0.133 (executor driver) (183/200) -26/04/01 08:50:36 INFO Executor: Running task 190.0 in stage 17.0 (TID 724) -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:36 INFO Executor: Finished task 183.0 in stage 17.0 (TID 717). 6401 bytes result sent to driver -26/04/01 08:50:36 INFO TaskSetManager: Starting task 191.0 in stage 17.0 (TID 725) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:36 INFO TaskSetManager: Finished task 183.0 in stage 17.0 (TID 717) in 1832 ms on 10.0.0.133 (executor driver) (184/200) -26/04/01 08:50:36 INFO Executor: Running task 191.0 in stage 17.0 (TID 725) -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Getting 208 (75.3 MiB) non-empty blocks including 208 (75.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:36 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:37 INFO Executor: Finished task 184.0 in stage 17.0 (TID 718). 6401 bytes result sent to driver -26/04/01 08:50:37 INFO TaskSetManager: Starting task 192.0 in stage 17.0 (TID 726) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:37 INFO Executor: Running task 192.0 in stage 17.0 (TID 726) -26/04/01 08:50:37 INFO TaskSetManager: Finished task 184.0 in stage 17.0 (TID 718) in 1842 ms on 10.0.0.133 (executor driver) (185/200) -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:37 INFO Executor: Finished task 185.0 in stage 17.0 (TID 719). 6401 bytes result sent to driver -26/04/01 08:50:37 INFO TaskSetManager: Starting task 193.0 in stage 17.0 (TID 727) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:37 INFO TaskSetManager: Finished task 185.0 in stage 17.0 (TID 719) in 1831 ms on 10.0.0.133 (executor driver) (186/200) -26/04/01 08:50:37 INFO Executor: Running task 193.0 in stage 17.0 (TID 727) -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (75.7 MiB) non-empty blocks including 208 (75.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:37 INFO Executor: Finished task 186.0 in stage 17.0 (TID 720). 6401 bytes result sent to driver -26/04/01 08:50:37 INFO TaskSetManager: Starting task 194.0 in stage 17.0 (TID 728) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:37 INFO Executor: Running task 194.0 in stage 17.0 (TID 728) -26/04/01 08:50:37 INFO TaskSetManager: Finished task 186.0 in stage 17.0 (TID 720) in 1822 ms on 10.0.0.133 (executor driver) (187/200) -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:37 INFO Executor: Finished task 187.0 in stage 17.0 (TID 721). 6401 bytes result sent to driver -26/04/01 08:50:37 INFO TaskSetManager: Starting task 195.0 in stage 17.0 (TID 729) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:37 INFO TaskSetManager: Finished task 187.0 in stage 17.0 (TID 721) in 1851 ms on 10.0.0.133 (executor driver) (188/200) -26/04/01 08:50:37 INFO Executor: Running task 195.0 in stage 17.0 (TID 729) -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:37 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:37 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:38 INFO Executor: Finished task 188.0 in stage 17.0 (TID 722). 6401 bytes result sent to driver -26/04/01 08:50:38 INFO TaskSetManager: Starting task 196.0 in stage 17.0 (TID 730) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:38 INFO Executor: Running task 196.0 in stage 17.0 (TID 730) -26/04/01 08:50:38 INFO TaskSetManager: Finished task 188.0 in stage 17.0 (TID 722) in 1851 ms on 10.0.0.133 (executor driver) (189/200) -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (75.2 MiB) non-empty blocks including 208 (75.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:38 INFO Executor: Finished task 189.0 in stage 17.0 (TID 723). 6401 bytes result sent to driver -26/04/01 08:50:38 INFO TaskSetManager: Starting task 197.0 in stage 17.0 (TID 731) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:38 INFO Executor: Running task 197.0 in stage 17.0 (TID 731) -26/04/01 08:50:38 INFO TaskSetManager: Finished task 189.0 in stage 17.0 (TID 723) in 1840 ms on 10.0.0.133 (executor driver) (190/200) -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (75.0 MiB) non-empty blocks including 208 (75.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:38 INFO Executor: Finished task 190.0 in stage 17.0 (TID 724). 6401 bytes result sent to driver -26/04/01 08:50:38 INFO TaskSetManager: Starting task 198.0 in stage 17.0 (TID 732) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:38 INFO TaskSetManager: Finished task 190.0 in stage 17.0 (TID 724) in 1838 ms on 10.0.0.133 (executor driver) (191/200) -26/04/01 08:50:38 INFO Executor: Running task 198.0 in stage 17.0 (TID 732) -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.8 MiB) non-empty blocks including 208 (74.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:38 INFO Executor: Finished task 191.0 in stage 17.0 (TID 725). 6401 bytes result sent to driver -26/04/01 08:50:38 INFO TaskSetManager: Starting task 199.0 in stage 17.0 (TID 733) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9624 bytes) -26/04/01 08:50:38 INFO Executor: Running task 199.0 in stage 17.0 (TID 733) -26/04/01 08:50:38 INFO TaskSetManager: Finished task 191.0 in stage 17.0 (TID 725) in 1843 ms on 10.0.0.133 (executor driver) (192/200) -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Getting 208 (74.9 MiB) non-empty blocks including 208 (74.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:39 INFO Executor: Finished task 192.0 in stage 17.0 (TID 726). 6401 bytes result sent to driver -26/04/01 08:50:39 INFO TaskSetManager: Finished task 192.0 in stage 17.0 (TID 726) in 1842 ms on 10.0.0.133 (executor driver) (193/200) -26/04/01 08:50:39 INFO Executor: Finished task 193.0 in stage 17.0 (TID 727). 6401 bytes result sent to driver -26/04/01 08:50:39 INFO TaskSetManager: Finished task 193.0 in stage 17.0 (TID 727) in 1846 ms on 10.0.0.133 (executor driver) (194/200) -26/04/01 08:50:39 INFO Executor: Finished task 194.0 in stage 17.0 (TID 728). 6401 bytes result sent to driver -26/04/01 08:50:39 INFO TaskSetManager: Finished task 194.0 in stage 17.0 (TID 728) in 1827 ms on 10.0.0.133 (executor driver) (195/200) -26/04/01 08:50:39 INFO Executor: Finished task 195.0 in stage 17.0 (TID 729). 6401 bytes result sent to driver -26/04/01 08:50:39 INFO TaskSetManager: Finished task 195.0 in stage 17.0 (TID 729) in 1856 ms on 10.0.0.133 (executor driver) (196/200) -26/04/01 08:50:40 INFO Executor: Finished task 196.0 in stage 17.0 (TID 730). 6401 bytes result sent to driver -26/04/01 08:50:40 INFO TaskSetManager: Finished task 196.0 in stage 17.0 (TID 730) in 1846 ms on 10.0.0.133 (executor driver) (197/200) -26/04/01 08:50:40 INFO Executor: Finished task 197.0 in stage 17.0 (TID 731). 6401 bytes result sent to driver -26/04/01 08:50:40 INFO TaskSetManager: Finished task 197.0 in stage 17.0 (TID 731) in 1813 ms on 10.0.0.133 (executor driver) (198/200) -26/04/01 08:50:40 INFO Executor: Finished task 198.0 in stage 17.0 (TID 732). 6401 bytes result sent to driver -26/04/01 08:50:40 INFO TaskSetManager: Finished task 198.0 in stage 17.0 (TID 732) in 1809 ms on 10.0.0.133 (executor driver) (199/200) -26/04/01 08:50:40 INFO Executor: Finished task 199.0 in stage 17.0 (TID 733). 6401 bytes result sent to driver -26/04/01 08:50:40 INFO TaskSetManager: Finished task 199.0 in stage 17.0 (TID 733) in 1802 ms on 10.0.0.133 (executor driver) (200/200) -26/04/01 08:50:40 INFO TaskSchedulerImpl: Removed TaskSet 17.0, whose tasks have all completed, from pool -26/04/01 08:50:40 INFO DAGScheduler: ShuffleMapStage 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 47.315 s -26/04/01 08:50:40 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:50:40 INFO DAGScheduler: running: Set() -26/04/01 08:50:40 INFO DAGScheduler: waiting: Set() -26/04/01 08:50:40 INFO DAGScheduler: failed: Set() -26/04/01 08:50:40 INFO ShufflePartitionsUtil: For shuffle(5, 2), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 08:50:40 INFO DAGScheduler: Registering RDD 43 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 6 -26/04/01 08:50:40 INFO DAGScheduler: Got map stage job 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 14 output partitions -26/04/01 08:50:40 INFO DAGScheduler: Final stage: ShuffleMapStage 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:50:40 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 19, ShuffleMapStage 20) -26/04/01 08:50:40 INFO DAGScheduler: Missing parents: List() -26/04/01 08:50:40 INFO DAGScheduler: Submitting ShuffleMapStage 21 (MapPartitionsRDD[43] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:50:40 INFO MemoryStore: Block broadcast_24 stored as values in memory (estimated size 24.1 KiB, free 12.6 GiB) -26/04/01 08:50:40 INFO MemoryStore: Block broadcast_24_piece0 stored as bytes in memory (estimated size 9.9 KiB, free 12.6 GiB) -26/04/01 08:50:40 INFO BlockManagerInfo: Added broadcast_24_piece0 in memory on 10.0.0.133:59333 (size: 9.9 KiB, free: 12.6 GiB) -26/04/01 08:50:40 INFO SparkContext: Created broadcast 24 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:50:40 INFO DAGScheduler: Submitting 14 missing tasks from ShuffleMapStage 21 (MapPartitionsRDD[43] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)) -26/04/01 08:50:40 INFO TaskSchedulerImpl: Adding task set 21.0 with 14 tasks resource profile 0 -26/04/01 08:50:40 INFO TaskSetManager: Starting task 0.0 in stage 21.0 (TID 734) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:40 INFO TaskSetManager: Starting task 1.0 in stage 21.0 (TID 735) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:40 INFO TaskSetManager: Starting task 2.0 in stage 21.0 (TID 736) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:40 INFO TaskSetManager: Starting task 3.0 in stage 21.0 (TID 737) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:40 INFO TaskSetManager: Starting task 4.0 in stage 21.0 (TID 738) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:40 INFO TaskSetManager: Starting task 5.0 in stage 21.0 (TID 739) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:40 INFO TaskSetManager: Starting task 6.0 in stage 21.0 (TID 740) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:40 INFO TaskSetManager: Starting task 7.0 in stage 21.0 (TID 741) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:40 INFO Executor: Running task 1.0 in stage 21.0 (TID 735) -26/04/01 08:50:40 INFO Executor: Running task 2.0 in stage 21.0 (TID 736) -26/04/01 08:50:40 INFO Executor: Running task 7.0 in stage 21.0 (TID 741) -26/04/01 08:50:40 INFO Executor: Running task 0.0 in stage 21.0 (TID 734) -26/04/01 08:50:40 INFO Executor: Running task 6.0 in stage 21.0 (TID 740) -26/04/01 08:50:40 INFO Executor: Running task 3.0 in stage 21.0 (TID 737) -26/04/01 08:50:40 INFO Executor: Running task 4.0 in stage 21.0 (TID 738) -26/04/01 08:50:40 INFO Executor: Running task 5.0 in stage 21.0 (TID 739) -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.3 MiB) non-empty blocks including 200 (61.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.0 MiB) non-empty blocks including 200 (61.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.6 MiB) non-empty blocks including 200 (61.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 6 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (624.5 KiB) non-empty blocks including 8 (624.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (619.1 KiB) non-empty blocks including 8 (619.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (618.3 KiB) non-empty blocks including 8 (618.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (619.7 KiB) non-empty blocks including 8 (619.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (621.6 KiB) non-empty blocks including 8 (621.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (618.2 KiB) non-empty blocks including 8 (618.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.3 MiB) non-empty blocks including 200 (61.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.0 MiB) non-empty blocks including 200 (61.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 200 (61.6 MiB) non-empty blocks including 200 (61.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (621.6 KiB) non-empty blocks including 8 (621.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (619.1 KiB) non-empty blocks including 8 (619.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (618.3 KiB) non-empty blocks including 8 (618.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (618.2 KiB) non-empty blocks including 8 (618.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (616.7 KiB) non-empty blocks including 8 (616.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (624.5 KiB) non-empty blocks including 8 (624.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Getting 8 (619.7 KiB) non-empty blocks including 8 (619.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:40 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:42 INFO BlockManagerInfo: Removed broadcast_23_piece0 on 10.0.0.133:59333 in memory (size: 11.0 KiB, free: 12.6 GiB) -26/04/01 08:50:50 INFO Executor: Finished task 1.0 in stage 21.0 (TID 735). 9594 bytes result sent to driver -26/04/01 08:50:50 INFO TaskSetManager: Starting task 8.0 in stage 21.0 (TID 742) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:50 INFO Executor: Running task 8.0 in stage 21.0 (TID 742) -26/04/01 08:50:50 INFO TaskSetManager: Finished task 1.0 in stage 21.0 (TID 735) in 9928 ms on 10.0.0.133 (executor driver) (1/14) -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (623.7 KiB) non-empty blocks including 8 (623.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (623.7 KiB) non-empty blocks including 8 (623.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:50 INFO Executor: Finished task 5.0 in stage 21.0 (TID 739). 9594 bytes result sent to driver -26/04/01 08:50:50 INFO TaskSetManager: Starting task 9.0 in stage 21.0 (TID 743) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:50 INFO TaskSetManager: Finished task 5.0 in stage 21.0 (TID 739) in 9984 ms on 10.0.0.133 (executor driver) (2/14) -26/04/01 08:50:50 INFO Executor: Running task 9.0 in stage 21.0 (TID 743) -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (623.6 KiB) non-empty blocks including 8 (623.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (623.6 KiB) non-empty blocks including 8 (623.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:50 INFO Executor: Finished task 7.0 in stage 21.0 (TID 741). 9594 bytes result sent to driver -26/04/01 08:50:50 INFO TaskSetManager: Starting task 10.0 in stage 21.0 (TID 744) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:50 INFO TaskSetManager: Finished task 7.0 in stage 21.0 (TID 741) in 10018 ms on 10.0.0.133 (executor driver) (3/14) -26/04/01 08:50:50 INFO Executor: Running task 10.0 in stage 21.0 (TID 744) -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.2 MiB) non-empty blocks including 200 (61.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (617.6 KiB) non-empty blocks including 8 (617.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.2 MiB) non-empty blocks including 200 (61.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (617.6 KiB) non-empty blocks including 8 (617.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:50 INFO Executor: Finished task 6.0 in stage 21.0 (TID 740). 9594 bytes result sent to driver -26/04/01 08:50:50 INFO TaskSetManager: Starting task 11.0 in stage 21.0 (TID 745) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:50 INFO Executor: Running task 11.0 in stage 21.0 (TID 745) -26/04/01 08:50:50 INFO TaskSetManager: Finished task 6.0 in stage 21.0 (TID 740) in 10044 ms on 10.0.0.133 (executor driver) (4/14) -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (622.2 KiB) non-empty blocks including 8 (622.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.4 MiB) non-empty blocks including 200 (61.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (622.2 KiB) non-empty blocks including 8 (622.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:50 INFO Executor: Finished task 4.0 in stage 21.0 (TID 738). 9594 bytes result sent to driver -26/04/01 08:50:50 INFO TaskSetManager: Starting task 12.0 in stage 21.0 (TID 746) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:50 INFO TaskSetManager: Finished task 4.0 in stage 21.0 (TID 738) in 10074 ms on 10.0.0.133 (executor driver) (5/14) -26/04/01 08:50:50 INFO Executor: Running task 12.0 in stage 21.0 (TID 746) -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (622.0 KiB) non-empty blocks including 8 (622.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (61.1 MiB) non-empty blocks including 200 (61.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (622.0 KiB) non-empty blocks including 8 (622.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:50 INFO Executor: Finished task 0.0 in stage 21.0 (TID 734). 9594 bytes result sent to driver -26/04/01 08:50:50 INFO TaskSetManager: Starting task 13.0 in stage 21.0 (TID 747) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:50 INFO TaskSetManager: Finished task 0.0 in stage 21.0 (TID 734) in 10122 ms on 10.0.0.133 (executor driver) (6/14) -26/04/01 08:50:50 INFO Executor: Running task 13.0 in stage 21.0 (TID 747) -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (20.4 MiB) non-empty blocks including 200 (20.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (207.3 KiB) non-empty blocks including 8 (207.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 200 (20.4 MiB) non-empty blocks including 200 (20.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Getting 8 (207.3 KiB) non-empty blocks including 8 (207.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:50 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:50 INFO Executor: Finished task 2.0 in stage 21.0 (TID 736). 9594 bytes result sent to driver -26/04/01 08:50:50 INFO TaskSetManager: Finished task 2.0 in stage 21.0 (TID 736) in 10152 ms on 10.0.0.133 (executor driver) (7/14) -26/04/01 08:50:50 INFO Executor: Finished task 3.0 in stage 21.0 (TID 737). 9594 bytes result sent to driver -26/04/01 08:50:50 INFO TaskSetManager: Finished task 3.0 in stage 21.0 (TID 737) in 10181 ms on 10.0.0.133 (executor driver) (8/14) -26/04/01 08:50:53 INFO Executor: Finished task 13.0 in stage 21.0 (TID 747). 9594 bytes result sent to driver -26/04/01 08:50:53 INFO TaskSetManager: Finished task 13.0 in stage 21.0 (TID 747) in 2971 ms on 10.0.0.133 (executor driver) (9/14) -26/04/01 08:50:59 INFO Executor: Finished task 10.0 in stage 21.0 (TID 744). 9594 bytes result sent to driver -26/04/01 08:50:59 INFO TaskSetManager: Finished task 10.0 in stage 21.0 (TID 744) in 9232 ms on 10.0.0.133 (executor driver) (10/14) -26/04/01 08:50:59 INFO Executor: Finished task 12.0 in stage 21.0 (TID 746). 9594 bytes result sent to driver -26/04/01 08:50:59 INFO TaskSetManager: Finished task 12.0 in stage 21.0 (TID 746) in 9250 ms on 10.0.0.133 (executor driver) (11/14) -26/04/01 08:50:59 INFO Executor: Finished task 9.0 in stage 21.0 (TID 743). 9594 bytes result sent to driver -26/04/01 08:50:59 INFO TaskSetManager: Finished task 9.0 in stage 21.0 (TID 743) in 9345 ms on 10.0.0.133 (executor driver) (12/14) -26/04/01 08:50:59 INFO Executor: Finished task 8.0 in stage 21.0 (TID 742). 9594 bytes result sent to driver -26/04/01 08:50:59 INFO TaskSetManager: Finished task 8.0 in stage 21.0 (TID 742) in 9403 ms on 10.0.0.133 (executor driver) (13/14) -26/04/01 08:50:59 INFO Executor: Finished task 11.0 in stage 21.0 (TID 745). 9594 bytes result sent to driver -26/04/01 08:50:59 INFO TaskSetManager: Finished task 11.0 in stage 21.0 (TID 745) in 9323 ms on 10.0.0.133 (executor driver) (14/14) -26/04/01 08:50:59 INFO TaskSchedulerImpl: Removed TaskSet 21.0, whose tasks have all completed, from pool -26/04/01 08:50:59 INFO DAGScheduler: ShuffleMapStage 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 19.369 s -26/04/01 08:50:59 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:50:59 INFO DAGScheduler: running: Set() -26/04/01 08:50:59 INFO DAGScheduler: waiting: Set() -26/04/01 08:50:59 INFO DAGScheduler: failed: Set() -26/04/01 08:50:59 INFO ShufflePartitionsUtil: For shuffle(6, 3), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 08:50:59 INFO DAGScheduler: Registering RDD 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 7 -26/04/01 08:50:59 INFO DAGScheduler: Got map stage job 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 29 output partitions -26/04/01 08:50:59 INFO DAGScheduler: Final stage: ShuffleMapStage 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:50:59 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 25, ShuffleMapStage 26) -26/04/01 08:50:59 INFO DAGScheduler: Missing parents: List() -26/04/01 08:50:59 INFO DAGScheduler: Submitting ShuffleMapStage 27 (MapPartitionsRDD[47] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:50:59 INFO MemoryStore: Block broadcast_25 stored as values in memory (estimated size 28.9 KiB, free 12.6 GiB) -26/04/01 08:50:59 INFO MemoryStore: Block broadcast_25_piece0 stored as bytes in memory (estimated size 10.3 KiB, free 12.6 GiB) -26/04/01 08:50:59 INFO BlockManagerInfo: Added broadcast_25_piece0 in memory on 10.0.0.133:59333 (size: 10.3 KiB, free: 12.6 GiB) -26/04/01 08:50:59 INFO SparkContext: Created broadcast 25 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:50:59 INFO DAGScheduler: Submitting 29 missing tasks from ShuffleMapStage 27 (MapPartitionsRDD[47] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:50:59 INFO TaskSchedulerImpl: Adding task set 27.0 with 29 tasks resource profile 0 -26/04/01 08:50:59 INFO TaskSetManager: Starting task 0.0 in stage 27.0 (TID 748) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:59 INFO TaskSetManager: Starting task 1.0 in stage 27.0 (TID 749) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:59 INFO TaskSetManager: Starting task 2.0 in stage 27.0 (TID 750) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:59 INFO TaskSetManager: Starting task 3.0 in stage 27.0 (TID 751) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:59 INFO TaskSetManager: Starting task 4.0 in stage 27.0 (TID 752) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:59 INFO TaskSetManager: Starting task 5.0 in stage 27.0 (TID 753) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:59 INFO TaskSetManager: Starting task 6.0 in stage 27.0 (TID 754) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:59 INFO TaskSetManager: Starting task 7.0 in stage 27.0 (TID 755) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9428 bytes) -26/04/01 08:50:59 INFO Executor: Running task 2.0 in stage 27.0 (TID 750) -26/04/01 08:50:59 INFO Executor: Running task 0.0 in stage 27.0 (TID 748) -26/04/01 08:50:59 INFO Executor: Running task 5.0 in stage 27.0 (TID 753) -26/04/01 08:50:59 INFO Executor: Running task 6.0 in stage 27.0 (TID 754) -26/04/01 08:50:59 INFO Executor: Running task 7.0 in stage 27.0 (TID 755) -26/04/01 08:50:59 INFO Executor: Running task 4.0 in stage 27.0 (TID 752) -26/04/01 08:50:59 INFO Executor: Running task 3.0 in stage 27.0 (TID 751) -26/04/01 08:50:59 INFO Executor: Running task 1.0 in stage 27.0 (TID 749) -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.6 MiB) non-empty blocks including 14 (23.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.2 MiB) non-empty blocks including 14 (23.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.6 MiB) non-empty blocks including 14 (23.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 14 (23.2 MiB) non-empty blocks including 14 (23.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:50:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:09 INFO Executor: Finished task 4.0 in stage 27.0 (TID 752). 9594 bytes result sent to driver -26/04/01 08:51:09 INFO TaskSetManager: Starting task 8.0 in stage 27.0 (TID 756) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:09 INFO TaskSetManager: Finished task 4.0 in stage 27.0 (TID 752) in 9377 ms on 10.0.0.133 (executor driver) (1/29) -26/04/01 08:51:09 INFO Executor: Running task 8.0 in stage 27.0 (TID 756) -26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:09 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:10 INFO Executor: Finished task 6.0 in stage 27.0 (TID 754). 9594 bytes result sent to driver -26/04/01 08:51:10 INFO TaskSetManager: Starting task 9.0 in stage 27.0 (TID 757) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:10 INFO TaskSetManager: Finished task 6.0 in stage 27.0 (TID 754) in 10383 ms on 10.0.0.133 (executor driver) (2/29) -26/04/01 08:51:10 INFO Executor: Running task 9.0 in stage 27.0 (TID 757) -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:10 INFO Executor: Finished task 3.0 in stage 27.0 (TID 751). 9594 bytes result sent to driver -26/04/01 08:51:10 INFO TaskSetManager: Starting task 10.0 in stage 27.0 (TID 758) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:10 INFO TaskSetManager: Finished task 3.0 in stage 27.0 (TID 751) in 10414 ms on 10.0.0.133 (executor driver) (3/29) -26/04/01 08:51:10 INFO Executor: Running task 10.0 in stage 27.0 (TID 758) -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:10 INFO Executor: Finished task 5.0 in stage 27.0 (TID 753). 9594 bytes result sent to driver -26/04/01 08:51:10 INFO TaskSetManager: Starting task 11.0 in stage 27.0 (TID 759) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:10 INFO TaskSetManager: Finished task 5.0 in stage 27.0 (TID 753) in 10560 ms on 10.0.0.133 (executor driver) (4/29) -26/04/01 08:51:10 INFO Executor: Running task 11.0 in stage 27.0 (TID 759) -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:10 INFO Executor: Finished task 2.0 in stage 27.0 (TID 750). 9637 bytes result sent to driver -26/04/01 08:51:10 INFO TaskSetManager: Starting task 12.0 in stage 27.0 (TID 760) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:10 INFO TaskSetManager: Finished task 2.0 in stage 27.0 (TID 750) in 10588 ms on 10.0.0.133 (executor driver) (5/29) -26/04/01 08:51:10 INFO Executor: Running task 12.0 in stage 27.0 (TID 760) -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:10 INFO Executor: Finished task 1.0 in stage 27.0 (TID 749). 9637 bytes result sent to driver -26/04/01 08:51:10 INFO TaskSetManager: Starting task 13.0 in stage 27.0 (TID 761) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:10 INFO Executor: Running task 13.0 in stage 27.0 (TID 761) -26/04/01 08:51:10 INFO TaskSetManager: Finished task 1.0 in stage 27.0 (TID 749) in 10786 ms on 10.0.0.133 (executor driver) (6/29) -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:10 INFO Executor: Finished task 0.0 in stage 27.0 (TID 748). 9594 bytes result sent to driver -26/04/01 08:51:10 INFO TaskSetManager: Starting task 14.0 in stage 27.0 (TID 762) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:10 INFO TaskSetManager: Finished task 0.0 in stage 27.0 (TID 748) in 10867 ms on 10.0.0.133 (executor driver) (7/29) -26/04/01 08:51:10 INFO Executor: Running task 14.0 in stage 27.0 (TID 762) -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:10 INFO Executor: Finished task 7.0 in stage 27.0 (TID 755). 9594 bytes result sent to driver -26/04/01 08:51:10 INFO TaskSetManager: Starting task 15.0 in stage 27.0 (TID 763) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:10 INFO TaskSetManager: Finished task 7.0 in stage 27.0 (TID 755) in 11015 ms on 10.0.0.133 (executor driver) (8/29) -26/04/01 08:51:10 INFO Executor: Running task 15.0 in stage 27.0 (TID 763) -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:10 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:17 INFO Executor: Finished task 8.0 in stage 27.0 (TID 756). 9594 bytes result sent to driver -26/04/01 08:51:17 INFO TaskSetManager: Starting task 16.0 in stage 27.0 (TID 764) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:17 INFO TaskSetManager: Finished task 8.0 in stage 27.0 (TID 756) in 8585 ms on 10.0.0.133 (executor driver) (9/29) -26/04/01 08:51:17 INFO Executor: Running task 16.0 in stage 27.0 (TID 764) -26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:17 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:19 INFO Executor: Finished task 9.0 in stage 27.0 (TID 757). 9594 bytes result sent to driver -26/04/01 08:51:19 INFO TaskSetManager: Starting task 17.0 in stage 27.0 (TID 765) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:19 INFO TaskSetManager: Finished task 9.0 in stage 27.0 (TID 757) in 8956 ms on 10.0.0.133 (executor driver) (10/29) -26/04/01 08:51:19 INFO Executor: Running task 17.0 in stage 27.0 (TID 765) -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:19 INFO Executor: Finished task 11.0 in stage 27.0 (TID 759). 9594 bytes result sent to driver -26/04/01 08:51:19 INFO TaskSetManager: Starting task 18.0 in stage 27.0 (TID 766) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:19 INFO TaskSetManager: Finished task 11.0 in stage 27.0 (TID 759) in 9564 ms on 10.0.0.133 (executor driver) (11/29) -26/04/01 08:51:19 INFO Executor: Running task 18.0 in stage 27.0 (TID 766) -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:19 INFO Executor: Finished task 10.0 in stage 27.0 (TID 758). 9594 bytes result sent to driver -26/04/01 08:51:19 INFO TaskSetManager: Starting task 19.0 in stage 27.0 (TID 767) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:19 INFO Executor: Running task 19.0 in stage 27.0 (TID 767) -26/04/01 08:51:19 INFO TaskSetManager: Finished task 10.0 in stage 27.0 (TID 758) in 9744 ms on 10.0.0.133 (executor driver) (12/29) -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:19 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:20 INFO Executor: Finished task 12.0 in stage 27.0 (TID 760). 9594 bytes result sent to driver -26/04/01 08:51:20 INFO TaskSetManager: Starting task 20.0 in stage 27.0 (TID 768) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:20 INFO Executor: Running task 20.0 in stage 27.0 (TID 768) -26/04/01 08:51:20 INFO TaskSetManager: Finished task 12.0 in stage 27.0 (TID 760) in 9795 ms on 10.0.0.133 (executor driver) (13/29) -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:20 INFO Executor: Finished task 15.0 in stage 27.0 (TID 763). 9594 bytes result sent to driver -26/04/01 08:51:20 INFO TaskSetManager: Starting task 21.0 in stage 27.0 (TID 769) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:20 INFO Executor: Running task 21.0 in stage 27.0 (TID 769) -26/04/01 08:51:20 INFO TaskSetManager: Finished task 15.0 in stage 27.0 (TID 763) in 9408 ms on 10.0.0.133 (executor driver) (14/29) -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:20 INFO Executor: Finished task 14.0 in stage 27.0 (TID 762). 9594 bytes result sent to driver -26/04/01 08:51:20 INFO TaskSetManager: Starting task 22.0 in stage 27.0 (TID 770) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:20 INFO Executor: Running task 22.0 in stage 27.0 (TID 770) -26/04/01 08:51:20 INFO TaskSetManager: Finished task 14.0 in stage 27.0 (TID 762) in 9615 ms on 10.0.0.133 (executor driver) (15/29) -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.3 MiB) non-empty blocks including 14 (23.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:20 INFO Executor: Finished task 13.0 in stage 27.0 (TID 761). 9594 bytes result sent to driver -26/04/01 08:51:20 INFO TaskSetManager: Starting task 23.0 in stage 27.0 (TID 771) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:20 INFO Executor: Running task 23.0 in stage 27.0 (TID 771) -26/04/01 08:51:20 INFO TaskSetManager: Finished task 13.0 in stage 27.0 (TID 761) in 9761 ms on 10.0.0.133 (executor driver) (16/29) -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:20 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:26 INFO Executor: Finished task 16.0 in stage 27.0 (TID 764). 9594 bytes result sent to driver -26/04/01 08:51:26 INFO TaskSetManager: Starting task 24.0 in stage 27.0 (TID 772) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:26 INFO Executor: Running task 24.0 in stage 27.0 (TID 772) -26/04/01 08:51:26 INFO TaskSetManager: Finished task 16.0 in stage 27.0 (TID 764) in 8810 ms on 10.0.0.133 (executor driver) (17/29) -26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:26 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:27 INFO Executor: Finished task 17.0 in stage 27.0 (TID 765). 9594 bytes result sent to driver -26/04/01 08:51:27 INFO TaskSetManager: Starting task 25.0 in stage 27.0 (TID 773) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:27 INFO TaskSetManager: Finished task 17.0 in stage 27.0 (TID 765) in 8907 ms on 10.0.0.133 (executor driver) (18/29) -26/04/01 08:51:27 INFO Executor: Running task 25.0 in stage 27.0 (TID 773) -26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:27 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:28 INFO Executor: Finished task 19.0 in stage 27.0 (TID 767). 9594 bytes result sent to driver -26/04/01 08:51:28 INFO TaskSetManager: Starting task 26.0 in stage 27.0 (TID 774) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:28 INFO TaskSetManager: Finished task 19.0 in stage 27.0 (TID 767) in 9161 ms on 10.0.0.133 (executor driver) (19/29) -26/04/01 08:51:28 INFO Executor: Running task 26.0 in stage 27.0 (TID 774) -26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Getting 14 (23.5 MiB) non-empty blocks including 14 (23.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:28 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:28 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:29 INFO Executor: Finished task 18.0 in stage 27.0 (TID 766). 9594 bytes result sent to driver -26/04/01 08:51:29 INFO TaskSetManager: Starting task 27.0 in stage 27.0 (TID 775) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:29 INFO Executor: Running task 27.0 in stage 27.0 (TID 775) -26/04/01 08:51:29 INFO TaskSetManager: Finished task 18.0 in stage 27.0 (TID 766) in 9739 ms on 10.0.0.133 (executor driver) (20/29) -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 14 (23.4 MiB) non-empty blocks including 14 (23.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 37 (38.2 MiB) non-empty blocks including 37 (38.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:29 INFO Executor: Finished task 22.0 in stage 27.0 (TID 770). 9594 bytes result sent to driver -26/04/01 08:51:29 INFO TaskSetManager: Starting task 28.0 in stage 27.0 (TID 776) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9428 bytes) -26/04/01 08:51:29 INFO Executor: Running task 28.0 in stage 27.0 (TID 776) -26/04/01 08:51:29 INFO TaskSetManager: Finished task 22.0 in stage 27.0 (TID 770) in 9628 ms on 10.0.0.133 (executor driver) (21/29) -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 14 (13.4 MiB) non-empty blocks including 14 (13.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 37 (21.8 MiB) non-empty blocks including 37 (21.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 14 (13.4 MiB) non-empty blocks including 14 (13.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Getting 37 (21.8 MiB) non-empty blocks including 37 (21.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:29 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:29 INFO Executor: Finished task 20.0 in stage 27.0 (TID 768). 9594 bytes result sent to driver -26/04/01 08:51:29 INFO TaskSetManager: Finished task 20.0 in stage 27.0 (TID 768) in 9810 ms on 10.0.0.133 (executor driver) (22/29) -26/04/01 08:51:29 INFO Executor: Finished task 21.0 in stage 27.0 (TID 769). 9594 bytes result sent to driver -26/04/01 08:51:29 INFO TaskSetManager: Finished task 21.0 in stage 27.0 (TID 769) in 9799 ms on 10.0.0.133 (executor driver) (23/29) -26/04/01 08:51:29 INFO Executor: Finished task 23.0 in stage 27.0 (TID 771). 9594 bytes result sent to driver -26/04/01 08:51:29 INFO TaskSetManager: Finished task 23.0 in stage 27.0 (TID 771) in 9744 ms on 10.0.0.133 (executor driver) (24/29) -26/04/01 08:51:34 INFO Executor: Finished task 28.0 in stage 27.0 (TID 776). 9594 bytes result sent to driver -26/04/01 08:51:34 INFO TaskSetManager: Finished task 28.0 in stage 27.0 (TID 776) in 4896 ms on 10.0.0.133 (executor driver) (25/29) -26/04/01 08:51:35 INFO Executor: Finished task 24.0 in stage 27.0 (TID 772). 9594 bytes result sent to driver -26/04/01 08:51:35 INFO TaskSetManager: Finished task 24.0 in stage 27.0 (TID 772) in 8825 ms on 10.0.0.133 (executor driver) (26/29) -26/04/01 08:51:36 INFO Executor: Finished task 25.0 in stage 27.0 (TID 773). 9594 bytes result sent to driver -26/04/01 08:51:36 INFO TaskSetManager: Finished task 25.0 in stage 27.0 (TID 773) in 8790 ms on 10.0.0.133 (executor driver) (27/29) -26/04/01 08:51:37 INFO Executor: Finished task 26.0 in stage 27.0 (TID 774). 9594 bytes result sent to driver -26/04/01 08:51:37 INFO TaskSetManager: Finished task 26.0 in stage 27.0 (TID 774) in 8755 ms on 10.0.0.133 (executor driver) (28/29) -26/04/01 08:51:38 INFO Executor: Finished task 27.0 in stage 27.0 (TID 775). 9594 bytes result sent to driver -26/04/01 08:51:38 INFO TaskSetManager: Finished task 27.0 in stage 27.0 (TID 775) in 8708 ms on 10.0.0.133 (executor driver) (29/29) -26/04/01 08:51:38 INFO TaskSchedulerImpl: Removed TaskSet 27.0, whose tasks have all completed, from pool -26/04/01 08:51:38 INFO DAGScheduler: ShuffleMapStage 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 38.572 s -26/04/01 08:51:38 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:51:38 INFO DAGScheduler: running: Set() -26/04/01 08:51:38 INFO DAGScheduler: waiting: Set() -26/04/01 08:51:38 INFO DAGScheduler: failed: Set() -26/04/01 08:51:38 INFO ShufflePartitionsUtil: For shuffle(7, 4), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 08:51:38 INFO DAGScheduler: Registering RDD 52 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 8 -26/04/01 08:51:38 INFO DAGScheduler: Got map stage job 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 34 output partitions -26/04/01 08:51:38 INFO DAGScheduler: Final stage: ShuffleMapStage 35 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:51:38 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 33, ShuffleMapStage 34) -26/04/01 08:51:38 INFO DAGScheduler: Missing parents: List() -26/04/01 08:51:38 INFO DAGScheduler: Submitting ShuffleMapStage 35 (MapPartitionsRDD[52] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:51:38 INFO MemoryStore: Block broadcast_26 stored as values in memory (estimated size 44.4 KiB, free 12.6 GiB) -26/04/01 08:51:38 INFO MemoryStore: Block broadcast_26_piece0 stored as bytes in memory (estimated size 12.6 KiB, free 12.6 GiB) -26/04/01 08:51:38 INFO BlockManagerInfo: Added broadcast_26_piece0 in memory on 10.0.0.133:59333 (size: 12.6 KiB, free: 12.6 GiB) -26/04/01 08:51:38 INFO SparkContext: Created broadcast 26 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:51:38 INFO DAGScheduler: Submitting 34 missing tasks from ShuffleMapStage 35 (MapPartitionsRDD[52] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:51:38 INFO TaskSchedulerImpl: Adding task set 35.0 with 34 tasks resource profile 0 -26/04/01 08:51:38 INFO TaskSetManager: Starting task 0.0 in stage 35.0 (TID 777) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:38 INFO TaskSetManager: Starting task 1.0 in stage 35.0 (TID 778) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:38 INFO TaskSetManager: Starting task 2.0 in stage 35.0 (TID 779) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:38 INFO TaskSetManager: Starting task 3.0 in stage 35.0 (TID 780) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:38 INFO TaskSetManager: Starting task 4.0 in stage 35.0 (TID 781) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:38 INFO TaskSetManager: Starting task 5.0 in stage 35.0 (TID 782) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:38 INFO TaskSetManager: Starting task 6.0 in stage 35.0 (TID 783) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:38 INFO TaskSetManager: Starting task 7.0 in stage 35.0 (TID 784) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:38 INFO Executor: Running task 1.0 in stage 35.0 (TID 778) -26/04/01 08:51:38 INFO Executor: Running task 0.0 in stage 35.0 (TID 777) -26/04/01 08:51:38 INFO Executor: Running task 7.0 in stage 35.0 (TID 784) -26/04/01 08:51:38 INFO Executor: Running task 4.0 in stage 35.0 (TID 781) -26/04/01 08:51:38 INFO Executor: Running task 3.0 in stage 35.0 (TID 780) -26/04/01 08:51:38 INFO Executor: Running task 2.0 in stage 35.0 (TID 779) -26/04/01 08:51:38 INFO Executor: Running task 6.0 in stage 35.0 (TID 783) -26/04/01 08:51:38 INFO Executor: Running task 5.0 in stage 35.0 (TID 782) -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:38 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO Executor: Finished task 3.0 in stage 35.0 (TID 780). 10455 bytes result sent to driver -26/04/01 08:51:45 INFO TaskSetManager: Starting task 8.0 in stage 35.0 (TID 785) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:45 INFO Executor: Running task 8.0 in stage 35.0 (TID 785) -26/04/01 08:51:45 INFO TaskSetManager: Finished task 3.0 in stage 35.0 (TID 780) in 7062 ms on 10.0.0.133 (executor driver) (1/34) -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO Executor: Finished task 2.0 in stage 35.0 (TID 779). 10455 bytes result sent to driver -26/04/01 08:51:45 INFO TaskSetManager: Starting task 9.0 in stage 35.0 (TID 786) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:45 INFO TaskSetManager: Finished task 2.0 in stage 35.0 (TID 779) in 7288 ms on 10.0.0.133 (executor driver) (2/34) -26/04/01 08:51:45 INFO Executor: Running task 9.0 in stage 35.0 (TID 786) -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO Executor: Finished task 4.0 in stage 35.0 (TID 781). 10455 bytes result sent to driver -26/04/01 08:51:45 INFO TaskSetManager: Starting task 10.0 in stage 35.0 (TID 787) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:45 INFO TaskSetManager: Finished task 4.0 in stage 35.0 (TID 781) in 7324 ms on 10.0.0.133 (executor driver) (3/34) -26/04/01 08:51:45 INFO Executor: Running task 10.0 in stage 35.0 (TID 787) -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO Executor: Finished task 1.0 in stage 35.0 (TID 778). 10455 bytes result sent to driver -26/04/01 08:51:45 INFO TaskSetManager: Starting task 11.0 in stage 35.0 (TID 788) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:45 INFO TaskSetManager: Finished task 1.0 in stage 35.0 (TID 778) in 7358 ms on 10.0.0.133 (executor driver) (4/34) -26/04/01 08:51:45 INFO Executor: Running task 11.0 in stage 35.0 (TID 788) -26/04/01 08:51:45 INFO Executor: Finished task 6.0 in stage 35.0 (TID 783). 10455 bytes result sent to driver -26/04/01 08:51:45 INFO TaskSetManager: Starting task 12.0 in stage 35.0 (TID 789) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:45 INFO Executor: Running task 12.0 in stage 35.0 (TID 789) -26/04/01 08:51:45 INFO TaskSetManager: Finished task 6.0 in stage 35.0 (TID 783) in 7358 ms on 10.0.0.133 (executor driver) (5/34) -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO Executor: Finished task 7.0 in stage 35.0 (TID 784). 10455 bytes result sent to driver -26/04/01 08:51:45 INFO TaskSetManager: Starting task 13.0 in stage 35.0 (TID 790) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:45 INFO Executor: Running task 13.0 in stage 35.0 (TID 790) -26/04/01 08:51:45 INFO TaskSetManager: Finished task 7.0 in stage 35.0 (TID 784) in 7391 ms on 10.0.0.133 (executor driver) (6/34) -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO Executor: Finished task 0.0 in stage 35.0 (TID 777). 10455 bytes result sent to driver -26/04/01 08:51:45 INFO TaskSetManager: Starting task 14.0 in stage 35.0 (TID 791) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:45 INFO TaskSetManager: Finished task 0.0 in stage 35.0 (TID 777) in 7413 ms on 10.0.0.133 (executor driver) (7/34) -26/04/01 08:51:45 INFO Executor: Running task 14.0 in stage 35.0 (TID 791) -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO Executor: Finished task 5.0 in stage 35.0 (TID 782). 10455 bytes result sent to driver -26/04/01 08:51:45 INFO TaskSetManager: Starting task 15.0 in stage 35.0 (TID 792) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:45 INFO Executor: Running task 15.0 in stage 35.0 (TID 792) -26/04/01 08:51:45 INFO TaskSetManager: Finished task 5.0 in stage 35.0 (TID 782) in 7422 ms on 10.0.0.133 (executor driver) (8/34) -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:45 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:45 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO Executor: Finished task 8.0 in stage 35.0 (TID 785). 10455 bytes result sent to driver -26/04/01 08:51:52 INFO TaskSetManager: Starting task 16.0 in stage 35.0 (TID 793) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:52 INFO Executor: Running task 16.0 in stage 35.0 (TID 793) -26/04/01 08:51:52 INFO TaskSetManager: Finished task 8.0 in stage 35.0 (TID 785) in 6908 ms on 10.0.0.133 (executor driver) (9/34) -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO Executor: Finished task 9.0 in stage 35.0 (TID 786). 10455 bytes result sent to driver -26/04/01 08:51:52 INFO TaskSetManager: Starting task 17.0 in stage 35.0 (TID 794) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:52 INFO Executor: Running task 17.0 in stage 35.0 (TID 794) -26/04/01 08:51:52 INFO TaskSetManager: Finished task 9.0 in stage 35.0 (TID 786) in 6936 ms on 10.0.0.133 (executor driver) (10/34) -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO Executor: Finished task 10.0 in stage 35.0 (TID 787). 10455 bytes result sent to driver -26/04/01 08:51:52 INFO TaskSetManager: Starting task 18.0 in stage 35.0 (TID 795) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:52 INFO TaskSetManager: Finished task 10.0 in stage 35.0 (TID 787) in 6947 ms on 10.0.0.133 (executor driver) (11/34) -26/04/01 08:51:52 INFO Executor: Running task 18.0 in stage 35.0 (TID 795) -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO Executor: Finished task 12.0 in stage 35.0 (TID 789). 10455 bytes result sent to driver -26/04/01 08:51:52 INFO TaskSetManager: Starting task 19.0 in stage 35.0 (TID 796) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:52 INFO TaskSetManager: Finished task 12.0 in stage 35.0 (TID 789) in 7067 ms on 10.0.0.133 (executor driver) (12/34) -26/04/01 08:51:52 INFO Executor: Running task 19.0 in stage 35.0 (TID 796) -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO Executor: Finished task 11.0 in stage 35.0 (TID 788). 10455 bytes result sent to driver -26/04/01 08:51:52 INFO TaskSetManager: Starting task 20.0 in stage 35.0 (TID 797) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:52 INFO Executor: Running task 20.0 in stage 35.0 (TID 797) -26/04/01 08:51:52 INFO TaskSetManager: Finished task 11.0 in stage 35.0 (TID 788) in 7077 ms on 10.0.0.133 (executor driver) (13/34) -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO Executor: Finished task 13.0 in stage 35.0 (TID 790). 10455 bytes result sent to driver -26/04/01 08:51:52 INFO TaskSetManager: Starting task 21.0 in stage 35.0 (TID 798) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:52 INFO TaskSetManager: Finished task 13.0 in stage 35.0 (TID 790) in 7099 ms on 10.0.0.133 (executor driver) (14/34) -26/04/01 08:51:52 INFO Executor: Running task 21.0 in stage 35.0 (TID 798) -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO Executor: Finished task 14.0 in stage 35.0 (TID 791). 10455 bytes result sent to driver -26/04/01 08:51:52 INFO TaskSetManager: Starting task 22.0 in stage 35.0 (TID 799) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:52 INFO TaskSetManager: Finished task 14.0 in stage 35.0 (TID 791) in 7087 ms on 10.0.0.133 (executor driver) (15/34) -26/04/01 08:51:52 INFO Executor: Running task 22.0 in stage 35.0 (TID 799) -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO Executor: Finished task 15.0 in stage 35.0 (TID 792). 10455 bytes result sent to driver -26/04/01 08:51:52 INFO TaskSetManager: Starting task 23.0 in stage 35.0 (TID 800) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:52 INFO TaskSetManager: Finished task 15.0 in stage 35.0 (TID 792) in 7079 ms on 10.0.0.133 (executor driver) (16/34) -26/04/01 08:51:52 INFO Executor: Running task 23.0 in stage 35.0 (TID 800) -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:52 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO Executor: Finished task 16.0 in stage 35.0 (TID 793). 9724 bytes result sent to driver -26/04/01 08:51:59 INFO TaskSetManager: Starting task 24.0 in stage 35.0 (TID 801) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:59 INFO Executor: Running task 24.0 in stage 35.0 (TID 801) -26/04/01 08:51:59 INFO TaskSetManager: Finished task 16.0 in stage 35.0 (TID 793) in 6982 ms on 10.0.0.133 (executor driver) (17/34) -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO Executor: Finished task 17.0 in stage 35.0 (TID 794). 9724 bytes result sent to driver -26/04/01 08:51:59 INFO TaskSetManager: Starting task 25.0 in stage 35.0 (TID 802) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:59 INFO TaskSetManager: Finished task 17.0 in stage 35.0 (TID 794) in 6779 ms on 10.0.0.133 (executor driver) (18/34) -26/04/01 08:51:59 INFO Executor: Running task 25.0 in stage 35.0 (TID 802) -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO Executor: Finished task 18.0 in stage 35.0 (TID 795). 9724 bytes result sent to driver -26/04/01 08:51:59 INFO TaskSetManager: Starting task 26.0 in stage 35.0 (TID 803) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:59 INFO TaskSetManager: Finished task 18.0 in stage 35.0 (TID 795) in 6878 ms on 10.0.0.133 (executor driver) (19/34) -26/04/01 08:51:59 INFO Executor: Running task 26.0 in stage 35.0 (TID 803) -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO Executor: Finished task 20.0 in stage 35.0 (TID 797). 9724 bytes result sent to driver -26/04/01 08:51:59 INFO TaskSetManager: Starting task 27.0 in stage 35.0 (TID 804) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:59 INFO TaskSetManager: Finished task 20.0 in stage 35.0 (TID 797) in 6924 ms on 10.0.0.133 (executor driver) (20/34) -26/04/01 08:51:59 INFO Executor: Running task 27.0 in stage 35.0 (TID 804) -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO Executor: Finished task 19.0 in stage 35.0 (TID 796). 9724 bytes result sent to driver -26/04/01 08:51:59 INFO TaskSetManager: Starting task 28.0 in stage 35.0 (TID 805) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:59 INFO TaskSetManager: Finished task 19.0 in stage 35.0 (TID 796) in 6955 ms on 10.0.0.133 (executor driver) (21/34) -26/04/01 08:51:59 INFO Executor: Running task 28.0 in stage 35.0 (TID 805) -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO Executor: Finished task 22.0 in stage 35.0 (TID 799). 9724 bytes result sent to driver -26/04/01 08:51:59 INFO TaskSetManager: Starting task 29.0 in stage 35.0 (TID 806) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:59 INFO Executor: Running task 29.0 in stage 35.0 (TID 806) -26/04/01 08:51:59 INFO TaskSetManager: Finished task 22.0 in stage 35.0 (TID 799) in 6959 ms on 10.0.0.133 (executor driver) (22/34) -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO Executor: Finished task 21.0 in stage 35.0 (TID 798). 9724 bytes result sent to driver -26/04/01 08:51:59 INFO TaskSetManager: Starting task 30.0 in stage 35.0 (TID 807) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:59 INFO TaskSetManager: Finished task 21.0 in stage 35.0 (TID 798) in 6993 ms on 10.0.0.133 (executor driver) (23/34) -26/04/01 08:51:59 INFO Executor: Running task 30.0 in stage 35.0 (TID 807) -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO Executor: Finished task 23.0 in stage 35.0 (TID 800). 9724 bytes result sent to driver -26/04/01 08:51:59 INFO TaskSetManager: Starting task 31.0 in stage 35.0 (TID 808) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9644 bytes) -26/04/01 08:51:59 INFO Executor: Running task 31.0 in stage 35.0 (TID 808) -26/04/01 08:51:59 INFO TaskSetManager: Finished task 23.0 in stage 35.0 (TID 800) in 7043 ms on 10.0.0.133 (executor driver) (24/34) -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Getting 64 (36.2 MiB) non-empty blocks including 64 (36.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:51:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:51:59 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:52:06 INFO Executor: Finished task 24.0 in stage 35.0 (TID 801). 10455 bytes result sent to driver -26/04/01 08:52:06 INFO TaskSetManager: Starting task 32.0 in stage 35.0 (TID 809) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9644 bytes) -26/04/01 08:52:06 INFO TaskSetManager: Finished task 24.0 in stage 35.0 (TID 801) in 6883 ms on 10.0.0.133 (executor driver) (25/34) -26/04/01 08:52:06 INFO Executor: Running task 32.0 in stage 35.0 (TID 809) -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 29 (24.4 MiB) non-empty blocks including 29 (24.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 64 (36.1 MiB) non-empty blocks including 64 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:52:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:52:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:52:06 INFO Executor: Finished task 25.0 in stage 35.0 (TID 802). 10455 bytes result sent to driver -26/04/01 08:52:06 INFO TaskSetManager: Starting task 33.0 in stage 35.0 (TID 810) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9644 bytes) -26/04/01 08:52:06 INFO Executor: Running task 33.0 in stage 35.0 (TID 810) -26/04/01 08:52:06 INFO TaskSetManager: Finished task 25.0 in stage 35.0 (TID 802) in 6981 ms on 10.0.0.133 (executor driver) (26/34) -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 29 (8.1 MiB) non-empty blocks including 29 (8.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 64 (12.0 MiB) non-empty blocks including 64 (12.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 29 (8.1 MiB) non-empty blocks including 29 (8.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Getting 64 (12.0 MiB) non-empty blocks including 64 (12.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:52:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:52:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:52:06 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:52:06 INFO Executor: Finished task 26.0 in stage 35.0 (TID 803). 10455 bytes result sent to driver -26/04/01 08:52:06 INFO TaskSetManager: Finished task 26.0 in stage 35.0 (TID 803) in 6918 ms on 10.0.0.133 (executor driver) (27/34) -26/04/01 08:52:06 INFO Executor: Finished task 27.0 in stage 35.0 (TID 804). 10455 bytes result sent to driver -26/04/01 08:52:06 INFO TaskSetManager: Finished task 27.0 in stage 35.0 (TID 804) in 6927 ms on 10.0.0.133 (executor driver) (28/34) -26/04/01 08:52:06 INFO Executor: Finished task 31.0 in stage 35.0 (TID 808). 10455 bytes result sent to driver -26/04/01 08:52:06 INFO TaskSetManager: Finished task 31.0 in stage 35.0 (TID 808) in 6776 ms on 10.0.0.133 (executor driver) (29/34) -26/04/01 08:52:06 INFO Executor: Finished task 29.0 in stage 35.0 (TID 806). 10455 bytes result sent to driver -26/04/01 08:52:06 INFO TaskSetManager: Finished task 29.0 in stage 35.0 (TID 806) in 6880 ms on 10.0.0.133 (executor driver) (30/34) -26/04/01 08:52:06 INFO Executor: Finished task 28.0 in stage 35.0 (TID 805). 10455 bytes result sent to driver -26/04/01 08:52:06 INFO TaskSetManager: Finished task 28.0 in stage 35.0 (TID 805) in 6959 ms on 10.0.0.133 (executor driver) (31/34) -26/04/01 08:52:06 INFO Executor: Finished task 30.0 in stage 35.0 (TID 807). 10455 bytes result sent to driver -26/04/01 08:52:06 INFO TaskSetManager: Finished task 30.0 in stage 35.0 (TID 807) in 6953 ms on 10.0.0.133 (executor driver) (32/34) -26/04/01 08:52:08 INFO Executor: Finished task 33.0 in stage 35.0 (TID 810). 9724 bytes result sent to driver -26/04/01 08:52:08 INFO TaskSetManager: Finished task 33.0 in stage 35.0 (TID 810) in 2145 ms on 10.0.0.133 (executor driver) (33/34) -26/04/01 08:52:12 INFO Executor: Finished task 32.0 in stage 35.0 (TID 809). 10455 bytes result sent to driver -26/04/01 08:52:12 INFO TaskSetManager: Finished task 32.0 in stage 35.0 (TID 809) in 6571 ms on 10.0.0.133 (executor driver) (34/34) -26/04/01 08:52:12 INFO TaskSchedulerImpl: Removed TaskSet 35.0, whose tasks have all completed, from pool -26/04/01 08:52:12 INFO DAGScheduler: ShuffleMapStage 35 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 34.407 s -26/04/01 08:52:12 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:52:12 INFO DAGScheduler: running: Set() -26/04/01 08:52:12 INFO DAGScheduler: waiting: Set() -26/04/01 08:52:12 INFO DAGScheduler: failed: Set() -26/04/01 08:52:12 INFO ShufflePartitionsUtil: For shuffle(8), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 08:52:12 INFO CodeGenerator: Code generated in 61.72675 ms -26/04/01 08:52:12 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:52:12 INFO DAGScheduler: Got job 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 2 output partitions -26/04/01 08:52:12 INFO DAGScheduler: Final stage: ResultStage 44 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:52:12 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 43) -26/04/01 08:52:12 INFO DAGScheduler: Missing parents: List() -26/04/01 08:52:12 INFO DAGScheduler: Submitting ResultStage 44 (MapPartitionsRDD[57] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:52:12 INFO MemoryStore: Block broadcast_27 stored as values in memory (estimated size 20.9 KiB, free 12.6 GiB) -26/04/01 08:52:12 INFO MemoryStore: Block broadcast_27_piece0 stored as bytes in memory (estimated size 8.3 KiB, free 12.6 GiB) -26/04/01 08:52:12 INFO BlockManagerInfo: Added broadcast_27_piece0 in memory on 10.0.0.133:59333 (size: 8.3 KiB, free: 12.6 GiB) -26/04/01 08:52:12 INFO SparkContext: Created broadcast 27 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:52:12 INFO DAGScheduler: Submitting 2 missing tasks from ResultStage 44 (MapPartitionsRDD[57] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1)) -26/04/01 08:52:12 INFO TaskSchedulerImpl: Adding task set 44.0 with 2 tasks resource profile 0 -26/04/01 08:52:12 INFO TaskSetManager: Starting task 0.0 in stage 44.0 (TID 811) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) -26/04/01 08:52:12 INFO TaskSetManager: Starting task 1.0 in stage 44.0 (TID 812) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9415 bytes) -26/04/01 08:52:12 INFO Executor: Running task 1.0 in stage 44.0 (TID 812) -26/04/01 08:52:12 INFO Executor: Running task 0.0 in stage 44.0 (TID 811) -26/04/01 08:52:12 INFO ShuffleBlockFetcherIterator: Getting 34 (1186.0 KiB) non-empty blocks including 34 (1186.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:52:12 INFO ShuffleBlockFetcherIterator: Getting 34 (1039.8 KiB) non-empty blocks including 34 (1039.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:52:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:52:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:52:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:52:12 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:52:12 INFO CodeGenerator: Code generated in 7.274292 ms -26/04/01 08:52:13 INFO Executor: Finished task 0.0 in stage 44.0 (TID 811). 8901 bytes result sent to driver -26/04/01 08:52:13 INFO TaskSetManager: Finished task 0.0 in stage 44.0 (TID 811) in 501 ms on 10.0.0.133 (executor driver) (1/2) -26/04/01 08:52:13 INFO Executor: Finished task 1.0 in stage 44.0 (TID 812). 9919 bytes result sent to driver -26/04/01 08:52:13 INFO TaskSetManager: Finished task 1.0 in stage 44.0 (TID 812) in 561 ms on 10.0.0.133 (executor driver) (2/2) -26/04/01 08:52:13 INFO TaskSchedulerImpl: Removed TaskSet 44.0, whose tasks have all completed, from pool -26/04/01 08:52:13 INFO DAGScheduler: ResultStage 44 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.562 s -26/04/01 08:52:13 INFO DAGScheduler: Job 19 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:52:13 INFO TaskSchedulerImpl: Killing all running tasks in stage 44: Stage finished -26/04/01 08:52:13 INFO DAGScheduler: Job 19 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.564987 s -26/04/01 08:52:13 INFO DAGScheduler: Registering RDD 58 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 9 -26/04/01 08:52:13 INFO DAGScheduler: Got map stage job 20 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 2 output partitions -26/04/01 08:52:13 INFO DAGScheduler: Final stage: ShuffleMapStage 53 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:52:13 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 52) -26/04/01 08:52:13 INFO DAGScheduler: Missing parents: List() -26/04/01 08:52:13 INFO DAGScheduler: Submitting ShuffleMapStage 53 (MapPartitionsRDD[58] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:52:13 INFO MemoryStore: Block broadcast_28 stored as values in memory (estimated size 31.4 KiB, free 12.6 GiB) -26/04/01 08:52:13 INFO MemoryStore: Block broadcast_28_piece0 stored as bytes in memory (estimated size 10.6 KiB, free 12.6 GiB) -26/04/01 08:52:13 INFO BlockManagerInfo: Added broadcast_28_piece0 in memory on 10.0.0.133:59333 (size: 10.6 KiB, free: 12.6 GiB) -26/04/01 08:52:13 INFO SparkContext: Created broadcast 28 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:52:13 INFO DAGScheduler: Submitting 2 missing tasks from ShuffleMapStage 53 (MapPartitionsRDD[58] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1)) -26/04/01 08:52:13 INFO TaskSchedulerImpl: Adding task set 53.0 with 2 tasks resource profile 0 -26/04/01 08:52:13 INFO TaskSetManager: Starting task 0.0 in stage 53.0 (TID 813) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9404 bytes) -26/04/01 08:52:13 INFO TaskSetManager: Starting task 1.0 in stage 53.0 (TID 814) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9404 bytes) -26/04/01 08:52:13 INFO Executor: Running task 1.0 in stage 53.0 (TID 814) -26/04/01 08:52:13 INFO Executor: Running task 0.0 in stage 53.0 (TID 813) -26/04/01 08:52:13 INFO CodeGenerator: Code generated in 3.569375 ms -26/04/01 08:52:13 INFO ShuffleBlockFetcherIterator: Getting 34 (1186.0 KiB) non-empty blocks including 34 (1186.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:52:13 INFO ShuffleBlockFetcherIterator: Getting 34 (1039.8 KiB) non-empty blocks including 34 (1039.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:52:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:52:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:52:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:52:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:52:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:52:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:52:13 INFO Executor: Finished task 0.0 in stage 53.0 (TID 813). 6566 bytes result sent to driver -26/04/01 08:52:13 INFO TaskSetManager: Finished task 0.0 in stage 53.0 (TID 813) in 491 ms on 10.0.0.133 (executor driver) (1/2) -26/04/01 08:52:13 INFO Executor: Finished task 1.0 in stage 53.0 (TID 814). 6566 bytes result sent to driver -26/04/01 08:52:13 INFO TaskSetManager: Finished task 1.0 in stage 53.0 (TID 814) in 553 ms on 10.0.0.133 (executor driver) (2/2) -26/04/01 08:52:13 INFO TaskSchedulerImpl: Removed TaskSet 53.0, whose tasks have all completed, from pool -26/04/01 08:52:13 INFO DAGScheduler: ShuffleMapStage 53 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.555 s -26/04/01 08:52:13 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:52:13 INFO DAGScheduler: running: Set() -26/04/01 08:52:13 INFO DAGScheduler: waiting: Set() -26/04/01 08:52:13 INFO DAGScheduler: failed: Set() -26/04/01 08:52:13 INFO ShufflePartitionsUtil: For shuffle(9), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 08:52:13 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:52:13 INFO DAGScheduler: Got job 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 08:52:13 INFO DAGScheduler: Final stage: ResultStage 63 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:52:13 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 62) -26/04/01 08:52:13 INFO DAGScheduler: Missing parents: List() -26/04/01 08:52:13 INFO DAGScheduler: Submitting ResultStage 63 (MapPartitionsRDD[62] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:52:13 INFO MemoryStore: Block broadcast_29 stored as values in memory (estimated size 28.6 KiB, free 12.6 GiB) -26/04/01 08:52:13 INFO MemoryStore: Block broadcast_29_piece0 stored as bytes in memory (estimated size 10.4 KiB, free 12.6 GiB) -26/04/01 08:52:13 INFO BlockManagerInfo: Added broadcast_29_piece0 in memory on 10.0.0.133:59333 (size: 10.4 KiB, free: 12.6 GiB) -26/04/01 08:52:13 INFO SparkContext: Created broadcast 29 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:52:13 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 63 (MapPartitionsRDD[62] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:52:13 INFO TaskSchedulerImpl: Adding task set 63.0 with 1 tasks resource profile 0 -26/04/01 08:52:13 INFO TaskSetManager: Starting task 0.0 in stage 63.0 (TID 815) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9415 bytes) -26/04/01 08:52:13 INFO Executor: Running task 0.0 in stage 63.0 (TID 815) -26/04/01 08:52:13 INFO ShuffleBlockFetcherIterator: Getting 2 (76.4 KiB) non-empty blocks including 2 (76.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:52:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:52:13 INFO CometExecIterator: memoryPoolType=fair_unified, offHeapSize=8192 MB, memoryFraction=1.0, memoryLimit=8192 MB, memoryLimitPerTask=1024 MB -26/04/01 08:52:13 INFO Executor: Finished task 0.0 in stage 63.0 (TID 815). 8247 bytes result sent to driver -26/04/01 08:52:13 INFO TaskSetManager: Finished task 0.0 in stage 63.0 (TID 815) in 47 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:52:13 INFO TaskSchedulerImpl: Removed TaskSet 63.0, whose tasks have all completed, from pool -26/04/01 08:52:13 INFO DAGScheduler: ResultStage 63 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.048 s -26/04/01 08:52:13 INFO DAGScheduler: Job 21 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:52:13 INFO TaskSchedulerImpl: Killing all running tasks in stage 63: Stage finished -26/04/01 08:52:13 INFO DAGScheduler: Job 21 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.049815 s -26/04/01 08:52:13 INFO SparkContext: SparkContext is stopping with exitCode 0. -26/04/01 08:52:13 INFO CometDriverPlugin: CometDriverPlugin shutdown -26/04/01 08:52:13 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! -26/04/01 08:52:17 INFO MemoryStore: MemoryStore cleared -26/04/01 08:52:17 INFO BlockManager: BlockManager stopped -26/04/01 08:52:17 INFO BlockManagerMaster: BlockManagerMaster stopped -26/04/01 08:52:17 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! -26/04/01 08:52:17 INFO SparkContext: Successfully stopped SparkContext -26/04/01 08:52:18 INFO ShutdownHookManager: Shutdown hook called -26/04/01 08:52:18 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-7b7c98ca-005e-4989-9d8f-bb80c5e1e895 -26/04/01 08:52:18 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-ec23fb53-cea4-469c-8aca-32f0bdc724e4 -26/04/01 08:52:18 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-7b7c98ca-005e-4989-9d8f-bb80c5e1e895/pyspark-a55b127f-06d7-458f-8190-828f6f7a60ff - 384.01 real 3099.31 user 33.62 sys - 8859074560 maximum resident set size - 0 average shared memory size - 0 average unshared data size - 0 average unshared stack size - 623152 page reclaims - 70 page faults - 0 swaps - 0 block input operations - 0 block output operations - 1278 messages sent - 2365 messages received - 36 signals received - 104418 voluntary context switches - 893124 involuntary context switches - 43657947224407 instructions retired - 11278973166768 cycles elapsed - 3867993376 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.log b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.log deleted file mode 100644 index e62bd4aaf5..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.log +++ /dev/null @@ -1,80 +0,0 @@ -Registering table customer from /opt/tpch/sf100/customer -Registering table lineitem from /opt/tpch/sf100/lineitem -Registering table nation from /opt/tpch/sf100/nation -Registering table orders from /opt/tpch/sf100/orders -Registering table part from /opt/tpch/sf100/part -Registering table partsupp from /opt/tpch/sf100/partsupp -Registering table region from /opt/tpch/sf100/region -Registering table supplier from /opt/tpch/sf100/supplier - -============================================================ -Starting iteration 1 of 1 -============================================================ - -Running query 1 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q1.sql -Executing: -- CometBench-H query 1 derived from TPC-H query 1 under the terms of the TPC Fair Use Policy. --- TP... -== Physical Plan == -AdaptiveSparkPlan (9) -+- Sort (8) - +- Exchange (7) - +- HashAggregate (6) - +- Exchange (5) - +- HashAggregate (4) - +- Project (3) - +- Filter (2) - +- Scan parquet (1) - - -(1) Scan parquet -Output [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] -PushedFilters: [IsNotNull(l_shipdate), LessThanOrEqual(l_shipdate,1998-09-24)] -ReadSchema: struct - -(2) Filter -Input [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] -Condition : (isnotnull(l_shipdate#26) AND (l_shipdate#26 <= 1998-09-24)) - -(3) Project -Output [6]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25] -Input [7]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25, l_shipdate#26] - -(4) HashAggregate -Input [6]: [l_quantity#20, l_extendedprice#21, l_discount#22, l_tax#23, l_returnflag#24, l_linestatus#25] -Keys [2]: [l_returnflag#24, l_linestatus#25] -Functions [8]: [partial_sum(l_quantity#20), partial_sum(l_extendedprice#21), partial_sum((l_extendedprice#21 * (1 - l_discount#22))), partial_sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23))), partial_avg(l_quantity#20), partial_avg(l_extendedprice#21), partial_avg(l_discount#22), partial_count(1)] -Aggregate Attributes [15]: [sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, count#157L, sum#158, count#159L, sum#160, count#161L, count#162L] -Results [17]: [l_returnflag#24, l_linestatus#25, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, count#172L, sum#173, count#174L, sum#175, count#176L, count#177L] - -(5) Exchange -Input [17]: [l_returnflag#24, l_linestatus#25, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, count#172L, sum#173, count#174L, sum#175, count#176L, count#177L] -Arguments: hashpartitioning(l_returnflag#24, l_linestatus#25, 200), ENSURE_REQUIREMENTS, [plan_id=43] - -(6) HashAggregate -Input [17]: [l_returnflag#24, l_linestatus#25, sum#163, isEmpty#164, sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, count#172L, sum#173, count#174L, sum#175, count#176L, count#177L] -Keys [2]: [l_returnflag#24, l_linestatus#25] -Functions [8]: [sum(l_quantity#20), sum(l_extendedprice#21), sum((l_extendedprice#21 * (1 - l_discount#22))), sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23))), avg(l_quantity#20), avg(l_extendedprice#21), avg(l_discount#22), count(1)] -Aggregate Attributes [8]: [sum(l_quantity#20)#131, sum(l_extendedprice#21)#132, sum((l_extendedprice#21 * (1 - l_discount#22)))#136, sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23)))#137, avg(l_quantity#20)#133, avg(l_extendedprice#21)#134, avg(l_discount#22)#135, count(1)#130L] -Results [10]: [l_returnflag#24, l_linestatus#25, sum(l_quantity#20)#131 AS sum_qty#122, sum(l_extendedprice#21)#132 AS sum_base_price#123, sum((l_extendedprice#21 * (1 - l_discount#22)))#136 AS sum_disc_price#124, sum(((l_extendedprice#21 * (1 - l_discount#22)) * (1 + l_tax#23)))#137 AS sum_charge#125, avg(l_quantity#20)#133 AS avg_qty#126, avg(l_extendedprice#21)#134 AS avg_price#127, avg(l_discount#22)#135 AS avg_disc#128, count(1)#130L AS count_order#129L] - -(7) Exchange -Input [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] -Arguments: rangepartitioning(l_returnflag#24 ASC NULLS FIRST, l_linestatus#25 ASC NULLS FIRST, 200), ENSURE_REQUIREMENTS, [plan_id=46] - -(8) Sort -Input [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] -Arguments: [l_returnflag#24 ASC NULLS FIRST, l_linestatus#25 ASC NULLS FIRST], true, 0 - -(9) AdaptiveSparkPlan -Output [10]: [l_returnflag#24, l_linestatus#25, sum_qty#122, sum_base_price#123, sum_disc_price#124, sum_charge#125, avg_qty#126, avg_price#127, avg_disc#128, count_order#129L] -Arguments: isFinalPlan=false - - -Query 1 returned 4 rows, hash=6839cc802df6220ed2efdaaaa48373a6 -Query 1 took 86.17 seconds - -Iteration 1 took 86.17 seconds - -Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/spark-offheap4g-q1-tpch-1775044569301.json diff --git a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.time b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.time deleted file mode 100644 index a304cc0253..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q1.time +++ /dev/null @@ -1,1650 +0,0 @@ -26/04/01 05:54:40 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) -26/04/01 05:54:40 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address -26/04/01 05:54:41 INFO SparkContext: Running Spark version 3.5.8 -26/04/01 05:54:41 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 05:54:41 INFO SparkContext: Java version 17.0.17 -26/04/01 05:54:41 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable -26/04/01 05:54:41 INFO ResourceUtils: ============================================================== -26/04/01 05:54:41 INFO ResourceUtils: No custom resources configured for spark.driver. -26/04/01 05:54:41 INFO ResourceUtils: ============================================================== -26/04/01 05:54:41 INFO SparkContext: Submitted application: spark-offheap4g-q1 benchmark derived from tpch -26/04/01 05:54:41 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 4096, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) -26/04/01 05:54:41 INFO ResourceProfile: Limiting resource is cpu -26/04/01 05:54:41 INFO ResourceProfileManager: Added ResourceProfile id: 0 -26/04/01 05:54:41 INFO SecurityManager: Changing view acls to: andy -26/04/01 05:54:41 INFO SecurityManager: Changing modify acls to: andy -26/04/01 05:54:41 INFO SecurityManager: Changing view acls groups to: -26/04/01 05:54:41 INFO SecurityManager: Changing modify acls groups to: -26/04/01 05:54:41 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY -26/04/01 05:54:41 INFO Utils: Successfully started service 'sparkDriver' on port 58186. -26/04/01 05:54:41 INFO SparkEnv: Registering MapOutputTracker -26/04/01 05:54:41 INFO SparkEnv: Registering BlockManagerMaster -26/04/01 05:54:41 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information -26/04/01 05:54:41 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up -26/04/01 05:54:41 INFO SparkEnv: Registering BlockManagerMasterHeartbeat -26/04/01 05:54:41 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-4f91f881-2f04-42af-8e2f-9947d49e7a91 -26/04/01 05:54:41 INFO MemoryStore: MemoryStore started with capacity 8.6 GiB -26/04/01 05:54:41 INFO SparkEnv: Registering OutputCommitCoordinator -26/04/01 05:54:41 INFO Executor: Starting executor ID driver on host 10.0.0.133 -26/04/01 05:54:41 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 05:54:41 INFO Executor: Java version 17.0.17 -26/04/01 05:54:41 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' -26/04/01 05:54:41 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@1b3b2259 for default. -26/04/01 05:54:41 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 58187. -26/04/01 05:54:41 INFO NettyBlockTransferService: Server created on 10.0.0.133:58187 -26/04/01 05:54:41 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy -26/04/01 05:54:41 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 58187, None) -26/04/01 05:54:41 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:58187 with 8.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 58187, None) -26/04/01 05:54:41 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 58187, None) -26/04/01 05:54:41 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 58187, None) -26/04/01 05:54:41 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. -26/04/01 05:54:41 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. -26/04/01 05:54:41 INFO InMemoryFileIndex: It took 21 ms to list leaf files for 1 paths. -26/04/01 05:54:42 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:54:42 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:54:42 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:54:42 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:54:42 INFO DAGScheduler: Missing parents: List() -26/04/01 05:54:42 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:54:42 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:54:42 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:54:42 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:42 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:54:42 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:54:42 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 -26/04/01 05:54:42 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) -26/04/01 05:54:42 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) -26/04/01 05:54:42 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 1998 bytes result sent to driver -26/04/01 05:54:42 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 127 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:54:42 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool -26/04/01 05:54:42 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.328 s -26/04/01 05:54:42 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:54:42 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished -26/04/01 05:54:42 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.339591 s -26/04/01 05:54:42 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:42 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. -26/04/01 05:54:42 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:54:42 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:54:42 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:54:42 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:54:42 INFO DAGScheduler: Missing parents: List() -26/04/01 05:54:42 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:54:42 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:54:42 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:54:42 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:42 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:54:42 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:54:42 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 -26/04/01 05:54:42 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) -26/04/01 05:54:42 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) -26/04/01 05:54:42 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2344 bytes result sent to driver -26/04/01 05:54:42 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 15 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:54:42 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool -26/04/01 05:54:42 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.023 s -26/04/01 05:54:42 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:54:42 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished -26/04/01 05:54:42 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.025069 s -26/04/01 05:54:42 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. -26/04/01 05:54:42 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:54:42 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:54:42 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:54:42 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:54:42 INFO DAGScheduler: Missing parents: List() -26/04/01 05:54:42 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:54:42 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:54:42 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:54:42 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:42 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:54:42 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:54:42 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 -26/04/01 05:54:42 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) -26/04/01 05:54:42 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) -26/04/01 05:54:42 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver -26/04/01 05:54:42 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 7 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:54:42 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool -26/04/01 05:54:42 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.016 s -26/04/01 05:54:42 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:54:42 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished -26/04/01 05:54:42 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.017186 s -26/04/01 05:54:42 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. -26/04/01 05:54:42 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:54:42 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:54:42 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:54:42 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:54:42 INFO DAGScheduler: Missing parents: List() -26/04/01 05:54:42 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:54:42 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:42 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:54:42 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:42 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:54:42 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:42 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:54:42 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:54:42 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 -26/04/01 05:54:42 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) -26/04/01 05:54:42 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) -26/04/01 05:54:42 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver -26/04/01 05:54:42 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 7 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:54:42 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool -26/04/01 05:54:42 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.017 s -26/04/01 05:54:42 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:54:42 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished -26/04/01 05:54:42 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.018134 s -26/04/01 05:54:42 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. -26/04/01 05:54:42 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:54:42 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:54:42 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:54:42 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:54:42 INFO DAGScheduler: Missing parents: List() -26/04/01 05:54:42 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:54:42 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:54:42 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:54:42 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:42 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:54:42 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:54:42 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 -26/04/01 05:54:42 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9134 bytes) -26/04/01 05:54:42 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) -26/04/01 05:54:42 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver -26/04/01 05:54:42 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:54:42 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool -26/04/01 05:54:42 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s -26/04/01 05:54:42 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:54:42 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished -26/04/01 05:54:42 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.015374 s -26/04/01 05:54:42 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. -26/04/01 05:54:43 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:54:43 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:54:43 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:54:43 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:54:43 INFO DAGScheduler: Missing parents: List() -26/04/01 05:54:43 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:54:43 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:54:43 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:54:43 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:43 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:43 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:54:43 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:54:43 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 -26/04/01 05:54:43 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) -26/04/01 05:54:43 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:43 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) -26/04/01 05:54:43 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver -26/04/01 05:54:43 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:54:43 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool -26/04/01 05:54:43 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s -26/04/01 05:54:43 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:54:43 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished -26/04/01 05:54:43 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.016021 s -26/04/01 05:54:43 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. -26/04/01 05:54:43 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:54:43 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:54:43 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:54:43 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:54:43 INFO DAGScheduler: Missing parents: List() -26/04/01 05:54:43 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:54:43 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:54:43 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:54:43 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:43 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:54:43 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:54:43 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 -26/04/01 05:54:43 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) -26/04/01 05:54:43 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) -26/04/01 05:54:43 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver -26/04/01 05:54:43 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:54:43 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool -26/04/01 05:54:43 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.011 s -26/04/01 05:54:43 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:54:43 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished -26/04/01 05:54:43 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.013125 s -26/04/01 05:54:43 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 05:54:43 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:54:43 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:54:43 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:54:43 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:54:43 INFO DAGScheduler: Missing parents: List() -26/04/01 05:54:43 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:54:43 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:54:43 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:54:43 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:43 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:58187 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:43 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:54:43 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:54:43 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 -26/04/01 05:54:43 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) -26/04/01 05:54:43 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:43 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) -26/04/01 05:54:43 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver -26/04/01 05:54:43 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:54:43 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool -26/04/01 05:54:43 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s -26/04/01 05:54:43 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:54:43 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished -26/04/01 05:54:43 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.016616 s -26/04/01 05:54:43 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_shipdate),LessThanOrEqual(l_shipdate,1998-09-24) -26/04/01 05:54:43 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_shipdate#26),(l_shipdate#26 <= 1998-09-24) -26/04/01 05:54:43 INFO HashAggregateExec: spark.sql.codegen.aggregate.map.twolevel.enabled is set to true, but current version of codegened fast hashmap does not support this aggregate. -26/04/01 05:54:43 INFO CodeGenerator: Code generated in 113.466292 ms -26/04/01 05:54:43 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 202.5 KiB, free 8.6 GiB) -26/04/01 05:54:43 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.3 KiB, free 8.6 GiB) -26/04/01 05:54:43 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:58187 (size: 35.3 KiB, free: 8.6 GiB) -26/04/01 05:54:43 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:54:43 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 05:54:43 INFO DAGScheduler: Registering RDD 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 -26/04/01 05:54:43 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions -26/04/01 05:54:43 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:54:43 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:54:43 INFO DAGScheduler: Missing parents: List() -26/04/01 05:54:43 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[19] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:54:43 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 72.6 KiB, free 8.6 GiB) -26/04/01 05:54:43 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 27.2 KiB, free 8.6 GiB) -26/04/01 05:54:43 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:58187 (size: 27.2 KiB, free: 8.6 GiB) -26/04/01 05:54:43 INFO SparkContext: Created broadcast 9 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:54:43 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[19] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 05:54:43 INFO TaskSchedulerImpl: Adding task set 8.0 with 208 tasks resource profile 0 -26/04/01 05:54:43 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:43 INFO TaskSetManager: Starting task 1.0 in stage 8.0 (TID 9) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:43 INFO TaskSetManager: Starting task 2.0 in stage 8.0 (TID 10) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:43 INFO TaskSetManager: Starting task 3.0 in stage 8.0 (TID 11) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:43 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) -26/04/01 05:54:43 INFO Executor: Running task 1.0 in stage 8.0 (TID 9) -26/04/01 05:54:43 INFO Executor: Running task 2.0 in stage 8.0 (TID 10) -26/04/01 05:54:43 INFO Executor: Running task 3.0 in stage 8.0 (TID 11) -26/04/01 05:54:43 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:58187 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:54:43 INFO CodeGenerator: Code generated in 21.949417 ms -26/04/01 05:54:43 INFO CodeGenerator: Code generated in 5.071291 ms -26/04/01 05:54:43 INFO CodeGenerator: Code generated in 2.986917 ms -26/04/01 05:54:43 INFO CodeGenerator: Code generated in 2.117208 ms -26/04/01 05:54:43 INFO CodeGenerator: Code generated in 5.845834 ms -26/04/01 05:54:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:54:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:54:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:54:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:54:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:43 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 05:54:43 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 05:54:43 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 05:54:43 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 05:54:45 INFO Executor: Finished task 3.0 in stage 8.0 (TID 11). 3027 bytes result sent to driver -26/04/01 05:54:45 INFO Executor: Finished task 1.0 in stage 8.0 (TID 9). 3027 bytes result sent to driver -26/04/01 05:54:45 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 3027 bytes result sent to driver -26/04/01 05:54:45 INFO Executor: Finished task 2.0 in stage 8.0 (TID 10). 3027 bytes result sent to driver -26/04/01 05:54:45 INFO TaskSetManager: Starting task 4.0 in stage 8.0 (TID 12) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:45 INFO Executor: Running task 4.0 in stage 8.0 (TID 12) -26/04/01 05:54:45 INFO TaskSetManager: Starting task 5.0 in stage 8.0 (TID 13) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:45 INFO TaskSetManager: Finished task 3.0 in stage 8.0 (TID 11) in 2319 ms on 10.0.0.133 (executor driver) (1/208) -26/04/01 05:54:45 INFO Executor: Running task 5.0 in stage 8.0 (TID 13) -26/04/01 05:54:45 INFO TaskSetManager: Starting task 6.0 in stage 8.0 (TID 14) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:45 INFO Executor: Running task 6.0 in stage 8.0 (TID 14) -26/04/01 05:54:45 INFO TaskSetManager: Finished task 2.0 in stage 8.0 (TID 10) in 2320 ms on 10.0.0.133 (executor driver) (2/208) -26/04/01 05:54:45 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 2322 ms on 10.0.0.133 (executor driver) (3/208) -26/04/01 05:54:45 INFO TaskSetManager: Starting task 7.0 in stage 8.0 (TID 15) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:45 INFO Executor: Running task 7.0 in stage 8.0 (TID 15) -26/04/01 05:54:45 INFO TaskSetManager: Finished task 1.0 in stage 8.0 (TID 9) in 2321 ms on 10.0.0.133 (executor driver) (4/208) -26/04/01 05:54:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:54:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:54:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:54:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:54:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:47 INFO Executor: Finished task 7.0 in stage 8.0 (TID 15). 3027 bytes result sent to driver -26/04/01 05:54:47 INFO Executor: Finished task 6.0 in stage 8.0 (TID 14). 3027 bytes result sent to driver -26/04/01 05:54:47 INFO TaskSetManager: Starting task 8.0 in stage 8.0 (TID 16) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:47 INFO Executor: Running task 8.0 in stage 8.0 (TID 16) -26/04/01 05:54:47 INFO TaskSetManager: Starting task 9.0 in stage 8.0 (TID 17) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:47 INFO TaskSetManager: Finished task 7.0 in stage 8.0 (TID 15) in 1741 ms on 10.0.0.133 (executor driver) (5/208) -26/04/01 05:54:47 INFO Executor: Running task 9.0 in stage 8.0 (TID 17) -26/04/01 05:54:47 INFO TaskSetManager: Finished task 6.0 in stage 8.0 (TID 14) in 1742 ms on 10.0.0.133 (executor driver) (6/208) -26/04/01 05:54:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:54:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:54:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:47 INFO Executor: Finished task 5.0 in stage 8.0 (TID 13). 3027 bytes result sent to driver -26/04/01 05:54:47 INFO TaskSetManager: Starting task 10.0 in stage 8.0 (TID 18) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:47 INFO TaskSetManager: Finished task 5.0 in stage 8.0 (TID 13) in 1761 ms on 10.0.0.133 (executor driver) (7/208) -26/04/01 05:54:47 INFO Executor: Running task 10.0 in stage 8.0 (TID 18) -26/04/01 05:54:47 INFO Executor: Finished task 4.0 in stage 8.0 (TID 12). 3027 bytes result sent to driver -26/04/01 05:54:47 INFO TaskSetManager: Starting task 11.0 in stage 8.0 (TID 19) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:47 INFO Executor: Running task 11.0 in stage 8.0 (TID 19) -26/04/01 05:54:47 INFO TaskSetManager: Finished task 4.0 in stage 8.0 (TID 12) in 1767 ms on 10.0.0.133 (executor driver) (8/208) -26/04/01 05:54:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:54:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:54:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:49 INFO Executor: Finished task 9.0 in stage 8.0 (TID 17). 3027 bytes result sent to driver -26/04/01 05:54:49 INFO TaskSetManager: Starting task 12.0 in stage 8.0 (TID 20) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:49 INFO Executor: Running task 12.0 in stage 8.0 (TID 20) -26/04/01 05:54:49 INFO TaskSetManager: Finished task 9.0 in stage 8.0 (TID 17) in 1708 ms on 10.0.0.133 (executor driver) (9/208) -26/04/01 05:54:49 INFO Executor: Finished task 8.0 in stage 8.0 (TID 16). 3027 bytes result sent to driver -26/04/01 05:54:49 INFO TaskSetManager: Starting task 13.0 in stage 8.0 (TID 21) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:49 INFO TaskSetManager: Finished task 8.0 in stage 8.0 (TID 16) in 1712 ms on 10.0.0.133 (executor driver) (10/208) -26/04/01 05:54:49 INFO Executor: Running task 13.0 in stage 8.0 (TID 21) -26/04/01 05:54:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:54:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:54:49 INFO Executor: Finished task 10.0 in stage 8.0 (TID 18). 3027 bytes result sent to driver -26/04/01 05:54:49 INFO TaskSetManager: Starting task 14.0 in stage 8.0 (TID 22) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:49 INFO TaskSetManager: Finished task 10.0 in stage 8.0 (TID 18) in 1700 ms on 10.0.0.133 (executor driver) (11/208) -26/04/01 05:54:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:49 INFO Executor: Running task 14.0 in stage 8.0 (TID 22) -26/04/01 05:54:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:49 INFO Executor: Finished task 11.0 in stage 8.0 (TID 19). 3027 bytes result sent to driver -26/04/01 05:54:49 INFO TaskSetManager: Starting task 15.0 in stage 8.0 (TID 23) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:54:49 INFO TaskSetManager: Finished task 11.0 in stage 8.0 (TID 19) in 1700 ms on 10.0.0.133 (executor driver) (12/208) -26/04/01 05:54:49 INFO Executor: Running task 15.0 in stage 8.0 (TID 23) -26/04/01 05:54:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:54:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:51 INFO Executor: Finished task 13.0 in stage 8.0 (TID 21). 3027 bytes result sent to driver -26/04/01 05:54:51 INFO Executor: Finished task 12.0 in stage 8.0 (TID 20). 3027 bytes result sent to driver -26/04/01 05:54:51 INFO TaskSetManager: Starting task 16.0 in stage 8.0 (TID 24) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:51 INFO TaskSetManager: Finished task 13.0 in stage 8.0 (TID 21) in 1664 ms on 10.0.0.133 (executor driver) (13/208) -26/04/01 05:54:51 INFO Executor: Running task 16.0 in stage 8.0 (TID 24) -26/04/01 05:54:51 INFO TaskSetManager: Starting task 17.0 in stage 8.0 (TID 25) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:51 INFO TaskSetManager: Finished task 12.0 in stage 8.0 (TID 20) in 1668 ms on 10.0.0.133 (executor driver) (14/208) -26/04/01 05:54:51 INFO Executor: Running task 17.0 in stage 8.0 (TID 25) -26/04/01 05:54:51 INFO Executor: Finished task 14.0 in stage 8.0 (TID 22). 3027 bytes result sent to driver -26/04/01 05:54:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:54:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:54:51 INFO TaskSetManager: Starting task 18.0 in stage 8.0 (TID 26) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:54:51 INFO TaskSetManager: Finished task 14.0 in stage 8.0 (TID 22) in 1662 ms on 10.0.0.133 (executor driver) (15/208) -26/04/01 05:54:51 INFO Executor: Running task 18.0 in stage 8.0 (TID 26) -26/04/01 05:54:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:54:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:51 INFO Executor: Finished task 15.0 in stage 8.0 (TID 23). 3027 bytes result sent to driver -26/04/01 05:54:51 INFO TaskSetManager: Starting task 19.0 in stage 8.0 (TID 27) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:54:51 INFO Executor: Running task 19.0 in stage 8.0 (TID 27) -26/04/01 05:54:51 INFO TaskSetManager: Finished task 15.0 in stage 8.0 (TID 23) in 1664 ms on 10.0.0.133 (executor driver) (16/208) -26/04/01 05:54:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:54:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:52 INFO Executor: Finished task 16.0 in stage 8.0 (TID 24). 3027 bytes result sent to driver -26/04/01 05:54:52 INFO TaskSetManager: Starting task 20.0 in stage 8.0 (TID 28) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:54:52 INFO TaskSetManager: Finished task 16.0 in stage 8.0 (TID 24) in 1686 ms on 10.0.0.133 (executor driver) (17/208) -26/04/01 05:54:52 INFO Executor: Running task 20.0 in stage 8.0 (TID 28) -26/04/01 05:54:52 INFO Executor: Finished task 19.0 in stage 8.0 (TID 27). 3027 bytes result sent to driver -26/04/01 05:54:52 INFO TaskSetManager: Starting task 21.0 in stage 8.0 (TID 29) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:54:52 INFO Executor: Running task 21.0 in stage 8.0 (TID 29) -26/04/01 05:54:52 INFO TaskSetManager: Finished task 19.0 in stage 8.0 (TID 27) in 1675 ms on 10.0.0.133 (executor driver) (18/208) -26/04/01 05:54:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:54:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:54:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:52 INFO Executor: Finished task 18.0 in stage 8.0 (TID 26). 3027 bytes result sent to driver -26/04/01 05:54:52 INFO TaskSetManager: Starting task 22.0 in stage 8.0 (TID 30) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:54:52 INFO TaskSetManager: Finished task 18.0 in stage 8.0 (TID 26) in 1706 ms on 10.0.0.133 (executor driver) (19/208) -26/04/01 05:54:52 INFO Executor: Running task 22.0 in stage 8.0 (TID 30) -26/04/01 05:54:52 INFO Executor: Finished task 17.0 in stage 8.0 (TID 25). 3027 bytes result sent to driver -26/04/01 05:54:52 INFO TaskSetManager: Starting task 23.0 in stage 8.0 (TID 31) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:54:52 INFO TaskSetManager: Finished task 17.0 in stage 8.0 (TID 25) in 1712 ms on 10.0.0.133 (executor driver) (20/208) -26/04/01 05:54:52 INFO Executor: Running task 23.0 in stage 8.0 (TID 31) -26/04/01 05:54:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:54:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:54:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:54 INFO Executor: Finished task 21.0 in stage 8.0 (TID 29). 3027 bytes result sent to driver -26/04/01 05:54:54 INFO TaskSetManager: Starting task 24.0 in stage 8.0 (TID 32) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:54 INFO Executor: Running task 24.0 in stage 8.0 (TID 32) -26/04/01 05:54:54 INFO TaskSetManager: Finished task 21.0 in stage 8.0 (TID 29) in 1659 ms on 10.0.0.133 (executor driver) (21/208) -26/04/01 05:54:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:54:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:54 INFO Executor: Finished task 20.0 in stage 8.0 (TID 28). 3027 bytes result sent to driver -26/04/01 05:54:54 INFO TaskSetManager: Starting task 25.0 in stage 8.0 (TID 33) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:54 INFO TaskSetManager: Finished task 20.0 in stage 8.0 (TID 28) in 1681 ms on 10.0.0.133 (executor driver) (22/208) -26/04/01 05:54:54 INFO Executor: Running task 25.0 in stage 8.0 (TID 33) -26/04/01 05:54:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:54:54 INFO Executor: Finished task 22.0 in stage 8.0 (TID 30). 3027 bytes result sent to driver -26/04/01 05:54:54 INFO TaskSetManager: Starting task 26.0 in stage 8.0 (TID 34) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:54 INFO TaskSetManager: Finished task 22.0 in stage 8.0 (TID 30) in 1662 ms on 10.0.0.133 (executor driver) (23/208) -26/04/01 05:54:54 INFO Executor: Running task 26.0 in stage 8.0 (TID 34) -26/04/01 05:54:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:54:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:54 INFO Executor: Finished task 23.0 in stage 8.0 (TID 31). 2984 bytes result sent to driver -26/04/01 05:54:54 INFO TaskSetManager: Starting task 27.0 in stage 8.0 (TID 35) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:54 INFO TaskSetManager: Finished task 23.0 in stage 8.0 (TID 31) in 1672 ms on 10.0.0.133 (executor driver) (24/208) -26/04/01 05:54:54 INFO Executor: Running task 27.0 in stage 8.0 (TID 35) -26/04/01 05:54:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:54:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:56 INFO Executor: Finished task 24.0 in stage 8.0 (TID 32). 3027 bytes result sent to driver -26/04/01 05:54:56 INFO TaskSetManager: Starting task 28.0 in stage 8.0 (TID 36) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:56 INFO TaskSetManager: Finished task 24.0 in stage 8.0 (TID 32) in 1680 ms on 10.0.0.133 (executor driver) (25/208) -26/04/01 05:54:56 INFO Executor: Running task 28.0 in stage 8.0 (TID 36) -26/04/01 05:54:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:54:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:56 INFO Executor: Finished task 25.0 in stage 8.0 (TID 33). 3027 bytes result sent to driver -26/04/01 05:54:56 INFO TaskSetManager: Starting task 29.0 in stage 8.0 (TID 37) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:56 INFO TaskSetManager: Finished task 25.0 in stage 8.0 (TID 33) in 1683 ms on 10.0.0.133 (executor driver) (26/208) -26/04/01 05:54:56 INFO Executor: Running task 29.0 in stage 8.0 (TID 37) -26/04/01 05:54:56 INFO Executor: Finished task 26.0 in stage 8.0 (TID 34). 3027 bytes result sent to driver -26/04/01 05:54:56 INFO TaskSetManager: Starting task 30.0 in stage 8.0 (TID 38) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:56 INFO Executor: Running task 30.0 in stage 8.0 (TID 38) -26/04/01 05:54:56 INFO TaskSetManager: Finished task 26.0 in stage 8.0 (TID 34) in 1677 ms on 10.0.0.133 (executor driver) (27/208) -26/04/01 05:54:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:54:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:54:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:56 INFO Executor: Finished task 27.0 in stage 8.0 (TID 35). 3027 bytes result sent to driver -26/04/01 05:54:56 INFO TaskSetManager: Starting task 31.0 in stage 8.0 (TID 39) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:56 INFO TaskSetManager: Finished task 27.0 in stage 8.0 (TID 35) in 1679 ms on 10.0.0.133 (executor driver) (28/208) -26/04/01 05:54:56 INFO Executor: Running task 31.0 in stage 8.0 (TID 39) -26/04/01 05:54:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:54:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:57 INFO Executor: Finished task 28.0 in stage 8.0 (TID 36). 3027 bytes result sent to driver -26/04/01 05:54:57 INFO TaskSetManager: Starting task 32.0 in stage 8.0 (TID 40) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:57 INFO Executor: Running task 32.0 in stage 8.0 (TID 40) -26/04/01 05:54:57 INFO TaskSetManager: Finished task 28.0 in stage 8.0 (TID 36) in 1687 ms on 10.0.0.133 (executor driver) (29/208) -26/04/01 05:54:57 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:54:57 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:57 INFO Executor: Finished task 30.0 in stage 8.0 (TID 38). 3027 bytes result sent to driver -26/04/01 05:54:57 INFO TaskSetManager: Starting task 33.0 in stage 8.0 (TID 41) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:57 INFO Executor: Running task 33.0 in stage 8.0 (TID 41) -26/04/01 05:54:57 INFO TaskSetManager: Finished task 30.0 in stage 8.0 (TID 38) in 1681 ms on 10.0.0.133 (executor driver) (30/208) -26/04/01 05:54:57 INFO Executor: Finished task 29.0 in stage 8.0 (TID 37). 3027 bytes result sent to driver -26/04/01 05:54:57 INFO TaskSetManager: Starting task 34.0 in stage 8.0 (TID 42) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:57 INFO TaskSetManager: Finished task 29.0 in stage 8.0 (TID 37) in 1685 ms on 10.0.0.133 (executor driver) (31/208) -26/04/01 05:54:57 INFO Executor: Running task 34.0 in stage 8.0 (TID 42) -26/04/01 05:54:57 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:54:57 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:54:57 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:57 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:57 INFO Executor: Finished task 31.0 in stage 8.0 (TID 39). 3027 bytes result sent to driver -26/04/01 05:54:57 INFO TaskSetManager: Starting task 35.0 in stage 8.0 (TID 43) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:57 INFO TaskSetManager: Finished task 31.0 in stage 8.0 (TID 39) in 1682 ms on 10.0.0.133 (executor driver) (32/208) -26/04/01 05:54:57 INFO Executor: Running task 35.0 in stage 8.0 (TID 43) -26/04/01 05:54:57 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:54:57 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:59 INFO Executor: Finished task 32.0 in stage 8.0 (TID 40). 3027 bytes result sent to driver -26/04/01 05:54:59 INFO TaskSetManager: Starting task 36.0 in stage 8.0 (TID 44) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:59 INFO TaskSetManager: Finished task 32.0 in stage 8.0 (TID 40) in 1679 ms on 10.0.0.133 (executor driver) (33/208) -26/04/01 05:54:59 INFO Executor: Running task 36.0 in stage 8.0 (TID 44) -26/04/01 05:54:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:54:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:59 INFO Executor: Finished task 33.0 in stage 8.0 (TID 41). 3027 bytes result sent to driver -26/04/01 05:54:59 INFO TaskSetManager: Starting task 37.0 in stage 8.0 (TID 45) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:59 INFO Executor: Finished task 34.0 in stage 8.0 (TID 42). 3027 bytes result sent to driver -26/04/01 05:54:59 INFO Executor: Running task 37.0 in stage 8.0 (TID 45) -26/04/01 05:54:59 INFO TaskSetManager: Starting task 38.0 in stage 8.0 (TID 46) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:59 INFO TaskSetManager: Finished task 33.0 in stage 8.0 (TID 41) in 1671 ms on 10.0.0.133 (executor driver) (34/208) -26/04/01 05:54:59 INFO Executor: Running task 38.0 in stage 8.0 (TID 46) -26/04/01 05:54:59 INFO TaskSetManager: Finished task 34.0 in stage 8.0 (TID 42) in 1669 ms on 10.0.0.133 (executor driver) (35/208) -26/04/01 05:54:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:54:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:54:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:54:59 INFO Executor: Finished task 35.0 in stage 8.0 (TID 43). 3027 bytes result sent to driver -26/04/01 05:54:59 INFO TaskSetManager: Starting task 39.0 in stage 8.0 (TID 47) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:54:59 INFO TaskSetManager: Finished task 35.0 in stage 8.0 (TID 43) in 1673 ms on 10.0.0.133 (executor driver) (36/208) -26/04/01 05:54:59 INFO Executor: Running task 39.0 in stage 8.0 (TID 47) -26/04/01 05:54:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:54:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:01 INFO Executor: Finished task 36.0 in stage 8.0 (TID 44). 3027 bytes result sent to driver -26/04/01 05:55:01 INFO TaskSetManager: Starting task 40.0 in stage 8.0 (TID 48) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:01 INFO Executor: Running task 40.0 in stage 8.0 (TID 48) -26/04/01 05:55:01 INFO TaskSetManager: Finished task 36.0 in stage 8.0 (TID 44) in 1669 ms on 10.0.0.133 (executor driver) (37/208) -26/04/01 05:55:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:01 INFO Executor: Finished task 38.0 in stage 8.0 (TID 46). 3027 bytes result sent to driver -26/04/01 05:55:01 INFO Executor: Finished task 37.0 in stage 8.0 (TID 45). 3027 bytes result sent to driver -26/04/01 05:55:01 INFO TaskSetManager: Starting task 41.0 in stage 8.0 (TID 49) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:01 INFO Executor: Running task 41.0 in stage 8.0 (TID 49) -26/04/01 05:55:01 INFO TaskSetManager: Starting task 42.0 in stage 8.0 (TID 50) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:01 INFO Executor: Running task 42.0 in stage 8.0 (TID 50) -26/04/01 05:55:01 INFO TaskSetManager: Finished task 38.0 in stage 8.0 (TID 46) in 1674 ms on 10.0.0.133 (executor driver) (38/208) -26/04/01 05:55:01 INFO TaskSetManager: Finished task 37.0 in stage 8.0 (TID 45) in 1674 ms on 10.0.0.133 (executor driver) (39/208) -26/04/01 05:55:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:01 INFO Executor: Finished task 39.0 in stage 8.0 (TID 47). 3027 bytes result sent to driver -26/04/01 05:55:01 INFO TaskSetManager: Starting task 43.0 in stage 8.0 (TID 51) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:01 INFO TaskSetManager: Finished task 39.0 in stage 8.0 (TID 47) in 1675 ms on 10.0.0.133 (executor driver) (40/208) -26/04/01 05:55:01 INFO Executor: Running task 43.0 in stage 8.0 (TID 51) -26/04/01 05:55:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:02 INFO Executor: Finished task 40.0 in stage 8.0 (TID 48). 3027 bytes result sent to driver -26/04/01 05:55:02 INFO TaskSetManager: Starting task 44.0 in stage 8.0 (TID 52) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:02 INFO TaskSetManager: Finished task 40.0 in stage 8.0 (TID 48) in 1683 ms on 10.0.0.133 (executor driver) (41/208) -26/04/01 05:55:02 INFO Executor: Running task 44.0 in stage 8.0 (TID 52) -26/04/01 05:55:02 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:02 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:02 INFO Executor: Finished task 41.0 in stage 8.0 (TID 49). 3027 bytes result sent to driver -26/04/01 05:55:02 INFO TaskSetManager: Starting task 45.0 in stage 8.0 (TID 53) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:02 INFO TaskSetManager: Finished task 41.0 in stage 8.0 (TID 49) in 1676 ms on 10.0.0.133 (executor driver) (42/208) -26/04/01 05:55:02 INFO Executor: Running task 45.0 in stage 8.0 (TID 53) -26/04/01 05:55:02 INFO Executor: Finished task 42.0 in stage 8.0 (TID 50). 2984 bytes result sent to driver -26/04/01 05:55:02 INFO TaskSetManager: Starting task 46.0 in stage 8.0 (TID 54) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:02 INFO Executor: Running task 46.0 in stage 8.0 (TID 54) -26/04/01 05:55:02 INFO TaskSetManager: Finished task 42.0 in stage 8.0 (TID 50) in 1679 ms on 10.0.0.133 (executor driver) (43/208) -26/04/01 05:55:02 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:02 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:02 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:02 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:02 INFO Executor: Finished task 43.0 in stage 8.0 (TID 51). 3027 bytes result sent to driver -26/04/01 05:55:02 INFO TaskSetManager: Starting task 47.0 in stage 8.0 (TID 55) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:02 INFO TaskSetManager: Finished task 43.0 in stage 8.0 (TID 51) in 1671 ms on 10.0.0.133 (executor driver) (44/208) -26/04/01 05:55:02 INFO Executor: Running task 47.0 in stage 8.0 (TID 55) -26/04/01 05:55:02 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:02 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:04 INFO Executor: Finished task 44.0 in stage 8.0 (TID 52). 3027 bytes result sent to driver -26/04/01 05:55:04 INFO TaskSetManager: Starting task 48.0 in stage 8.0 (TID 56) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:04 INFO TaskSetManager: Finished task 44.0 in stage 8.0 (TID 52) in 1678 ms on 10.0.0.133 (executor driver) (45/208) -26/04/01 05:55:04 INFO Executor: Running task 48.0 in stage 8.0 (TID 56) -26/04/01 05:55:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:04 INFO Executor: Finished task 45.0 in stage 8.0 (TID 53). 3027 bytes result sent to driver -26/04/01 05:55:04 INFO TaskSetManager: Starting task 49.0 in stage 8.0 (TID 57) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:04 INFO Executor: Running task 49.0 in stage 8.0 (TID 57) -26/04/01 05:55:04 INFO TaskSetManager: Finished task 45.0 in stage 8.0 (TID 53) in 1682 ms on 10.0.0.133 (executor driver) (46/208) -26/04/01 05:55:04 INFO Executor: Finished task 46.0 in stage 8.0 (TID 54). 2984 bytes result sent to driver -26/04/01 05:55:04 INFO TaskSetManager: Starting task 50.0 in stage 8.0 (TID 58) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:04 INFO TaskSetManager: Finished task 46.0 in stage 8.0 (TID 54) in 1680 ms on 10.0.0.133 (executor driver) (47/208) -26/04/01 05:55:04 INFO Executor: Running task 50.0 in stage 8.0 (TID 58) -26/04/01 05:55:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:04 INFO Executor: Finished task 47.0 in stage 8.0 (TID 55). 2984 bytes result sent to driver -26/04/01 05:55:04 INFO TaskSetManager: Starting task 51.0 in stage 8.0 (TID 59) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:04 INFO Executor: Running task 51.0 in stage 8.0 (TID 59) -26/04/01 05:55:04 INFO TaskSetManager: Finished task 47.0 in stage 8.0 (TID 55) in 1678 ms on 10.0.0.133 (executor driver) (48/208) -26/04/01 05:55:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:06 INFO Executor: Finished task 48.0 in stage 8.0 (TID 56). 3027 bytes result sent to driver -26/04/01 05:55:06 INFO TaskSetManager: Starting task 52.0 in stage 8.0 (TID 60) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:06 INFO Executor: Running task 52.0 in stage 8.0 (TID 60) -26/04/01 05:55:06 INFO TaskSetManager: Finished task 48.0 in stage 8.0 (TID 56) in 1681 ms on 10.0.0.133 (executor driver) (49/208) -26/04/01 05:55:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:06 INFO Executor: Finished task 49.0 in stage 8.0 (TID 57). 3027 bytes result sent to driver -26/04/01 05:55:06 INFO TaskSetManager: Starting task 53.0 in stage 8.0 (TID 61) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:06 INFO Executor: Running task 53.0 in stage 8.0 (TID 61) -26/04/01 05:55:06 INFO TaskSetManager: Finished task 49.0 in stage 8.0 (TID 57) in 1677 ms on 10.0.0.133 (executor driver) (50/208) -26/04/01 05:55:06 INFO Executor: Finished task 50.0 in stage 8.0 (TID 58). 3027 bytes result sent to driver -26/04/01 05:55:06 INFO TaskSetManager: Starting task 54.0 in stage 8.0 (TID 62) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:06 INFO TaskSetManager: Finished task 50.0 in stage 8.0 (TID 58) in 1679 ms on 10.0.0.133 (executor driver) (51/208) -26/04/01 05:55:06 INFO Executor: Running task 54.0 in stage 8.0 (TID 62) -26/04/01 05:55:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:06 INFO Executor: Finished task 51.0 in stage 8.0 (TID 59). 3027 bytes result sent to driver -26/04/01 05:55:06 INFO TaskSetManager: Starting task 55.0 in stage 8.0 (TID 63) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:06 INFO TaskSetManager: Finished task 51.0 in stage 8.0 (TID 59) in 1677 ms on 10.0.0.133 (executor driver) (52/208) -26/04/01 05:55:06 INFO Executor: Running task 55.0 in stage 8.0 (TID 63) -26/04/01 05:55:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:07 INFO Executor: Finished task 52.0 in stage 8.0 (TID 60). 3027 bytes result sent to driver -26/04/01 05:55:07 INFO TaskSetManager: Starting task 56.0 in stage 8.0 (TID 64) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:07 INFO TaskSetManager: Finished task 52.0 in stage 8.0 (TID 60) in 1682 ms on 10.0.0.133 (executor driver) (53/208) -26/04/01 05:55:07 INFO Executor: Running task 56.0 in stage 8.0 (TID 64) -26/04/01 05:55:07 INFO Executor: Finished task 53.0 in stage 8.0 (TID 61). 2984 bytes result sent to driver -26/04/01 05:55:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:07 INFO TaskSetManager: Starting task 57.0 in stage 8.0 (TID 65) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:07 INFO TaskSetManager: Finished task 53.0 in stage 8.0 (TID 61) in 1676 ms on 10.0.0.133 (executor driver) (54/208) -26/04/01 05:55:07 INFO Executor: Running task 57.0 in stage 8.0 (TID 65) -26/04/01 05:55:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:07 INFO Executor: Finished task 54.0 in stage 8.0 (TID 62). 2984 bytes result sent to driver -26/04/01 05:55:07 INFO TaskSetManager: Starting task 58.0 in stage 8.0 (TID 66) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:07 INFO TaskSetManager: Finished task 54.0 in stage 8.0 (TID 62) in 1681 ms on 10.0.0.133 (executor driver) (55/208) -26/04/01 05:55:07 INFO Executor: Running task 58.0 in stage 8.0 (TID 66) -26/04/01 05:55:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:07 INFO Executor: Finished task 55.0 in stage 8.0 (TID 63). 2984 bytes result sent to driver -26/04/01 05:55:07 INFO TaskSetManager: Starting task 59.0 in stage 8.0 (TID 67) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:07 INFO Executor: Running task 59.0 in stage 8.0 (TID 67) -26/04/01 05:55:07 INFO TaskSetManager: Finished task 55.0 in stage 8.0 (TID 63) in 1677 ms on 10.0.0.133 (executor driver) (56/208) -26/04/01 05:55:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:09 INFO Executor: Finished task 56.0 in stage 8.0 (TID 64). 3027 bytes result sent to driver -26/04/01 05:55:09 INFO TaskSetManager: Starting task 60.0 in stage 8.0 (TID 68) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:09 INFO TaskSetManager: Finished task 56.0 in stage 8.0 (TID 64) in 1676 ms on 10.0.0.133 (executor driver) (57/208) -26/04/01 05:55:09 INFO Executor: Running task 60.0 in stage 8.0 (TID 68) -26/04/01 05:55:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:09 INFO Executor: Finished task 57.0 in stage 8.0 (TID 65). 2984 bytes result sent to driver -26/04/01 05:55:09 INFO TaskSetManager: Starting task 61.0 in stage 8.0 (TID 69) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:09 INFO Executor: Running task 61.0 in stage 8.0 (TID 69) -26/04/01 05:55:09 INFO TaskSetManager: Finished task 57.0 in stage 8.0 (TID 65) in 1678 ms on 10.0.0.133 (executor driver) (58/208) -26/04/01 05:55:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:09 INFO Executor: Finished task 58.0 in stage 8.0 (TID 66). 2984 bytes result sent to driver -26/04/01 05:55:09 INFO TaskSetManager: Starting task 62.0 in stage 8.0 (TID 70) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:09 INFO TaskSetManager: Finished task 58.0 in stage 8.0 (TID 66) in 1675 ms on 10.0.0.133 (executor driver) (59/208) -26/04/01 05:55:09 INFO Executor: Running task 62.0 in stage 8.0 (TID 70) -26/04/01 05:55:09 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:09 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:09 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:09 INFO Executor: Finished task 59.0 in stage 8.0 (TID 67). 3027 bytes result sent to driver -26/04/01 05:55:09 INFO TaskSetManager: Starting task 63.0 in stage 8.0 (TID 71) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:09 INFO TaskSetManager: Finished task 59.0 in stage 8.0 (TID 67) in 1685 ms on 10.0.0.133 (executor driver) (60/208) -26/04/01 05:55:09 INFO Executor: Running task 63.0 in stage 8.0 (TID 71) -26/04/01 05:55:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:09 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:11 INFO Executor: Finished task 61.0 in stage 8.0 (TID 69). 2984 bytes result sent to driver -26/04/01 05:55:11 INFO TaskSetManager: Starting task 64.0 in stage 8.0 (TID 72) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:11 INFO TaskSetManager: Finished task 61.0 in stage 8.0 (TID 69) in 1670 ms on 10.0.0.133 (executor driver) (61/208) -26/04/01 05:55:11 INFO Executor: Running task 64.0 in stage 8.0 (TID 72) -26/04/01 05:55:11 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:11 INFO Executor: Finished task 60.0 in stage 8.0 (TID 68). 3027 bytes result sent to driver -26/04/01 05:55:11 INFO TaskSetManager: Starting task 65.0 in stage 8.0 (TID 73) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:11 INFO TaskSetManager: Finished task 60.0 in stage 8.0 (TID 68) in 1681 ms on 10.0.0.133 (executor driver) (62/208) -26/04/01 05:55:11 INFO Executor: Running task 65.0 in stage 8.0 (TID 73) -26/04/01 05:55:11 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:11 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:11 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:11 INFO Executor: Finished task 62.0 in stage 8.0 (TID 70). 2984 bytes result sent to driver -26/04/01 05:55:11 INFO TaskSetManager: Starting task 66.0 in stage 8.0 (TID 74) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:11 INFO TaskSetManager: Finished task 62.0 in stage 8.0 (TID 70) in 1680 ms on 10.0.0.133 (executor driver) (63/208) -26/04/01 05:55:11 INFO Executor: Running task 66.0 in stage 8.0 (TID 74) -26/04/01 05:55:11 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:11 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:11 INFO Executor: Finished task 63.0 in stage 8.0 (TID 71). 2984 bytes result sent to driver -26/04/01 05:55:11 INFO TaskSetManager: Starting task 67.0 in stage 8.0 (TID 75) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:11 INFO Executor: Running task 67.0 in stage 8.0 (TID 75) -26/04/01 05:55:11 INFO TaskSetManager: Finished task 63.0 in stage 8.0 (TID 71) in 1681 ms on 10.0.0.133 (executor driver) (64/208) -26/04/01 05:55:11 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:11 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:12 INFO Executor: Finished task 64.0 in stage 8.0 (TID 72). 3027 bytes result sent to driver -26/04/01 05:55:12 INFO Executor: Finished task 65.0 in stage 8.0 (TID 73). 2984 bytes result sent to driver -26/04/01 05:55:12 INFO TaskSetManager: Starting task 68.0 in stage 8.0 (TID 76) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:12 INFO Executor: Running task 68.0 in stage 8.0 (TID 76) -26/04/01 05:55:12 INFO TaskSetManager: Finished task 64.0 in stage 8.0 (TID 72) in 1686 ms on 10.0.0.133 (executor driver) (65/208) -26/04/01 05:55:12 INFO TaskSetManager: Starting task 69.0 in stage 8.0 (TID 77) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:12 INFO Executor: Running task 69.0 in stage 8.0 (TID 77) -26/04/01 05:55:12 INFO TaskSetManager: Finished task 65.0 in stage 8.0 (TID 73) in 1683 ms on 10.0.0.133 (executor driver) (66/208) -26/04/01 05:55:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:12 INFO Executor: Finished task 66.0 in stage 8.0 (TID 74). 2984 bytes result sent to driver -26/04/01 05:55:12 INFO TaskSetManager: Starting task 70.0 in stage 8.0 (TID 78) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:12 INFO TaskSetManager: Finished task 66.0 in stage 8.0 (TID 74) in 1679 ms on 10.0.0.133 (executor driver) (67/208) -26/04/01 05:55:12 INFO Executor: Running task 70.0 in stage 8.0 (TID 78) -26/04/01 05:55:12 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:12 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:12 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:12 INFO Executor: Finished task 67.0 in stage 8.0 (TID 75). 2984 bytes result sent to driver -26/04/01 05:55:12 INFO TaskSetManager: Starting task 71.0 in stage 8.0 (TID 79) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:12 INFO TaskSetManager: Finished task 67.0 in stage 8.0 (TID 75) in 1682 ms on 10.0.0.133 (executor driver) (68/208) -26/04/01 05:55:12 INFO Executor: Running task 71.0 in stage 8.0 (TID 79) -26/04/01 05:55:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:12 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:14 INFO Executor: Finished task 68.0 in stage 8.0 (TID 76). 3027 bytes result sent to driver -26/04/01 05:55:14 INFO Executor: Finished task 70.0 in stage 8.0 (TID 78). 2984 bytes result sent to driver -26/04/01 05:55:14 INFO TaskSetManager: Starting task 72.0 in stage 8.0 (TID 80) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:14 INFO TaskSetManager: Starting task 73.0 in stage 8.0 (TID 81) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:14 INFO Executor: Running task 72.0 in stage 8.0 (TID 80) -26/04/01 05:55:14 INFO TaskSetManager: Finished task 68.0 in stage 8.0 (TID 76) in 1685 ms on 10.0.0.133 (executor driver) (69/208) -26/04/01 05:55:14 INFO Executor: Running task 73.0 in stage 8.0 (TID 81) -26/04/01 05:55:14 INFO TaskSetManager: Finished task 70.0 in stage 8.0 (TID 78) in 1679 ms on 10.0.0.133 (executor driver) (70/208) -26/04/01 05:55:14 INFO Executor: Finished task 69.0 in stage 8.0 (TID 77). 3027 bytes result sent to driver -26/04/01 05:55:14 INFO TaskSetManager: Starting task 74.0 in stage 8.0 (TID 82) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:14 INFO TaskSetManager: Finished task 69.0 in stage 8.0 (TID 77) in 1685 ms on 10.0.0.133 (executor driver) (71/208) -26/04/01 05:55:14 INFO Executor: Running task 74.0 in stage 8.0 (TID 82) -26/04/01 05:55:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:14 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:14 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:14 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:14 INFO Executor: Finished task 71.0 in stage 8.0 (TID 79). 2984 bytes result sent to driver -26/04/01 05:55:14 INFO TaskSetManager: Starting task 75.0 in stage 8.0 (TID 83) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:14 INFO TaskSetManager: Finished task 71.0 in stage 8.0 (TID 79) in 1680 ms on 10.0.0.133 (executor driver) (72/208) -26/04/01 05:55:14 INFO Executor: Running task 75.0 in stage 8.0 (TID 83) -26/04/01 05:55:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:14 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:16 INFO Executor: Finished task 73.0 in stage 8.0 (TID 81). 3027 bytes result sent to driver -26/04/01 05:55:16 INFO TaskSetManager: Starting task 76.0 in stage 8.0 (TID 84) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:16 INFO Executor: Running task 76.0 in stage 8.0 (TID 84) -26/04/01 05:55:16 INFO TaskSetManager: Finished task 73.0 in stage 8.0 (TID 81) in 1685 ms on 10.0.0.133 (executor driver) (73/208) -26/04/01 05:55:16 INFO Executor: Finished task 72.0 in stage 8.0 (TID 80). 3027 bytes result sent to driver -26/04/01 05:55:16 INFO TaskSetManager: Starting task 77.0 in stage 8.0 (TID 85) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:16 INFO TaskSetManager: Finished task 72.0 in stage 8.0 (TID 80) in 1686 ms on 10.0.0.133 (executor driver) (74/208) -26/04/01 05:55:16 INFO Executor: Running task 77.0 in stage 8.0 (TID 85) -26/04/01 05:55:16 INFO Executor: Finished task 74.0 in stage 8.0 (TID 82). 2984 bytes result sent to driver -26/04/01 05:55:16 INFO TaskSetManager: Starting task 78.0 in stage 8.0 (TID 86) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:16 INFO TaskSetManager: Finished task 74.0 in stage 8.0 (TID 82) in 1687 ms on 10.0.0.133 (executor driver) (75/208) -26/04/01 05:55:16 INFO Executor: Running task 78.0 in stage 8.0 (TID 86) -26/04/01 05:55:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:16 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:16 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:16 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:16 INFO Executor: Finished task 75.0 in stage 8.0 (TID 83). 2984 bytes result sent to driver -26/04/01 05:55:16 INFO TaskSetManager: Starting task 79.0 in stage 8.0 (TID 87) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:16 INFO TaskSetManager: Finished task 75.0 in stage 8.0 (TID 83) in 1683 ms on 10.0.0.133 (executor driver) (76/208) -26/04/01 05:55:16 INFO Executor: Running task 79.0 in stage 8.0 (TID 87) -26/04/01 05:55:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:16 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:17 INFO Executor: Finished task 78.0 in stage 8.0 (TID 86). 2984 bytes result sent to driver -26/04/01 05:55:17 INFO Executor: Finished task 76.0 in stage 8.0 (TID 84). 3027 bytes result sent to driver -26/04/01 05:55:17 INFO TaskSetManager: Starting task 80.0 in stage 8.0 (TID 88) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:17 INFO TaskSetManager: Starting task 81.0 in stage 8.0 (TID 89) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:17 INFO Executor: Running task 80.0 in stage 8.0 (TID 88) -26/04/01 05:55:17 INFO TaskSetManager: Finished task 78.0 in stage 8.0 (TID 86) in 1674 ms on 10.0.0.133 (executor driver) (77/208) -26/04/01 05:55:17 INFO Executor: Running task 81.0 in stage 8.0 (TID 89) -26/04/01 05:55:17 INFO TaskSetManager: Finished task 76.0 in stage 8.0 (TID 84) in 1677 ms on 10.0.0.133 (executor driver) (78/208) -26/04/01 05:55:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:17 INFO Executor: Finished task 77.0 in stage 8.0 (TID 85). 2984 bytes result sent to driver -26/04/01 05:55:17 INFO TaskSetManager: Starting task 82.0 in stage 8.0 (TID 90) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:17 INFO TaskSetManager: Finished task 77.0 in stage 8.0 (TID 85) in 1683 ms on 10.0.0.133 (executor driver) (79/208) -26/04/01 05:55:17 INFO Executor: Running task 82.0 in stage 8.0 (TID 90) -26/04/01 05:55:17 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:17 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:17 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:17 INFO Executor: Finished task 79.0 in stage 8.0 (TID 87). 2984 bytes result sent to driver -26/04/01 05:55:17 INFO TaskSetManager: Starting task 83.0 in stage 8.0 (TID 91) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:17 INFO TaskSetManager: Finished task 79.0 in stage 8.0 (TID 87) in 1683 ms on 10.0.0.133 (executor driver) (80/208) -26/04/01 05:55:17 INFO Executor: Running task 83.0 in stage 8.0 (TID 91) -26/04/01 05:55:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:17 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:19 INFO Executor: Finished task 80.0 in stage 8.0 (TID 88). 3027 bytes result sent to driver -26/04/01 05:55:19 INFO TaskSetManager: Starting task 84.0 in stage 8.0 (TID 92) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:19 INFO TaskSetManager: Finished task 80.0 in stage 8.0 (TID 88) in 1680 ms on 10.0.0.133 (executor driver) (81/208) -26/04/01 05:55:19 INFO Executor: Running task 84.0 in stage 8.0 (TID 92) -26/04/01 05:55:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:19 INFO Executor: Finished task 81.0 in stage 8.0 (TID 89). 3027 bytes result sent to driver -26/04/01 05:55:19 INFO TaskSetManager: Starting task 85.0 in stage 8.0 (TID 93) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:19 INFO TaskSetManager: Finished task 81.0 in stage 8.0 (TID 89) in 1686 ms on 10.0.0.133 (executor driver) (82/208) -26/04/01 05:55:19 INFO Executor: Running task 85.0 in stage 8.0 (TID 93) -26/04/01 05:55:19 INFO Executor: Finished task 82.0 in stage 8.0 (TID 90). 2984 bytes result sent to driver -26/04/01 05:55:19 INFO TaskSetManager: Starting task 86.0 in stage 8.0 (TID 94) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:19 INFO TaskSetManager: Finished task 82.0 in stage 8.0 (TID 90) in 1682 ms on 10.0.0.133 (executor driver) (83/208) -26/04/01 05:55:19 INFO Executor: Running task 86.0 in stage 8.0 (TID 94) -26/04/01 05:55:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:19 INFO Executor: Finished task 83.0 in stage 8.0 (TID 91). 3027 bytes result sent to driver -26/04/01 05:55:19 INFO TaskSetManager: Starting task 87.0 in stage 8.0 (TID 95) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:19 INFO TaskSetManager: Finished task 83.0 in stage 8.0 (TID 91) in 1677 ms on 10.0.0.133 (executor driver) (84/208) -26/04/01 05:55:19 INFO Executor: Running task 87.0 in stage 8.0 (TID 95) -26/04/01 05:55:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:21 INFO Executor: Finished task 84.0 in stage 8.0 (TID 92). 3027 bytes result sent to driver -26/04/01 05:55:21 INFO TaskSetManager: Starting task 88.0 in stage 8.0 (TID 96) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:21 INFO TaskSetManager: Finished task 84.0 in stage 8.0 (TID 92) in 1678 ms on 10.0.0.133 (executor driver) (85/208) -26/04/01 05:55:21 INFO Executor: Running task 88.0 in stage 8.0 (TID 96) -26/04/01 05:55:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:21 INFO Executor: Finished task 86.0 in stage 8.0 (TID 94). 2984 bytes result sent to driver -26/04/01 05:55:21 INFO TaskSetManager: Starting task 89.0 in stage 8.0 (TID 97) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:21 INFO TaskSetManager: Finished task 86.0 in stage 8.0 (TID 94) in 1679 ms on 10.0.0.133 (executor driver) (86/208) -26/04/01 05:55:21 INFO Executor: Running task 89.0 in stage 8.0 (TID 97) -26/04/01 05:55:21 INFO Executor: Finished task 85.0 in stage 8.0 (TID 93). 2984 bytes result sent to driver -26/04/01 05:55:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:21 INFO TaskSetManager: Starting task 90.0 in stage 8.0 (TID 98) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:21 INFO Executor: Running task 90.0 in stage 8.0 (TID 98) -26/04/01 05:55:21 INFO TaskSetManager: Finished task 85.0 in stage 8.0 (TID 93) in 1681 ms on 10.0.0.133 (executor driver) (87/208) -26/04/01 05:55:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:21 INFO Executor: Finished task 87.0 in stage 8.0 (TID 95). 2984 bytes result sent to driver -26/04/01 05:55:21 INFO TaskSetManager: Starting task 91.0 in stage 8.0 (TID 99) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:21 INFO TaskSetManager: Finished task 87.0 in stage 8.0 (TID 95) in 1680 ms on 10.0.0.133 (executor driver) (88/208) -26/04/01 05:55:21 INFO Executor: Running task 91.0 in stage 8.0 (TID 99) -26/04/01 05:55:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:22 INFO Executor: Finished task 90.0 in stage 8.0 (TID 98). 2984 bytes result sent to driver -26/04/01 05:55:22 INFO TaskSetManager: Starting task 92.0 in stage 8.0 (TID 100) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:22 INFO Executor: Running task 92.0 in stage 8.0 (TID 100) -26/04/01 05:55:22 INFO TaskSetManager: Finished task 90.0 in stage 8.0 (TID 98) in 1673 ms on 10.0.0.133 (executor driver) (89/208) -26/04/01 05:55:22 INFO Executor: Finished task 89.0 in stage 8.0 (TID 97). 2984 bytes result sent to driver -26/04/01 05:55:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:22 INFO TaskSetManager: Starting task 93.0 in stage 8.0 (TID 101) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:22 INFO TaskSetManager: Finished task 89.0 in stage 8.0 (TID 97) in 1678 ms on 10.0.0.133 (executor driver) (90/208) -26/04/01 05:55:22 INFO Executor: Running task 93.0 in stage 8.0 (TID 101) -26/04/01 05:55:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:22 INFO Executor: Finished task 88.0 in stage 8.0 (TID 96). 3027 bytes result sent to driver -26/04/01 05:55:22 INFO TaskSetManager: Starting task 94.0 in stage 8.0 (TID 102) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:22 INFO Executor: Running task 94.0 in stage 8.0 (TID 102) -26/04/01 05:55:22 INFO TaskSetManager: Finished task 88.0 in stage 8.0 (TID 96) in 1692 ms on 10.0.0.133 (executor driver) (91/208) -26/04/01 05:55:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:22 INFO Executor: Finished task 91.0 in stage 8.0 (TID 99). 2984 bytes result sent to driver -26/04/01 05:55:22 INFO TaskSetManager: Starting task 95.0 in stage 8.0 (TID 103) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:22 INFO TaskSetManager: Finished task 91.0 in stage 8.0 (TID 99) in 1694 ms on 10.0.0.133 (executor driver) (92/208) -26/04/01 05:55:22 INFO Executor: Running task 95.0 in stage 8.0 (TID 103) -26/04/01 05:55:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:24 INFO Executor: Finished task 93.0 in stage 8.0 (TID 101). 2984 bytes result sent to driver -26/04/01 05:55:24 INFO TaskSetManager: Starting task 96.0 in stage 8.0 (TID 104) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:24 INFO Executor: Running task 96.0 in stage 8.0 (TID 104) -26/04/01 05:55:24 INFO TaskSetManager: Finished task 93.0 in stage 8.0 (TID 101) in 1676 ms on 10.0.0.133 (executor driver) (93/208) -26/04/01 05:55:24 INFO Executor: Finished task 92.0 in stage 8.0 (TID 100). 3027 bytes result sent to driver -26/04/01 05:55:24 INFO TaskSetManager: Starting task 97.0 in stage 8.0 (TID 105) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:24 INFO Executor: Finished task 94.0 in stage 8.0 (TID 102). 2984 bytes result sent to driver -26/04/01 05:55:24 INFO TaskSetManager: Finished task 92.0 in stage 8.0 (TID 100) in 1682 ms on 10.0.0.133 (executor driver) (94/208) -26/04/01 05:55:24 INFO Executor: Running task 97.0 in stage 8.0 (TID 105) -26/04/01 05:55:24 INFO TaskSetManager: Starting task 98.0 in stage 8.0 (TID 106) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:24 INFO TaskSetManager: Finished task 94.0 in stage 8.0 (TID 102) in 1673 ms on 10.0.0.133 (executor driver) (95/208) -26/04/01 05:55:24 INFO Executor: Running task 98.0 in stage 8.0 (TID 106) -26/04/01 05:55:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:24 INFO Executor: Finished task 95.0 in stage 8.0 (TID 103). 3027 bytes result sent to driver -26/04/01 05:55:24 INFO TaskSetManager: Starting task 99.0 in stage 8.0 (TID 107) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:24 INFO TaskSetManager: Finished task 95.0 in stage 8.0 (TID 103) in 1685 ms on 10.0.0.133 (executor driver) (96/208) -26/04/01 05:55:24 INFO Executor: Running task 99.0 in stage 8.0 (TID 107) -26/04/01 05:55:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:26 INFO Executor: Finished task 97.0 in stage 8.0 (TID 105). 2984 bytes result sent to driver -26/04/01 05:55:26 INFO TaskSetManager: Starting task 100.0 in stage 8.0 (TID 108) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:26 INFO TaskSetManager: Finished task 97.0 in stage 8.0 (TID 105) in 1674 ms on 10.0.0.133 (executor driver) (97/208) -26/04/01 05:55:26 INFO Executor: Running task 100.0 in stage 8.0 (TID 108) -26/04/01 05:55:26 INFO Executor: Finished task 98.0 in stage 8.0 (TID 106). 2984 bytes result sent to driver -26/04/01 05:55:26 INFO TaskSetManager: Starting task 101.0 in stage 8.0 (TID 109) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:26 INFO Executor: Running task 101.0 in stage 8.0 (TID 109) -26/04/01 05:55:26 INFO TaskSetManager: Finished task 98.0 in stage 8.0 (TID 106) in 1676 ms on 10.0.0.133 (executor driver) (98/208) -26/04/01 05:55:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:26 INFO Executor: Finished task 96.0 in stage 8.0 (TID 104). 3027 bytes result sent to driver -26/04/01 05:55:26 INFO TaskSetManager: Starting task 102.0 in stage 8.0 (TID 110) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:26 INFO TaskSetManager: Finished task 96.0 in stage 8.0 (TID 104) in 1681 ms on 10.0.0.133 (executor driver) (99/208) -26/04/01 05:55:26 INFO Executor: Running task 102.0 in stage 8.0 (TID 110) -26/04/01 05:55:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:26 INFO Executor: Finished task 99.0 in stage 8.0 (TID 107). 2984 bytes result sent to driver -26/04/01 05:55:26 INFO TaskSetManager: Starting task 103.0 in stage 8.0 (TID 111) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:26 INFO TaskSetManager: Finished task 99.0 in stage 8.0 (TID 107) in 1680 ms on 10.0.0.133 (executor driver) (100/208) -26/04/01 05:55:26 INFO Executor: Running task 103.0 in stage 8.0 (TID 111) -26/04/01 05:55:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:27 INFO Executor: Finished task 100.0 in stage 8.0 (TID 108). 3027 bytes result sent to driver -26/04/01 05:55:27 INFO Executor: Finished task 101.0 in stage 8.0 (TID 109). 2984 bytes result sent to driver -26/04/01 05:55:27 INFO TaskSetManager: Starting task 104.0 in stage 8.0 (TID 112) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:27 INFO Executor: Running task 104.0 in stage 8.0 (TID 112) -26/04/01 05:55:27 INFO TaskSetManager: Starting task 105.0 in stage 8.0 (TID 113) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:27 INFO Executor: Running task 105.0 in stage 8.0 (TID 113) -26/04/01 05:55:27 INFO TaskSetManager: Finished task 100.0 in stage 8.0 (TID 108) in 1677 ms on 10.0.0.133 (executor driver) (101/208) -26/04/01 05:55:27 INFO TaskSetManager: Finished task 101.0 in stage 8.0 (TID 109) in 1674 ms on 10.0.0.133 (executor driver) (102/208) -26/04/01 05:55:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:27 INFO Executor: Finished task 102.0 in stage 8.0 (TID 110). 2984 bytes result sent to driver -26/04/01 05:55:27 INFO TaskSetManager: Starting task 106.0 in stage 8.0 (TID 114) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:27 INFO TaskSetManager: Finished task 102.0 in stage 8.0 (TID 110) in 1678 ms on 10.0.0.133 (executor driver) (103/208) -26/04/01 05:55:27 INFO Executor: Running task 106.0 in stage 8.0 (TID 114) -26/04/01 05:55:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:28 INFO Executor: Finished task 103.0 in stage 8.0 (TID 111). 3027 bytes result sent to driver -26/04/01 05:55:28 INFO TaskSetManager: Starting task 107.0 in stage 8.0 (TID 115) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:28 INFO TaskSetManager: Finished task 103.0 in stage 8.0 (TID 111) in 1675 ms on 10.0.0.133 (executor driver) (104/208) -26/04/01 05:55:28 INFO Executor: Running task 107.0 in stage 8.0 (TID 115) -26/04/01 05:55:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:28 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:29 INFO Executor: Finished task 105.0 in stage 8.0 (TID 113). 3027 bytes result sent to driver -26/04/01 05:55:29 INFO TaskSetManager: Starting task 108.0 in stage 8.0 (TID 116) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:29 INFO Executor: Running task 108.0 in stage 8.0 (TID 116) -26/04/01 05:55:29 INFO TaskSetManager: Finished task 105.0 in stage 8.0 (TID 113) in 1675 ms on 10.0.0.133 (executor driver) (105/208) -26/04/01 05:55:29 INFO Executor: Finished task 104.0 in stage 8.0 (TID 112). 3027 bytes result sent to driver -26/04/01 05:55:29 INFO TaskSetManager: Starting task 109.0 in stage 8.0 (TID 117) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:29 INFO TaskSetManager: Finished task 104.0 in stage 8.0 (TID 112) in 1676 ms on 10.0.0.133 (executor driver) (106/208) -26/04/01 05:55:29 INFO Executor: Running task 109.0 in stage 8.0 (TID 117) -26/04/01 05:55:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:29 INFO Executor: Finished task 106.0 in stage 8.0 (TID 114). 2984 bytes result sent to driver -26/04/01 05:55:29 INFO TaskSetManager: Starting task 110.0 in stage 8.0 (TID 118) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:29 INFO TaskSetManager: Finished task 106.0 in stage 8.0 (TID 114) in 1674 ms on 10.0.0.133 (executor driver) (107/208) -26/04/01 05:55:29 INFO Executor: Running task 110.0 in stage 8.0 (TID 118) -26/04/01 05:55:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:29 INFO Executor: Finished task 107.0 in stage 8.0 (TID 115). 3027 bytes result sent to driver -26/04/01 05:55:29 INFO TaskSetManager: Starting task 111.0 in stage 8.0 (TID 119) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:29 INFO TaskSetManager: Finished task 107.0 in stage 8.0 (TID 115) in 1677 ms on 10.0.0.133 (executor driver) (108/208) -26/04/01 05:55:29 INFO Executor: Running task 111.0 in stage 8.0 (TID 119) -26/04/01 05:55:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:31 INFO Executor: Finished task 108.0 in stage 8.0 (TID 116). 3027 bytes result sent to driver -26/04/01 05:55:31 INFO Executor: Finished task 109.0 in stage 8.0 (TID 117). 2984 bytes result sent to driver -26/04/01 05:55:31 INFO TaskSetManager: Starting task 112.0 in stage 8.0 (TID 120) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:31 INFO TaskSetManager: Starting task 113.0 in stage 8.0 (TID 121) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:31 INFO Executor: Running task 112.0 in stage 8.0 (TID 120) -26/04/01 05:55:31 INFO TaskSetManager: Finished task 108.0 in stage 8.0 (TID 116) in 1674 ms on 10.0.0.133 (executor driver) (109/208) -26/04/01 05:55:31 INFO Executor: Running task 113.0 in stage 8.0 (TID 121) -26/04/01 05:55:31 INFO TaskSetManager: Finished task 109.0 in stage 8.0 (TID 117) in 1672 ms on 10.0.0.133 (executor driver) (110/208) -26/04/01 05:55:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:31 INFO Executor: Finished task 110.0 in stage 8.0 (TID 118). 2984 bytes result sent to driver -26/04/01 05:55:31 INFO TaskSetManager: Starting task 114.0 in stage 8.0 (TID 122) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:31 INFO Executor: Running task 114.0 in stage 8.0 (TID 122) -26/04/01 05:55:31 INFO TaskSetManager: Finished task 110.0 in stage 8.0 (TID 118) in 1675 ms on 10.0.0.133 (executor driver) (111/208) -26/04/01 05:55:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:31 INFO Executor: Finished task 111.0 in stage 8.0 (TID 119). 3027 bytes result sent to driver -26/04/01 05:55:31 INFO TaskSetManager: Starting task 115.0 in stage 8.0 (TID 123) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:31 INFO TaskSetManager: Finished task 111.0 in stage 8.0 (TID 119) in 1667 ms on 10.0.0.133 (executor driver) (112/208) -26/04/01 05:55:31 INFO Executor: Running task 115.0 in stage 8.0 (TID 123) -26/04/01 05:55:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:32 INFO Executor: Finished task 113.0 in stage 8.0 (TID 121). 3027 bytes result sent to driver -26/04/01 05:55:32 INFO TaskSetManager: Starting task 116.0 in stage 8.0 (TID 124) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:32 INFO TaskSetManager: Finished task 113.0 in stage 8.0 (TID 121) in 1675 ms on 10.0.0.133 (executor driver) (113/208) -26/04/01 05:55:32 INFO Executor: Running task 116.0 in stage 8.0 (TID 124) -26/04/01 05:55:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:32 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:33 INFO Executor: Finished task 112.0 in stage 8.0 (TID 120). 3027 bytes result sent to driver -26/04/01 05:55:33 INFO TaskSetManager: Starting task 117.0 in stage 8.0 (TID 125) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:33 INFO TaskSetManager: Finished task 112.0 in stage 8.0 (TID 120) in 1702 ms on 10.0.0.133 (executor driver) (114/208) -26/04/01 05:55:33 INFO Executor: Running task 117.0 in stage 8.0 (TID 125) -26/04/01 05:55:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:33 INFO Executor: Finished task 115.0 in stage 8.0 (TID 123). 3027 bytes result sent to driver -26/04/01 05:55:33 INFO TaskSetManager: Starting task 118.0 in stage 8.0 (TID 126) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:33 INFO TaskSetManager: Finished task 115.0 in stage 8.0 (TID 123) in 1678 ms on 10.0.0.133 (executor driver) (115/208) -26/04/01 05:55:33 INFO Executor: Running task 118.0 in stage 8.0 (TID 126) -26/04/01 05:55:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:33 INFO Executor: Finished task 114.0 in stage 8.0 (TID 122). 2984 bytes result sent to driver -26/04/01 05:55:33 INFO TaskSetManager: Starting task 119.0 in stage 8.0 (TID 127) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:33 INFO TaskSetManager: Finished task 114.0 in stage 8.0 (TID 122) in 1710 ms on 10.0.0.133 (executor driver) (116/208) -26/04/01 05:55:33 INFO Executor: Running task 119.0 in stage 8.0 (TID 127) -26/04/01 05:55:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:34 INFO Executor: Finished task 116.0 in stage 8.0 (TID 124). 3027 bytes result sent to driver -26/04/01 05:55:34 INFO TaskSetManager: Starting task 120.0 in stage 8.0 (TID 128) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:34 INFO Executor: Running task 120.0 in stage 8.0 (TID 128) -26/04/01 05:55:34 INFO TaskSetManager: Finished task 116.0 in stage 8.0 (TID 124) in 1674 ms on 10.0.0.133 (executor driver) (117/208) -26/04/01 05:55:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:34 INFO Executor: Finished task 117.0 in stage 8.0 (TID 125). 2984 bytes result sent to driver -26/04/01 05:55:34 INFO TaskSetManager: Starting task 121.0 in stage 8.0 (TID 129) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:34 INFO TaskSetManager: Finished task 117.0 in stage 8.0 (TID 125) in 1672 ms on 10.0.0.133 (executor driver) (118/208) -26/04/01 05:55:34 INFO Executor: Running task 121.0 in stage 8.0 (TID 129) -26/04/01 05:55:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:34 INFO Executor: Finished task 118.0 in stage 8.0 (TID 126). 2984 bytes result sent to driver -26/04/01 05:55:34 INFO TaskSetManager: Starting task 122.0 in stage 8.0 (TID 130) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:34 INFO Executor: Running task 122.0 in stage 8.0 (TID 130) -26/04/01 05:55:34 INFO TaskSetManager: Finished task 118.0 in stage 8.0 (TID 126) in 1676 ms on 10.0.0.133 (executor driver) (119/208) -26/04/01 05:55:34 INFO Executor: Finished task 119.0 in stage 8.0 (TID 127). 3027 bytes result sent to driver -26/04/01 05:55:34 INFO TaskSetManager: Starting task 123.0 in stage 8.0 (TID 131) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:34 INFO Executor: Running task 123.0 in stage 8.0 (TID 131) -26/04/01 05:55:34 INFO TaskSetManager: Finished task 119.0 in stage 8.0 (TID 127) in 1674 ms on 10.0.0.133 (executor driver) (120/208) -26/04/01 05:55:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:36 INFO Executor: Finished task 120.0 in stage 8.0 (TID 128). 3027 bytes result sent to driver -26/04/01 05:55:36 INFO TaskSetManager: Starting task 124.0 in stage 8.0 (TID 132) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:36 INFO Executor: Running task 124.0 in stage 8.0 (TID 132) -26/04/01 05:55:36 INFO TaskSetManager: Finished task 120.0 in stage 8.0 (TID 128) in 1675 ms on 10.0.0.133 (executor driver) (121/208) -26/04/01 05:55:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:36 INFO Executor: Finished task 121.0 in stage 8.0 (TID 129). 3027 bytes result sent to driver -26/04/01 05:55:36 INFO TaskSetManager: Starting task 125.0 in stage 8.0 (TID 133) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:36 INFO TaskSetManager: Finished task 121.0 in stage 8.0 (TID 129) in 1670 ms on 10.0.0.133 (executor driver) (122/208) -26/04/01 05:55:36 INFO Executor: Running task 125.0 in stage 8.0 (TID 133) -26/04/01 05:55:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:36 INFO Executor: Finished task 123.0 in stage 8.0 (TID 131). 2984 bytes result sent to driver -26/04/01 05:55:36 INFO TaskSetManager: Starting task 126.0 in stage 8.0 (TID 134) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:36 INFO TaskSetManager: Finished task 123.0 in stage 8.0 (TID 131) in 1669 ms on 10.0.0.133 (executor driver) (123/208) -26/04/01 05:55:36 INFO Executor: Running task 126.0 in stage 8.0 (TID 134) -26/04/01 05:55:36 INFO Executor: Finished task 122.0 in stage 8.0 (TID 130). 2984 bytes result sent to driver -26/04/01 05:55:36 INFO TaskSetManager: Starting task 127.0 in stage 8.0 (TID 135) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:36 INFO Executor: Running task 127.0 in stage 8.0 (TID 135) -26/04/01 05:55:36 INFO TaskSetManager: Finished task 122.0 in stage 8.0 (TID 130) in 1673 ms on 10.0.0.133 (executor driver) (124/208) -26/04/01 05:55:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:38 INFO Executor: Finished task 124.0 in stage 8.0 (TID 132). 3027 bytes result sent to driver -26/04/01 05:55:38 INFO TaskSetManager: Starting task 128.0 in stage 8.0 (TID 136) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:38 INFO Executor: Running task 128.0 in stage 8.0 (TID 136) -26/04/01 05:55:38 INFO TaskSetManager: Finished task 124.0 in stage 8.0 (TID 132) in 1671 ms on 10.0.0.133 (executor driver) (125/208) -26/04/01 05:55:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:38 INFO Executor: Finished task 125.0 in stage 8.0 (TID 133). 2984 bytes result sent to driver -26/04/01 05:55:38 INFO TaskSetManager: Starting task 129.0 in stage 8.0 (TID 137) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:38 INFO TaskSetManager: Finished task 125.0 in stage 8.0 (TID 133) in 1665 ms on 10.0.0.133 (executor driver) (126/208) -26/04/01 05:55:38 INFO Executor: Running task 129.0 in stage 8.0 (TID 137) -26/04/01 05:55:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:38 INFO Executor: Finished task 126.0 in stage 8.0 (TID 134). 2984 bytes result sent to driver -26/04/01 05:55:38 INFO TaskSetManager: Starting task 130.0 in stage 8.0 (TID 138) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:38 INFO TaskSetManager: Finished task 126.0 in stage 8.0 (TID 134) in 1673 ms on 10.0.0.133 (executor driver) (127/208) -26/04/01 05:55:38 INFO Executor: Running task 130.0 in stage 8.0 (TID 138) -26/04/01 05:55:38 INFO Executor: Finished task 127.0 in stage 8.0 (TID 135). 3027 bytes result sent to driver -26/04/01 05:55:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:38 INFO TaskSetManager: Starting task 131.0 in stage 8.0 (TID 139) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:38 INFO Executor: Running task 131.0 in stage 8.0 (TID 139) -26/04/01 05:55:38 INFO TaskSetManager: Finished task 127.0 in stage 8.0 (TID 135) in 1673 ms on 10.0.0.133 (executor driver) (128/208) -26/04/01 05:55:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:39 INFO Executor: Finished task 128.0 in stage 8.0 (TID 136). 3027 bytes result sent to driver -26/04/01 05:55:39 INFO TaskSetManager: Starting task 132.0 in stage 8.0 (TID 140) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:39 INFO TaskSetManager: Finished task 128.0 in stage 8.0 (TID 136) in 1680 ms on 10.0.0.133 (executor driver) (129/208) -26/04/01 05:55:39 INFO Executor: Running task 132.0 in stage 8.0 (TID 140) -26/04/01 05:55:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:39 INFO Executor: Finished task 129.0 in stage 8.0 (TID 137). 2984 bytes result sent to driver -26/04/01 05:55:39 INFO TaskSetManager: Starting task 133.0 in stage 8.0 (TID 141) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:39 INFO TaskSetManager: Finished task 129.0 in stage 8.0 (TID 137) in 1682 ms on 10.0.0.133 (executor driver) (130/208) -26/04/01 05:55:39 INFO Executor: Running task 133.0 in stage 8.0 (TID 141) -26/04/01 05:55:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:39 INFO Executor: Finished task 130.0 in stage 8.0 (TID 138). 3027 bytes result sent to driver -26/04/01 05:55:39 INFO TaskSetManager: Starting task 134.0 in stage 8.0 (TID 142) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:39 INFO Executor: Finished task 131.0 in stage 8.0 (TID 139). 2984 bytes result sent to driver -26/04/01 05:55:39 INFO Executor: Running task 134.0 in stage 8.0 (TID 142) -26/04/01 05:55:39 INFO TaskSetManager: Starting task 135.0 in stage 8.0 (TID 143) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:39 INFO TaskSetManager: Finished task 130.0 in stage 8.0 (TID 138) in 1677 ms on 10.0.0.133 (executor driver) (131/208) -26/04/01 05:55:39 INFO Executor: Running task 135.0 in stage 8.0 (TID 143) -26/04/01 05:55:39 INFO TaskSetManager: Finished task 131.0 in stage 8.0 (TID 139) in 1673 ms on 10.0.0.133 (executor driver) (132/208) -26/04/01 05:55:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:41 INFO Executor: Finished task 133.0 in stage 8.0 (TID 141). 2984 bytes result sent to driver -26/04/01 05:55:41 INFO TaskSetManager: Starting task 136.0 in stage 8.0 (TID 144) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:41 INFO TaskSetManager: Finished task 133.0 in stage 8.0 (TID 141) in 1676 ms on 10.0.0.133 (executor driver) (133/208) -26/04/01 05:55:41 INFO Executor: Running task 136.0 in stage 8.0 (TID 144) -26/04/01 05:55:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:41 INFO Executor: Finished task 134.0 in stage 8.0 (TID 142). 2984 bytes result sent to driver -26/04/01 05:55:41 INFO TaskSetManager: Starting task 137.0 in stage 8.0 (TID 145) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:41 INFO TaskSetManager: Finished task 134.0 in stage 8.0 (TID 142) in 1673 ms on 10.0.0.133 (executor driver) (134/208) -26/04/01 05:55:41 INFO Executor: Running task 137.0 in stage 8.0 (TID 145) -26/04/01 05:55:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:41 INFO Executor: Finished task 132.0 in stage 8.0 (TID 140). 3027 bytes result sent to driver -26/04/01 05:55:41 INFO TaskSetManager: Starting task 138.0 in stage 8.0 (TID 146) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:41 INFO TaskSetManager: Finished task 132.0 in stage 8.0 (TID 140) in 1739 ms on 10.0.0.133 (executor driver) (135/208) -26/04/01 05:55:41 INFO Executor: Running task 138.0 in stage 8.0 (TID 146) -26/04/01 05:55:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:41 INFO Executor: Finished task 135.0 in stage 8.0 (TID 143). 2984 bytes result sent to driver -26/04/01 05:55:41 INFO TaskSetManager: Starting task 139.0 in stage 8.0 (TID 147) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:41 INFO TaskSetManager: Finished task 135.0 in stage 8.0 (TID 143) in 1734 ms on 10.0.0.133 (executor driver) (136/208) -26/04/01 05:55:41 INFO Executor: Running task 139.0 in stage 8.0 (TID 147) -26/04/01 05:55:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:43 INFO Executor: Finished task 136.0 in stage 8.0 (TID 144). 3027 bytes result sent to driver -26/04/01 05:55:43 INFO TaskSetManager: Starting task 140.0 in stage 8.0 (TID 148) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:43 INFO TaskSetManager: Finished task 136.0 in stage 8.0 (TID 144) in 1679 ms on 10.0.0.133 (executor driver) (137/208) -26/04/01 05:55:43 INFO Executor: Running task 140.0 in stage 8.0 (TID 148) -26/04/01 05:55:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:43 INFO Executor: Finished task 137.0 in stage 8.0 (TID 145). 3027 bytes result sent to driver -26/04/01 05:55:43 INFO TaskSetManager: Starting task 141.0 in stage 8.0 (TID 149) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:43 INFO TaskSetManager: Finished task 137.0 in stage 8.0 (TID 145) in 1675 ms on 10.0.0.133 (executor driver) (138/208) -26/04/01 05:55:43 INFO Executor: Running task 141.0 in stage 8.0 (TID 149) -26/04/01 05:55:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:43 INFO Executor: Finished task 138.0 in stage 8.0 (TID 146). 3027 bytes result sent to driver -26/04/01 05:55:43 INFO TaskSetManager: Starting task 142.0 in stage 8.0 (TID 150) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:43 INFO TaskSetManager: Finished task 138.0 in stage 8.0 (TID 146) in 1673 ms on 10.0.0.133 (executor driver) (139/208) -26/04/01 05:55:43 INFO Executor: Running task 142.0 in stage 8.0 (TID 150) -26/04/01 05:55:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:43 INFO Executor: Finished task 139.0 in stage 8.0 (TID 147). 2984 bytes result sent to driver -26/04/01 05:55:43 INFO TaskSetManager: Starting task 143.0 in stage 8.0 (TID 151) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:43 INFO TaskSetManager: Finished task 139.0 in stage 8.0 (TID 147) in 1675 ms on 10.0.0.133 (executor driver) (140/208) -26/04/01 05:55:43 INFO Executor: Running task 143.0 in stage 8.0 (TID 151) -26/04/01 05:55:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:44 INFO Executor: Finished task 141.0 in stage 8.0 (TID 149). 2984 bytes result sent to driver -26/04/01 05:55:44 INFO TaskSetManager: Starting task 144.0 in stage 8.0 (TID 152) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:44 INFO TaskSetManager: Finished task 141.0 in stage 8.0 (TID 149) in 1674 ms on 10.0.0.133 (executor driver) (141/208) -26/04/01 05:55:44 INFO Executor: Running task 144.0 in stage 8.0 (TID 152) -26/04/01 05:55:44 INFO Executor: Finished task 140.0 in stage 8.0 (TID 148). 3027 bytes result sent to driver -26/04/01 05:55:44 INFO TaskSetManager: Starting task 145.0 in stage 8.0 (TID 153) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:44 INFO TaskSetManager: Finished task 140.0 in stage 8.0 (TID 148) in 1683 ms on 10.0.0.133 (executor driver) (142/208) -26/04/01 05:55:44 INFO Executor: Running task 145.0 in stage 8.0 (TID 153) -26/04/01 05:55:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:44 INFO Executor: Finished task 142.0 in stage 8.0 (TID 150). 2984 bytes result sent to driver -26/04/01 05:55:44 INFO TaskSetManager: Starting task 146.0 in stage 8.0 (TID 154) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:44 INFO Executor: Running task 146.0 in stage 8.0 (TID 154) -26/04/01 05:55:44 INFO TaskSetManager: Finished task 142.0 in stage 8.0 (TID 150) in 1679 ms on 10.0.0.133 (executor driver) (143/208) -26/04/01 05:55:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:44 INFO Executor: Finished task 143.0 in stage 8.0 (TID 151). 2984 bytes result sent to driver -26/04/01 05:55:44 INFO TaskSetManager: Starting task 147.0 in stage 8.0 (TID 155) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:44 INFO TaskSetManager: Finished task 143.0 in stage 8.0 (TID 151) in 1672 ms on 10.0.0.133 (executor driver) (144/208) -26/04/01 05:55:44 INFO Executor: Running task 147.0 in stage 8.0 (TID 155) -26/04/01 05:55:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:46 INFO Executor: Finished task 144.0 in stage 8.0 (TID 152). 3027 bytes result sent to driver -26/04/01 05:55:46 INFO TaskSetManager: Starting task 148.0 in stage 8.0 (TID 156) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:46 INFO TaskSetManager: Finished task 144.0 in stage 8.0 (TID 152) in 1710 ms on 10.0.0.133 (executor driver) (145/208) -26/04/01 05:55:46 INFO Executor: Running task 148.0 in stage 8.0 (TID 156) -26/04/01 05:55:46 INFO Executor: Finished task 145.0 in stage 8.0 (TID 153). 2984 bytes result sent to driver -26/04/01 05:55:46 INFO TaskSetManager: Starting task 149.0 in stage 8.0 (TID 157) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:46 INFO TaskSetManager: Finished task 145.0 in stage 8.0 (TID 153) in 1712 ms on 10.0.0.133 (executor driver) (146/208) -26/04/01 05:55:46 INFO Executor: Running task 149.0 in stage 8.0 (TID 157) -26/04/01 05:55:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:46 INFO Executor: Finished task 146.0 in stage 8.0 (TID 154). 2984 bytes result sent to driver -26/04/01 05:55:46 INFO TaskSetManager: Starting task 150.0 in stage 8.0 (TID 158) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:46 INFO TaskSetManager: Finished task 146.0 in stage 8.0 (TID 154) in 1682 ms on 10.0.0.133 (executor driver) (147/208) -26/04/01 05:55:46 INFO Executor: Running task 150.0 in stage 8.0 (TID 158) -26/04/01 05:55:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:46 INFO Executor: Finished task 147.0 in stage 8.0 (TID 155). 2984 bytes result sent to driver -26/04/01 05:55:46 INFO TaskSetManager: Starting task 151.0 in stage 8.0 (TID 159) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:46 INFO TaskSetManager: Finished task 147.0 in stage 8.0 (TID 155) in 1679 ms on 10.0.0.133 (executor driver) (148/208) -26/04/01 05:55:46 INFO Executor: Running task 151.0 in stage 8.0 (TID 159) -26/04/01 05:55:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:48 INFO Executor: Finished task 148.0 in stage 8.0 (TID 156). 3027 bytes result sent to driver -26/04/01 05:55:48 INFO TaskSetManager: Starting task 152.0 in stage 8.0 (TID 160) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:48 INFO TaskSetManager: Finished task 148.0 in stage 8.0 (TID 156) in 1675 ms on 10.0.0.133 (executor driver) (149/208) -26/04/01 05:55:48 INFO Executor: Running task 152.0 in stage 8.0 (TID 160) -26/04/01 05:55:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:48 INFO Executor: Finished task 149.0 in stage 8.0 (TID 157). 2984 bytes result sent to driver -26/04/01 05:55:48 INFO TaskSetManager: Starting task 153.0 in stage 8.0 (TID 161) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:48 INFO TaskSetManager: Finished task 149.0 in stage 8.0 (TID 157) in 1679 ms on 10.0.0.133 (executor driver) (150/208) -26/04/01 05:55:48 INFO Executor: Running task 153.0 in stage 8.0 (TID 161) -26/04/01 05:55:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:48 INFO Executor: Finished task 150.0 in stage 8.0 (TID 158). 2984 bytes result sent to driver -26/04/01 05:55:48 INFO TaskSetManager: Starting task 154.0 in stage 8.0 (TID 162) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:48 INFO TaskSetManager: Finished task 150.0 in stage 8.0 (TID 158) in 1673 ms on 10.0.0.133 (executor driver) (151/208) -26/04/01 05:55:48 INFO Executor: Running task 154.0 in stage 8.0 (TID 162) -26/04/01 05:55:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:48 INFO Executor: Finished task 151.0 in stage 8.0 (TID 159). 2984 bytes result sent to driver -26/04/01 05:55:48 INFO TaskSetManager: Starting task 155.0 in stage 8.0 (TID 163) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:48 INFO TaskSetManager: Finished task 151.0 in stage 8.0 (TID 159) in 1675 ms on 10.0.0.133 (executor driver) (152/208) -26/04/01 05:55:48 INFO Executor: Running task 155.0 in stage 8.0 (TID 163) -26/04/01 05:55:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:49 INFO Executor: Finished task 154.0 in stage 8.0 (TID 162). 2984 bytes result sent to driver -26/04/01 05:55:49 INFO TaskSetManager: Starting task 156.0 in stage 8.0 (TID 164) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:49 INFO TaskSetManager: Finished task 154.0 in stage 8.0 (TID 162) in 1667 ms on 10.0.0.133 (executor driver) (153/208) -26/04/01 05:55:49 INFO Executor: Running task 156.0 in stage 8.0 (TID 164) -26/04/01 05:55:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:49 INFO Executor: Finished task 152.0 in stage 8.0 (TID 160). 3027 bytes result sent to driver -26/04/01 05:55:49 INFO TaskSetManager: Starting task 157.0 in stage 8.0 (TID 165) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:49 INFO TaskSetManager: Finished task 152.0 in stage 8.0 (TID 160) in 1683 ms on 10.0.0.133 (executor driver) (154/208) -26/04/01 05:55:49 INFO Executor: Running task 157.0 in stage 8.0 (TID 165) -26/04/01 05:55:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:49 INFO Executor: Finished task 153.0 in stage 8.0 (TID 161). 2984 bytes result sent to driver -26/04/01 05:55:49 INFO TaskSetManager: Starting task 158.0 in stage 8.0 (TID 166) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:49 INFO TaskSetManager: Finished task 153.0 in stage 8.0 (TID 161) in 1678 ms on 10.0.0.133 (executor driver) (155/208) -26/04/01 05:55:49 INFO Executor: Running task 158.0 in stage 8.0 (TID 166) -26/04/01 05:55:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:49 INFO Executor: Finished task 155.0 in stage 8.0 (TID 163). 2984 bytes result sent to driver -26/04/01 05:55:49 INFO TaskSetManager: Starting task 159.0 in stage 8.0 (TID 167) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:49 INFO Executor: Running task 159.0 in stage 8.0 (TID 167) -26/04/01 05:55:49 INFO TaskSetManager: Finished task 155.0 in stage 8.0 (TID 163) in 1684 ms on 10.0.0.133 (executor driver) (156/208) -26/04/01 05:55:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:51 INFO Executor: Finished task 157.0 in stage 8.0 (TID 165). 2984 bytes result sent to driver -26/04/01 05:55:51 INFO TaskSetManager: Starting task 160.0 in stage 8.0 (TID 168) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:51 INFO TaskSetManager: Finished task 157.0 in stage 8.0 (TID 165) in 1672 ms on 10.0.0.133 (executor driver) (157/208) -26/04/01 05:55:51 INFO Executor: Running task 160.0 in stage 8.0 (TID 168) -26/04/01 05:55:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:51 INFO Executor: Finished task 158.0 in stage 8.0 (TID 166). 2984 bytes result sent to driver -26/04/01 05:55:51 INFO TaskSetManager: Starting task 161.0 in stage 8.0 (TID 169) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:51 INFO TaskSetManager: Finished task 158.0 in stage 8.0 (TID 166) in 1675 ms on 10.0.0.133 (executor driver) (158/208) -26/04/01 05:55:51 INFO Executor: Running task 161.0 in stage 8.0 (TID 169) -26/04/01 05:55:51 INFO Executor: Finished task 156.0 in stage 8.0 (TID 164). 3027 bytes result sent to driver -26/04/01 05:55:51 INFO TaskSetManager: Starting task 162.0 in stage 8.0 (TID 170) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:51 INFO TaskSetManager: Finished task 156.0 in stage 8.0 (TID 164) in 1683 ms on 10.0.0.133 (executor driver) (159/208) -26/04/01 05:55:51 INFO Executor: Running task 162.0 in stage 8.0 (TID 170) -26/04/01 05:55:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:51 INFO Executor: Finished task 159.0 in stage 8.0 (TID 167). 3027 bytes result sent to driver -26/04/01 05:55:51 INFO TaskSetManager: Starting task 163.0 in stage 8.0 (TID 171) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:51 INFO TaskSetManager: Finished task 159.0 in stage 8.0 (TID 167) in 1673 ms on 10.0.0.133 (executor driver) (160/208) -26/04/01 05:55:51 INFO Executor: Running task 163.0 in stage 8.0 (TID 171) -26/04/01 05:55:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:53 INFO Executor: Finished task 160.0 in stage 8.0 (TID 168). 3027 bytes result sent to driver -26/04/01 05:55:53 INFO Executor: Finished task 162.0 in stage 8.0 (TID 170). 2984 bytes result sent to driver -26/04/01 05:55:53 INFO TaskSetManager: Starting task 164.0 in stage 8.0 (TID 172) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:53 INFO Executor: Running task 164.0 in stage 8.0 (TID 172) -26/04/01 05:55:53 INFO TaskSetManager: Starting task 165.0 in stage 8.0 (TID 173) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:53 INFO Executor: Finished task 161.0 in stage 8.0 (TID 169). 2984 bytes result sent to driver -26/04/01 05:55:53 INFO Executor: Running task 165.0 in stage 8.0 (TID 173) -26/04/01 05:55:53 INFO TaskSetManager: Starting task 166.0 in stage 8.0 (TID 174) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:53 INFO TaskSetManager: Finished task 160.0 in stage 8.0 (TID 168) in 1676 ms on 10.0.0.133 (executor driver) (161/208) -26/04/01 05:55:53 INFO Executor: Running task 166.0 in stage 8.0 (TID 174) -26/04/01 05:55:53 INFO TaskSetManager: Finished task 162.0 in stage 8.0 (TID 170) in 1672 ms on 10.0.0.133 (executor driver) (162/208) -26/04/01 05:55:53 INFO TaskSetManager: Finished task 161.0 in stage 8.0 (TID 169) in 1673 ms on 10.0.0.133 (executor driver) (163/208) -26/04/01 05:55:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:53 INFO Executor: Finished task 163.0 in stage 8.0 (TID 171). 2984 bytes result sent to driver -26/04/01 05:55:53 INFO TaskSetManager: Starting task 167.0 in stage 8.0 (TID 175) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:53 INFO TaskSetManager: Finished task 163.0 in stage 8.0 (TID 171) in 1676 ms on 10.0.0.133 (executor driver) (164/208) -26/04/01 05:55:53 INFO Executor: Running task 167.0 in stage 8.0 (TID 175) -26/04/01 05:55:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:54 INFO Executor: Finished task 165.0 in stage 8.0 (TID 173). 3027 bytes result sent to driver -26/04/01 05:55:54 INFO TaskSetManager: Starting task 168.0 in stage 8.0 (TID 176) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:54 INFO Executor: Running task 168.0 in stage 8.0 (TID 176) -26/04/01 05:55:54 INFO TaskSetManager: Finished task 165.0 in stage 8.0 (TID 173) in 1673 ms on 10.0.0.133 (executor driver) (165/208) -26/04/01 05:55:54 INFO Executor: Finished task 164.0 in stage 8.0 (TID 172). 3027 bytes result sent to driver -26/04/01 05:55:54 INFO TaskSetManager: Starting task 169.0 in stage 8.0 (TID 177) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:54 INFO TaskSetManager: Finished task 164.0 in stage 8.0 (TID 172) in 1674 ms on 10.0.0.133 (executor driver) (166/208) -26/04/01 05:55:54 INFO Executor: Running task 169.0 in stage 8.0 (TID 177) -26/04/01 05:55:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:54 INFO Executor: Finished task 166.0 in stage 8.0 (TID 174). 2984 bytes result sent to driver -26/04/01 05:55:54 INFO TaskSetManager: Starting task 170.0 in stage 8.0 (TID 178) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:54 INFO TaskSetManager: Finished task 166.0 in stage 8.0 (TID 174) in 1679 ms on 10.0.0.133 (executor driver) (167/208) -26/04/01 05:55:54 INFO Executor: Running task 170.0 in stage 8.0 (TID 178) -26/04/01 05:55:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:54 INFO Executor: Finished task 167.0 in stage 8.0 (TID 175). 2984 bytes result sent to driver -26/04/01 05:55:54 INFO TaskSetManager: Starting task 171.0 in stage 8.0 (TID 179) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:54 INFO TaskSetManager: Finished task 167.0 in stage 8.0 (TID 175) in 1676 ms on 10.0.0.133 (executor driver) (168/208) -26/04/01 05:55:54 INFO Executor: Running task 171.0 in stage 8.0 (TID 179) -26/04/01 05:55:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:56 INFO Executor: Finished task 168.0 in stage 8.0 (TID 176). 3027 bytes result sent to driver -26/04/01 05:55:56 INFO TaskSetManager: Starting task 172.0 in stage 8.0 (TID 180) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:56 INFO Executor: Running task 172.0 in stage 8.0 (TID 180) -26/04/01 05:55:56 INFO TaskSetManager: Finished task 168.0 in stage 8.0 (TID 176) in 1678 ms on 10.0.0.133 (executor driver) (169/208) -26/04/01 05:55:56 INFO Executor: Finished task 169.0 in stage 8.0 (TID 177). 2984 bytes result sent to driver -26/04/01 05:55:56 INFO TaskSetManager: Starting task 173.0 in stage 8.0 (TID 181) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:56 INFO TaskSetManager: Finished task 169.0 in stage 8.0 (TID 177) in 1679 ms on 10.0.0.133 (executor driver) (170/208) -26/04/01 05:55:56 INFO Executor: Running task 173.0 in stage 8.0 (TID 181) -26/04/01 05:55:56 INFO Executor: Finished task 170.0 in stage 8.0 (TID 178). 2984 bytes result sent to driver -26/04/01 05:55:56 INFO TaskSetManager: Starting task 174.0 in stage 8.0 (TID 182) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:56 INFO TaskSetManager: Finished task 170.0 in stage 8.0 (TID 178) in 1675 ms on 10.0.0.133 (executor driver) (171/208) -26/04/01 05:55:56 INFO Executor: Running task 174.0 in stage 8.0 (TID 182) -26/04/01 05:55:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:56 INFO Executor: Finished task 171.0 in stage 8.0 (TID 179). 2984 bytes result sent to driver -26/04/01 05:55:56 INFO TaskSetManager: Starting task 175.0 in stage 8.0 (TID 183) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:56 INFO TaskSetManager: Finished task 171.0 in stage 8.0 (TID 179) in 1685 ms on 10.0.0.133 (executor driver) (172/208) -26/04/01 05:55:56 INFO Executor: Running task 175.0 in stage 8.0 (TID 183) -26/04/01 05:55:56 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:56 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:58 INFO Executor: Finished task 174.0 in stage 8.0 (TID 182). 2984 bytes result sent to driver -26/04/01 05:55:58 INFO Executor: Finished task 173.0 in stage 8.0 (TID 181). 2984 bytes result sent to driver -26/04/01 05:55:58 INFO TaskSetManager: Starting task 176.0 in stage 8.0 (TID 184) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:58 INFO TaskSetManager: Starting task 177.0 in stage 8.0 (TID 185) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:58 INFO Executor: Running task 176.0 in stage 8.0 (TID 184) -26/04/01 05:55:58 INFO TaskSetManager: Finished task 174.0 in stage 8.0 (TID 182) in 1675 ms on 10.0.0.133 (executor driver) (173/208) -26/04/01 05:55:58 INFO Executor: Running task 177.0 in stage 8.0 (TID 185) -26/04/01 05:55:58 INFO TaskSetManager: Finished task 173.0 in stage 8.0 (TID 181) in 1676 ms on 10.0.0.133 (executor driver) (174/208) -26/04/01 05:55:58 INFO Executor: Finished task 172.0 in stage 8.0 (TID 180). 3027 bytes result sent to driver -26/04/01 05:55:58 INFO TaskSetManager: Starting task 178.0 in stage 8.0 (TID 186) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:58 INFO TaskSetManager: Finished task 172.0 in stage 8.0 (TID 180) in 1681 ms on 10.0.0.133 (executor driver) (175/208) -26/04/01 05:55:58 INFO Executor: Running task 178.0 in stage 8.0 (TID 186) -26/04/01 05:55:58 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:58 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:58 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:55:58 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:58 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:58 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:58 INFO Executor: Finished task 175.0 in stage 8.0 (TID 183). 3027 bytes result sent to driver -26/04/01 05:55:58 INFO TaskSetManager: Starting task 179.0 in stage 8.0 (TID 187) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:55:58 INFO TaskSetManager: Finished task 175.0 in stage 8.0 (TID 183) in 1677 ms on 10.0.0.133 (executor driver) (176/208) -26/04/01 05:55:58 INFO Executor: Running task 179.0 in stage 8.0 (TID 187) -26/04/01 05:55:58 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:55:58 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:59 INFO Executor: Finished task 178.0 in stage 8.0 (TID 186). 2984 bytes result sent to driver -26/04/01 05:55:59 INFO Executor: Finished task 177.0 in stage 8.0 (TID 185). 3027 bytes result sent to driver -26/04/01 05:55:59 INFO TaskSetManager: Starting task 180.0 in stage 8.0 (TID 188) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:59 INFO Executor: Finished task 176.0 in stage 8.0 (TID 184). 3027 bytes result sent to driver -26/04/01 05:55:59 INFO Executor: Running task 180.0 in stage 8.0 (TID 188) -26/04/01 05:55:59 INFO TaskSetManager: Starting task 181.0 in stage 8.0 (TID 189) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:59 INFO Executor: Running task 181.0 in stage 8.0 (TID 189) -26/04/01 05:55:59 INFO TaskSetManager: Starting task 182.0 in stage 8.0 (TID 190) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:59 INFO TaskSetManager: Finished task 178.0 in stage 8.0 (TID 186) in 1675 ms on 10.0.0.133 (executor driver) (177/208) -26/04/01 05:55:59 INFO Executor: Running task 182.0 in stage 8.0 (TID 190) -26/04/01 05:55:59 INFO TaskSetManager: Finished task 177.0 in stage 8.0 (TID 185) in 1678 ms on 10.0.0.133 (executor driver) (178/208) -26/04/01 05:55:59 INFO TaskSetManager: Finished task 176.0 in stage 8.0 (TID 184) in 1678 ms on 10.0.0.133 (executor driver) (179/208) -26/04/01 05:55:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:55:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:55:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:55:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:55:59 INFO Executor: Finished task 179.0 in stage 8.0 (TID 187). 2984 bytes result sent to driver -26/04/01 05:55:59 INFO TaskSetManager: Starting task 183.0 in stage 8.0 (TID 191) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:55:59 INFO TaskSetManager: Finished task 179.0 in stage 8.0 (TID 187) in 1675 ms on 10.0.0.133 (executor driver) (180/208) -26/04/01 05:55:59 INFO Executor: Running task 183.0 in stage 8.0 (TID 191) -26/04/01 05:55:59 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:55:59 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:01 INFO Executor: Finished task 182.0 in stage 8.0 (TID 190). 2984 bytes result sent to driver -26/04/01 05:56:01 INFO TaskSetManager: Starting task 184.0 in stage 8.0 (TID 192) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:01 INFO TaskSetManager: Finished task 182.0 in stage 8.0 (TID 190) in 1666 ms on 10.0.0.133 (executor driver) (181/208) -26/04/01 05:56:01 INFO Executor: Running task 184.0 in stage 8.0 (TID 192) -26/04/01 05:56:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:01 INFO Executor: Finished task 180.0 in stage 8.0 (TID 188). 3027 bytes result sent to driver -26/04/01 05:56:01 INFO TaskSetManager: Starting task 185.0 in stage 8.0 (TID 193) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:01 INFO Executor: Finished task 181.0 in stage 8.0 (TID 189). 3027 bytes result sent to driver -26/04/01 05:56:01 INFO Executor: Running task 185.0 in stage 8.0 (TID 193) -26/04/01 05:56:01 INFO TaskSetManager: Starting task 186.0 in stage 8.0 (TID 194) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:01 INFO TaskSetManager: Finished task 180.0 in stage 8.0 (TID 188) in 1676 ms on 10.0.0.133 (executor driver) (182/208) -26/04/01 05:56:01 INFO Executor: Running task 186.0 in stage 8.0 (TID 194) -26/04/01 05:56:01 INFO TaskSetManager: Finished task 181.0 in stage 8.0 (TID 189) in 1676 ms on 10.0.0.133 (executor driver) (183/208) -26/04/01 05:56:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:01 INFO Executor: Finished task 183.0 in stage 8.0 (TID 191). 3027 bytes result sent to driver -26/04/01 05:56:01 INFO TaskSetManager: Starting task 187.0 in stage 8.0 (TID 195) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:01 INFO Executor: Running task 187.0 in stage 8.0 (TID 195) -26/04/01 05:56:01 INFO TaskSetManager: Finished task 183.0 in stage 8.0 (TID 191) in 1668 ms on 10.0.0.133 (executor driver) (184/208) -26/04/01 05:56:01 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:01 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:03 INFO Executor: Finished task 185.0 in stage 8.0 (TID 193). 2984 bytes result sent to driver -26/04/01 05:56:03 INFO TaskSetManager: Starting task 188.0 in stage 8.0 (TID 196) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:03 INFO TaskSetManager: Finished task 185.0 in stage 8.0 (TID 193) in 1673 ms on 10.0.0.133 (executor driver) (185/208) -26/04/01 05:56:03 INFO Executor: Running task 188.0 in stage 8.0 (TID 196) -26/04/01 05:56:03 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:03 INFO Executor: Finished task 186.0 in stage 8.0 (TID 194). 2984 bytes result sent to driver -26/04/01 05:56:03 INFO TaskSetManager: Starting task 189.0 in stage 8.0 (TID 197) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:03 INFO TaskSetManager: Finished task 186.0 in stage 8.0 (TID 194) in 1678 ms on 10.0.0.133 (executor driver) (186/208) -26/04/01 05:56:03 INFO Executor: Running task 189.0 in stage 8.0 (TID 197) -26/04/01 05:56:03 INFO Executor: Finished task 184.0 in stage 8.0 (TID 192). 3027 bytes result sent to driver -26/04/01 05:56:03 INFO TaskSetManager: Starting task 190.0 in stage 8.0 (TID 198) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:03 INFO TaskSetManager: Finished task 184.0 in stage 8.0 (TID 192) in 1689 ms on 10.0.0.133 (executor driver) (187/208) -26/04/01 05:56:03 INFO Executor: Running task 190.0 in stage 8.0 (TID 198) -26/04/01 05:56:03 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:03 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:03 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:03 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:03 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:03 INFO Executor: Finished task 187.0 in stage 8.0 (TID 195). 3027 bytes result sent to driver -26/04/01 05:56:03 INFO TaskSetManager: Starting task 191.0 in stage 8.0 (TID 199) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:03 INFO Executor: Running task 191.0 in stage 8.0 (TID 199) -26/04/01 05:56:03 INFO TaskSetManager: Finished task 187.0 in stage 8.0 (TID 195) in 1680 ms on 10.0.0.133 (executor driver) (188/208) -26/04/01 05:56:03 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:03 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:04 INFO Executor: Finished task 189.0 in stage 8.0 (TID 197). 2984 bytes result sent to driver -26/04/01 05:56:04 INFO Executor: Finished task 188.0 in stage 8.0 (TID 196). 3027 bytes result sent to driver -26/04/01 05:56:04 INFO TaskSetManager: Starting task 192.0 in stage 8.0 (TID 200) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:04 INFO Executor: Running task 192.0 in stage 8.0 (TID 200) -26/04/01 05:56:04 INFO TaskSetManager: Starting task 193.0 in stage 8.0 (TID 201) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:04 INFO Executor: Running task 193.0 in stage 8.0 (TID 201) -26/04/01 05:56:04 INFO TaskSetManager: Finished task 189.0 in stage 8.0 (TID 197) in 1674 ms on 10.0.0.133 (executor driver) (189/208) -26/04/01 05:56:04 INFO TaskSetManager: Finished task 188.0 in stage 8.0 (TID 196) in 1680 ms on 10.0.0.133 (executor driver) (190/208) -26/04/01 05:56:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 805306368-848563442, partition values: [empty row] -26/04/01 05:56:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 805306368-848649288, partition values: [empty row] -26/04/01 05:56:04 INFO Executor: Finished task 190.0 in stage 8.0 (TID 198). 2984 bytes result sent to driver -26/04/01 05:56:04 INFO TaskSetManager: Starting task 194.0 in stage 8.0 (TID 202) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:04 INFO Executor: Running task 194.0 in stage 8.0 (TID 202) -26/04/01 05:56:04 INFO TaskSetManager: Finished task 190.0 in stage 8.0 (TID 198) in 1677 ms on 10.0.0.133 (executor driver) (191/208) -26/04/01 05:56:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 805306368-848496433, partition values: [empty row] -26/04/01 05:56:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:04 INFO Executor: Finished task 191.0 in stage 8.0 (TID 199). 3027 bytes result sent to driver -26/04/01 05:56:04 INFO TaskSetManager: Starting task 195.0 in stage 8.0 (TID 203) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:04 INFO TaskSetManager: Finished task 191.0 in stage 8.0 (TID 199) in 1674 ms on 10.0.0.133 (executor driver) (192/208) -26/04/01 05:56:04 INFO Executor: Running task 195.0 in stage 8.0 (TID 203) -26/04/01 05:56:04 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 805306368-848463796, partition values: [empty row] -26/04/01 05:56:04 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 805306368-848521656, partition values: [empty row] -26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 805306368-848489219, partition values: [empty row] -26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 805306368-848594494, partition values: [empty row] -26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 805306368-848442878, partition values: [empty row] -26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:05 INFO Executor: Finished task 193.0 in stage 8.0 (TID 201). 2984 bytes result sent to driver -26/04/01 05:56:05 INFO TaskSetManager: Starting task 196.0 in stage 8.0 (TID 204) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:05 INFO TaskSetManager: Finished task 193.0 in stage 8.0 (TID 201) in 1045 ms on 10.0.0.133 (executor driver) (193/208) -26/04/01 05:56:05 INFO Executor: Running task 196.0 in stage 8.0 (TID 204) -26/04/01 05:56:05 INFO Executor: Finished task 194.0 in stage 8.0 (TID 202). 2984 bytes result sent to driver -26/04/01 05:56:05 INFO TaskSetManager: Starting task 197.0 in stage 8.0 (TID 205) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:05 INFO TaskSetManager: Finished task 194.0 in stage 8.0 (TID 202) in 1042 ms on 10.0.0.133 (executor driver) (194/208) -26/04/01 05:56:05 INFO Executor: Running task 197.0 in stage 8.0 (TID 205) -26/04/01 05:56:05 INFO Executor: Finished task 192.0 in stage 8.0 (TID 200). 3027 bytes result sent to driver -26/04/01 05:56:05 INFO TaskSetManager: Starting task 198.0 in stage 8.0 (TID 206) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:05 INFO TaskSetManager: Finished task 192.0 in stage 8.0 (TID 200) in 1047 ms on 10.0.0.133 (executor driver) (195/208) -26/04/01 05:56:05 INFO Executor: Running task 198.0 in stage 8.0 (TID 206) -26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 805306368-848432523, partition values: [empty row] -26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 805306368-848422348, partition values: [empty row] -26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 805306368-848399844, partition values: [empty row] -26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:05 INFO Executor: Finished task 195.0 in stage 8.0 (TID 203). 2984 bytes result sent to driver -26/04/01 05:56:05 INFO TaskSetManager: Starting task 199.0 in stage 8.0 (TID 207) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:05 INFO TaskSetManager: Finished task 195.0 in stage 8.0 (TID 203) in 1044 ms on 10.0.0.133 (executor driver) (196/208) -26/04/01 05:56:05 INFO Executor: Running task 199.0 in stage 8.0 (TID 207) -26/04/01 05:56:05 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 805306368-848378208, partition values: [empty row] -26/04/01 05:56:05 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 805306368-848411809, partition values: [empty row] -26/04/01 05:56:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 805306368-848379799, partition values: [empty row] -26/04/01 05:56:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 805306368-848423659, partition values: [empty row] -26/04/01 05:56:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 805306368-848348606, partition values: [empty row] -26/04/01 05:56:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:06 INFO Executor: Finished task 198.0 in stage 8.0 (TID 206). 2984 bytes result sent to driver -26/04/01 05:56:06 INFO TaskSetManager: Starting task 200.0 in stage 8.0 (TID 208) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9708 bytes) -26/04/01 05:56:06 INFO Executor: Running task 200.0 in stage 8.0 (TID 208) -26/04/01 05:56:06 INFO TaskSetManager: Finished task 198.0 in stage 8.0 (TID 206) in 1038 ms on 10.0.0.133 (executor driver) (197/208) -26/04/01 05:56:06 INFO Executor: Finished task 196.0 in stage 8.0 (TID 204). 3027 bytes result sent to driver -26/04/01 05:56:06 INFO TaskSetManager: Starting task 201.0 in stage 8.0 (TID 209) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:06 INFO TaskSetManager: Finished task 196.0 in stage 8.0 (TID 204) in 1042 ms on 10.0.0.133 (executor driver) (198/208) -26/04/01 05:56:06 INFO Executor: Running task 201.0 in stage 8.0 (TID 209) -26/04/01 05:56:06 INFO Executor: Finished task 197.0 in stage 8.0 (TID 205). 2984 bytes result sent to driver -26/04/01 05:56:06 INFO TaskSetManager: Starting task 202.0 in stage 8.0 (TID 210) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:06 INFO TaskSetManager: Finished task 197.0 in stage 8.0 (TID 205) in 1041 ms on 10.0.0.133 (executor driver) (199/208) -26/04/01 05:56:06 INFO Executor: Running task 202.0 in stage 8.0 (TID 210) -26/04/01 05:56:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 805306368-848344114, partition values: [empty row] -26/04/01 05:56:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 805306368-848329366, partition values: [empty row] -26/04/01 05:56:06 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 805306368-848328570, partition values: [empty row] -26/04/01 05:56:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:06 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:07 INFO Executor: Finished task 199.0 in stage 8.0 (TID 207). 2984 bytes result sent to driver -26/04/01 05:56:07 INFO TaskSetManager: Starting task 203.0 in stage 8.0 (TID 211) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:07 INFO Executor: Running task 203.0 in stage 8.0 (TID 211) -26/04/01 05:56:07 INFO TaskSetManager: Finished task 199.0 in stage 8.0 (TID 207) in 1043 ms on 10.0.0.133 (executor driver) (200/208) -26/04/01 05:56:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 805306368-848301737, partition values: [empty row] -26/04/01 05:56:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 805306368-848338051, partition values: [empty row] -26/04/01 05:56:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 805306368-848329327, partition values: [empty row] -26/04/01 05:56:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 805306368-848306153, partition values: [empty row] -26/04/01 05:56:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 805306368-848278425, partition values: [empty row] -26/04/01 05:56:07 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:08 INFO Executor: Finished task 200.0 in stage 8.0 (TID 208). 3027 bytes result sent to driver -26/04/01 05:56:08 INFO TaskSetManager: Starting task 204.0 in stage 8.0 (TID 212) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:08 INFO Executor: Running task 204.0 in stage 8.0 (TID 212) -26/04/01 05:56:08 INFO TaskSetManager: Finished task 200.0 in stage 8.0 (TID 208) in 1042 ms on 10.0.0.133 (executor driver) (201/208) -26/04/01 05:56:08 INFO Executor: Finished task 201.0 in stage 8.0 (TID 209). 2984 bytes result sent to driver -26/04/01 05:56:08 INFO TaskSetManager: Starting task 205.0 in stage 8.0 (TID 213) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:08 INFO Executor: Finished task 202.0 in stage 8.0 (TID 210). 2984 bytes result sent to driver -26/04/01 05:56:08 INFO TaskSetManager: Finished task 201.0 in stage 8.0 (TID 209) in 1043 ms on 10.0.0.133 (executor driver) (202/208) -26/04/01 05:56:08 INFO Executor: Running task 205.0 in stage 8.0 (TID 213) -26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 805306368-848263904, partition values: [empty row] -26/04/01 05:56:08 INFO TaskSetManager: Starting task 206.0 in stage 8.0 (TID 214) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:08 INFO TaskSetManager: Finished task 202.0 in stage 8.0 (TID 210) in 1044 ms on 10.0.0.133 (executor driver) (203/208) -26/04/01 05:56:08 INFO Executor: Running task 206.0 in stage 8.0 (TID 214) -26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 805306368-848235021, partition values: [empty row] -26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 805306368-848107384, partition values: [empty row] -26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:08 INFO Executor: Finished task 203.0 in stage 8.0 (TID 211). 2984 bytes result sent to driver -26/04/01 05:56:08 INFO TaskSetManager: Starting task 207.0 in stage 8.0 (TID 215) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:08 INFO TaskSetManager: Finished task 203.0 in stage 8.0 (TID 211) in 1041 ms on 10.0.0.133 (executor driver) (204/208) -26/04/01 05:56:08 INFO Executor: Running task 207.0 in stage 8.0 (TID 215) -26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 805306368-848005421, partition values: [empty row] -26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 805306368-848247851, partition values: [empty row] -26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 805306368-848050794, partition values: [empty row] -26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 805306368-848234776, partition values: [empty row] -26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 805306368-847699715, partition values: [empty row] -26/04/01 05:56:08 INFO FilterCompat: Filtering using predicate: and(noteq(l_shipdate, null), lteq(l_shipdate, 10493)) -26/04/01 05:56:09 INFO Executor: Finished task 204.0 in stage 8.0 (TID 212). 2984 bytes result sent to driver -26/04/01 05:56:09 INFO TaskSetManager: Finished task 204.0 in stage 8.0 (TID 212) in 1042 ms on 10.0.0.133 (executor driver) (205/208) -26/04/01 05:56:09 INFO Executor: Finished task 205.0 in stage 8.0 (TID 213). 2984 bytes result sent to driver -26/04/01 05:56:09 INFO TaskSetManager: Finished task 205.0 in stage 8.0 (TID 213) in 1042 ms on 10.0.0.133 (executor driver) (206/208) -26/04/01 05:56:09 INFO Executor: Finished task 206.0 in stage 8.0 (TID 214). 2984 bytes result sent to driver -26/04/01 05:56:09 INFO TaskSetManager: Finished task 206.0 in stage 8.0 (TID 214) in 1044 ms on 10.0.0.133 (executor driver) (207/208) -26/04/01 05:56:09 INFO Executor: Finished task 207.0 in stage 8.0 (TID 215). 2984 bytes result sent to driver -26/04/01 05:56:09 INFO TaskSetManager: Finished task 207.0 in stage 8.0 (TID 215) in 1035 ms on 10.0.0.133 (executor driver) (208/208) -26/04/01 05:56:09 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool -26/04/01 05:56:09 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 85.502 s -26/04/01 05:56:09 INFO DAGScheduler: looking for newly runnable stages -26/04/01 05:56:09 INFO DAGScheduler: running: Set() -26/04/01 05:56:09 INFO DAGScheduler: waiting: Set() -26/04/01 05:56:09 INFO DAGScheduler: failed: Set() -26/04/01 05:56:09 INFO ShufflePartitionsUtil: For shuffle(0), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 05:56:09 INFO HashAggregateExec: spark.sql.codegen.aggregate.map.twolevel.enabled is set to true, but current version of codegened fast hashmap does not support this aggregate. -26/04/01 05:56:09 INFO CodeGenerator: Code generated in 13.867709 ms -26/04/01 05:56:09 INFO CodeGenerator: Code generated in 3.550125 ms -26/04/01 05:56:09 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:56:09 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 05:56:09 INFO DAGScheduler: Final stage: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:56:09 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 9) -26/04/01 05:56:09 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:09 INFO DAGScheduler: Submitting ResultStage 10 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:56:09 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 91.1 KiB, free 8.6 GiB) -26/04/01 05:56:09 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) -26/04/01 05:56:09 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:58187 (size: 35.0 KiB, free: 8.6 GiB) -26/04/01 05:56:09 INFO SparkContext: Created broadcast 10 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:09 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 10 (MapPartitionsRDD[24] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:56:09 INFO TaskSchedulerImpl: Adding task set 10.0 with 1 tasks resource profile 0 -26/04/01 05:56:09 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 216) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8999 bytes) -26/04/01 05:56:09 INFO Executor: Running task 0.0 in stage 10.0 (TID 216) -26/04/01 05:56:09 INFO ShuffleBlockFetcherIterator: Getting 208 (184.6 KiB) non-empty blocks including 208 (184.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms -26/04/01 05:56:09 INFO CodeGenerator: Code generated in 9.284042 ms -26/04/01 05:56:09 INFO Executor: Finished task 0.0 in stage 10.0 (TID 216). 5717 bytes result sent to driver -26/04/01 05:56:09 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 216) in 60 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:56:09 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool -26/04/01 05:56:09 INFO DAGScheduler: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.063 s -26/04/01 05:56:09 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:56:09 INFO TaskSchedulerImpl: Killing all running tasks in stage 10: Stage finished -26/04/01 05:56:09 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.067610 s -26/04/01 05:56:09 INFO DAGScheduler: Registering RDD 25 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 -26/04/01 05:56:09 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 05:56:09 INFO DAGScheduler: Final stage: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:56:09 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 11) -26/04/01 05:56:09 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:09 INFO DAGScheduler: Submitting ShuffleMapStage 12 (MapPartitionsRDD[25] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:56:09 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 91.8 KiB, free 8.6 GiB) -26/04/01 05:56:09 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 35.3 KiB, free 8.6 GiB) -26/04/01 05:56:09 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:58187 (size: 35.3 KiB, free: 8.6 GiB) -26/04/01 05:56:09 INFO SparkContext: Created broadcast 11 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:09 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 12 (MapPartitionsRDD[25] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:56:09 INFO TaskSchedulerImpl: Adding task set 12.0 with 1 tasks resource profile 0 -26/04/01 05:56:09 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 217) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8988 bytes) -26/04/01 05:56:09 INFO Executor: Running task 0.0 in stage 12.0 (TID 217) -26/04/01 05:56:09 INFO CodeGenerator: Code generated in 2.071125 ms -26/04/01 05:56:09 INFO ShuffleBlockFetcherIterator: Getting 208 (184.6 KiB) non-empty blocks including 208 (184.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:09 INFO Executor: Finished task 0.0 in stage 12.0 (TID 217). 5524 bytes result sent to driver -26/04/01 05:56:09 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 217) in 26 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:56:09 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool -26/04/01 05:56:09 INFO DAGScheduler: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.030 s -26/04/01 05:56:09 INFO DAGScheduler: looking for newly runnable stages -26/04/01 05:56:09 INFO DAGScheduler: running: Set() -26/04/01 05:56:09 INFO DAGScheduler: waiting: Set() -26/04/01 05:56:09 INFO DAGScheduler: failed: Set() -26/04/01 05:56:09 INFO ShufflePartitionsUtil: For shuffle(1), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 05:56:09 INFO CodeGenerator: Code generated in 3.744459 ms -26/04/01 05:56:09 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:56:09 INFO DAGScheduler: Got job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 05:56:09 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:56:09 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 14) -26/04/01 05:56:09 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:09 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:56:09 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 66.7 KiB, free 8.6 GiB) -26/04/01 05:56:09 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 27.3 KiB, free 8.6 GiB) -26/04/01 05:56:09 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:58187 (size: 27.3 KiB, free: 8.6 GiB) -26/04/01 05:56:09 INFO SparkContext: Created broadcast 12 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:09 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 15 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:56:09 INFO TaskSchedulerImpl: Adding task set 15.0 with 1 tasks resource profile 0 -26/04/01 05:56:09 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 218) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8999 bytes) -26/04/01 05:56:09 INFO Executor: Running task 0.0 in stage 15.0 (TID 218) -26/04/01 05:56:09 INFO ShuffleBlockFetcherIterator: Getting 1 (912.0 B) non-empty blocks including 1 (912.0 B) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:09 INFO CodeGenerator: Code generated in 2.812042 ms -26/04/01 05:56:09 INFO CodeGenerator: Code generated in 2.661958 ms -26/04/01 05:56:09 INFO Executor: Finished task 0.0 in stage 15.0 (TID 218). 7302 bytes result sent to driver -26/04/01 05:56:09 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 218) in 21 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:56:09 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool -26/04/01 05:56:09 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.024 s -26/04/01 05:56:09 INFO DAGScheduler: Job 11 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:56:09 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished -26/04/01 05:56:09 INFO DAGScheduler: Job 11 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.024247 s -26/04/01 05:56:09 INFO SparkContext: SparkContext is stopping with exitCode 0. -26/04/01 05:56:09 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! -26/04/01 05:56:09 INFO MemoryStore: MemoryStore cleared -26/04/01 05:56:09 INFO BlockManager: BlockManager stopped -26/04/01 05:56:09 INFO BlockManagerMaster: BlockManagerMaster stopped -26/04/01 05:56:09 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! -26/04/01 05:56:09 INFO SparkContext: Successfully stopped SparkContext -26/04/01 05:56:09 INFO ShutdownHookManager: Shutdown hook called -26/04/01 05:56:09 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-61f0e515-7a17-4e12-9e70-40ca151350fa -26/04/01 05:56:09 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-8e526fcf-8de7-4d28-9732-922dc858ca07/pyspark-260dfcd9-b64e-4cbd-bc76-c977a7a8a155 -26/04/01 05:56:09 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-8e526fcf-8de7-4d28-9732-922dc858ca07 - 89.94 real 363.21 user 10.80 sys - 2831646720 maximum resident set size - 0 average shared memory size - 0 average unshared data size - 0 average unshared stack size - 203207 page reclaims - 182 page faults - 0 swaps - 0 block input operations - 0 block output operations - 1015 messages sent - 1033 messages received - 1758 signals received - 16605 voluntary context switches - 539811 involuntary context switches - 7848607567173 instructions retired - 1350297036464 cycles elapsed - 2810317944 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.log b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.log deleted file mode 100644 index 90aba2134d..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.log +++ /dev/null @@ -1,330 +0,0 @@ -Registering table customer from /opt/tpch/sf100/customer -Registering table lineitem from /opt/tpch/sf100/lineitem -Registering table nation from /opt/tpch/sf100/nation -Registering table orders from /opt/tpch/sf100/orders -Registering table part from /opt/tpch/sf100/part -Registering table partsupp from /opt/tpch/sf100/partsupp -Registering table region from /opt/tpch/sf100/region -Registering table supplier from /opt/tpch/sf100/supplier - -============================================================ -Starting iteration 1 of 1 -============================================================ - -Running query 5 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q5.sql -Executing: -- CometBench-H query 5 derived from TPC-H query 5 under the terms of the TPC Fair Use Policy. --- TP... -== Physical Plan == -AdaptiveSparkPlan (44) -+- Sort (43) - +- Exchange (42) - +- HashAggregate (41) - +- Exchange (40) - +- HashAggregate (39) - +- Project (38) - +- BroadcastHashJoin Inner BuildRight (37) - :- Project (32) - : +- BroadcastHashJoin Inner BuildRight (31) - : :- Project (27) - : : +- SortMergeJoin Inner (26) - : : :- Sort (21) - : : : +- Exchange (20) - : : : +- Project (19) - : : : +- SortMergeJoin Inner (18) - : : : :- Sort (13) - : : : : +- Exchange (12) - : : : : +- Project (11) - : : : : +- SortMergeJoin Inner (10) - : : : : :- Sort (4) - : : : : : +- Exchange (3) - : : : : : +- Filter (2) - : : : : : +- Scan parquet (1) - : : : : +- Sort (9) - : : : : +- Exchange (8) - : : : : +- Project (7) - : : : : +- Filter (6) - : : : : +- Scan parquet (5) - : : : +- Sort (17) - : : : +- Exchange (16) - : : : +- Filter (15) - : : : +- Scan parquet (14) - : : +- Sort (25) - : : +- Exchange (24) - : : +- Filter (23) - : : +- Scan parquet (22) - : +- BroadcastExchange (30) - : +- Filter (29) - : +- Scan parquet (28) - +- BroadcastExchange (36) - +- Project (35) - +- Filter (34) - +- Scan parquet (33) - - -(1) Scan parquet -Output [2]: [c_custkey#0L, c_nationkey#3L] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/customer] -PushedFilters: [IsNotNull(c_custkey), IsNotNull(c_nationkey)] -ReadSchema: struct - -(2) Filter -Input [2]: [c_custkey#0L, c_nationkey#3L] -Condition : (isnotnull(c_custkey#0L) AND isnotnull(c_nationkey#3L)) - -(3) Exchange -Input [2]: [c_custkey#0L, c_nationkey#3L] -Arguments: hashpartitioning(c_custkey#0L, 200), ENSURE_REQUIREMENTS, [plan_id=159] - -(4) Sort -Input [2]: [c_custkey#0L, c_nationkey#3L] -Arguments: [c_custkey#0L ASC NULLS FIRST], false, 0 - -(5) Scan parquet -Output [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/orders] -PushedFilters: [IsNotNull(o_orderdate), GreaterThanOrEqual(o_orderdate,1994-01-01), LessThan(o_orderdate,1995-01-01), IsNotNull(o_custkey), IsNotNull(o_orderkey)] -ReadSchema: struct - -(6) Filter -Input [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] -Condition : ((((isnotnull(o_orderdate#60) AND (o_orderdate#60 >= 1994-01-01)) AND (o_orderdate#60 < 1995-01-01)) AND isnotnull(o_custkey#57L)) AND isnotnull(o_orderkey#56L)) - -(7) Project -Output [2]: [o_orderkey#56L, o_custkey#57L] -Input [3]: [o_orderkey#56L, o_custkey#57L, o_orderdate#60] - -(8) Exchange -Input [2]: [o_orderkey#56L, o_custkey#57L] -Arguments: hashpartitioning(o_custkey#57L, 200), ENSURE_REQUIREMENTS, [plan_id=160] - -(9) Sort -Input [2]: [o_orderkey#56L, o_custkey#57L] -Arguments: [o_custkey#57L ASC NULLS FIRST], false, 0 - -(10) SortMergeJoin -Left keys [1]: [c_custkey#0L] -Right keys [1]: [o_custkey#57L] -Join type: Inner -Join condition: None - -(11) Project -Output [2]: [c_nationkey#3L, o_orderkey#56L] -Input [4]: [c_custkey#0L, c_nationkey#3L, o_orderkey#56L, o_custkey#57L] - -(12) Exchange -Input [2]: [c_nationkey#3L, o_orderkey#56L] -Arguments: hashpartitioning(o_orderkey#56L, 200), ENSURE_REQUIREMENTS, [plan_id=167] - -(13) Sort -Input [2]: [c_nationkey#3L, o_orderkey#56L] -Arguments: [o_orderkey#56L ASC NULLS FIRST], false, 0 - -(14) Scan parquet -Output [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] -PushedFilters: [IsNotNull(l_orderkey), IsNotNull(l_suppkey)] -ReadSchema: struct - -(15) Filter -Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Condition : (isnotnull(l_orderkey#16L) AND isnotnull(l_suppkey#18L)) - -(16) Exchange -Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: hashpartitioning(l_orderkey#16L, 200), ENSURE_REQUIREMENTS, [plan_id=168] - -(17) Sort -Input [4]: [l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: [l_orderkey#16L ASC NULLS FIRST], false, 0 - -(18) SortMergeJoin -Left keys [1]: [o_orderkey#56L] -Right keys [1]: [l_orderkey#16L] -Join type: Inner -Join condition: None - -(19) Project -Output [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Input [6]: [c_nationkey#3L, o_orderkey#56L, l_orderkey#16L, l_suppkey#18L, l_extendedprice#21, l_discount#22] - -(20) Exchange -Input [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: hashpartitioning(l_suppkey#18L, c_nationkey#3L, 200), ENSURE_REQUIREMENTS, [plan_id=175] - -(21) Sort -Input [4]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22] -Arguments: [l_suppkey#18L ASC NULLS FIRST, c_nationkey#3L ASC NULLS FIRST], false, 0 - -(22) Scan parquet -Output [2]: [s_suppkey#108L, s_nationkey#111L] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/supplier] -PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] -ReadSchema: struct - -(23) Filter -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Condition : (isnotnull(s_suppkey#108L) AND isnotnull(s_nationkey#111L)) - -(24) Exchange -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: hashpartitioning(s_suppkey#108L, s_nationkey#111L, 200), ENSURE_REQUIREMENTS, [plan_id=176] - -(25) Sort -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: [s_suppkey#108L ASC NULLS FIRST, s_nationkey#111L ASC NULLS FIRST], false, 0 - -(26) SortMergeJoin -Left keys [2]: [l_suppkey#18L, c_nationkey#3L] -Right keys [2]: [s_suppkey#108L, s_nationkey#111L] -Join type: Inner -Join condition: None - -(27) Project -Output [3]: [l_extendedprice#21, l_discount#22, s_nationkey#111L] -Input [6]: [c_nationkey#3L, l_suppkey#18L, l_extendedprice#21, l_discount#22, s_suppkey#108L, s_nationkey#111L] - -(28) Scan parquet -Output [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/nation] -PushedFilters: [IsNotNull(n_nationkey), IsNotNull(n_regionkey)] -ReadSchema: struct - -(29) Filter -Input [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] -Condition : ((isnotnull(n_nationkey#48L) AND isnotnull(n_regionkey#50L)) AND might_contain(Subquery subquery#128, [id=#134], xxhash64(n_regionkey#50L, 42))) - -(30) BroadcastExchange -Input [3]: [n_nationkey#48L, n_name#49, n_regionkey#50L] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [plan_id=182] - -(31) BroadcastHashJoin -Left keys [1]: [s_nationkey#111L] -Right keys [1]: [n_nationkey#48L] -Join type: Inner -Join condition: None - -(32) Project -Output [4]: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L] -Input [6]: [l_extendedprice#21, l_discount#22, s_nationkey#111L, n_nationkey#48L, n_name#49, n_regionkey#50L] - -(33) Scan parquet -Output [2]: [r_regionkey#102L, r_name#103] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/region] -PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AFRICA), IsNotNull(r_regionkey)] -ReadSchema: struct - -(34) Filter -Input [2]: [r_regionkey#102L, r_name#103] -Condition : ((isnotnull(r_name#103) AND (r_name#103 = AFRICA)) AND isnotnull(r_regionkey#102L)) - -(35) Project -Output [1]: [r_regionkey#102L] -Input [2]: [r_regionkey#102L, r_name#103] - -(36) BroadcastExchange -Input [1]: [r_regionkey#102L] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, true]),false), [plan_id=186] - -(37) BroadcastHashJoin -Left keys [1]: [n_regionkey#50L] -Right keys [1]: [r_regionkey#102L] -Join type: Inner -Join condition: None - -(38) Project -Output [3]: [l_extendedprice#21, l_discount#22, n_name#49] -Input [5]: [l_extendedprice#21, l_discount#22, n_name#49, n_regionkey#50L, r_regionkey#102L] - -(39) HashAggregate -Input [3]: [l_extendedprice#21, l_discount#22, n_name#49] -Keys [1]: [n_name#49] -Functions [1]: [partial_sum((l_extendedprice#21 * (1 - l_discount#22)))] -Aggregate Attributes [2]: [sum#129, isEmpty#130] -Results [3]: [n_name#49, sum#131, isEmpty#132] - -(40) Exchange -Input [3]: [n_name#49, sum#131, isEmpty#132] -Arguments: hashpartitioning(n_name#49, 200), ENSURE_REQUIREMENTS, [plan_id=191] - -(41) HashAggregate -Input [3]: [n_name#49, sum#131, isEmpty#132] -Keys [1]: [n_name#49] -Functions [1]: [sum((l_extendedprice#21 * (1 - l_discount#22)))] -Aggregate Attributes [1]: [sum((l_extendedprice#21 * (1 - l_discount#22)))#123] -Results [2]: [n_name#49, sum((l_extendedprice#21 * (1 - l_discount#22)))#123 AS revenue#122] - -(42) Exchange -Input [2]: [n_name#49, revenue#122] -Arguments: rangepartitioning(revenue#122 DESC NULLS LAST, 200), ENSURE_REQUIREMENTS, [plan_id=194] - -(43) Sort -Input [2]: [n_name#49, revenue#122] -Arguments: [revenue#122 DESC NULLS LAST], true, 0 - -(44) AdaptiveSparkPlan -Output [2]: [n_name#49, revenue#122] -Arguments: isFinalPlan=false - -===== Subqueries ===== - -Subquery:1 Hosting operator id = 29 Hosting Expression = Subquery subquery#128, [id=#134] -AdaptiveSparkPlan (51) -+- ObjectHashAggregate (50) - +- Exchange (49) - +- ObjectHashAggregate (48) - +- Project (47) - +- Filter (46) - +- Scan parquet (45) - - -(45) Scan parquet -Output [2]: [r_regionkey#102L, r_name#103] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/region] -PushedFilters: [IsNotNull(r_name), EqualTo(r_name,AFRICA), IsNotNull(r_regionkey)] -ReadSchema: struct - -(46) Filter -Input [2]: [r_regionkey#102L, r_name#103] -Condition : ((isnotnull(r_name#103) AND (r_name#103 = AFRICA)) AND isnotnull(r_regionkey#102L)) - -(47) Project -Output [1]: [r_regionkey#102L] -Input [2]: [r_regionkey#102L, r_name#103] - -(48) ObjectHashAggregate -Input [1]: [r_regionkey#102L] -Keys: [] -Functions [1]: [partial_bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)] -Aggregate Attributes [1]: [buf#133] -Results [1]: [buf#134] - -(49) Exchange -Input [1]: [buf#134] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=132] - -(50) ObjectHashAggregate -Input [1]: [buf#134] -Keys: [] -Functions [1]: [bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)] -Aggregate Attributes [1]: [bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)#126] -Results [1]: [bloom_filter_agg(xxhash64(r_regionkey#102L, 42), 1000000, 8388608, 0, 0)#126 AS bloomFilter#127] - -(51) AdaptiveSparkPlan -Output [1]: [bloomFilter#127] -Arguments: isFinalPlan=false - - - -Query 5 returned 5 rows, hash=f1e68aab4aa9d0988709357fc210d775 -Query 5 took 84.19 seconds - -Iteration 1 took 84.19 seconds - -Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/spark-offheap4g-q5-tpch-1775044656629.json diff --git a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.time b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.time deleted file mode 100644 index e276705a6c..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q5.time +++ /dev/null @@ -1,4441 +0,0 @@ -26/04/01 05:56:10 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) -26/04/01 05:56:10 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address -26/04/01 05:56:10 INFO SparkContext: Running Spark version 3.5.8 -26/04/01 05:56:10 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 05:56:10 INFO SparkContext: Java version 17.0.17 -26/04/01 05:56:10 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable -26/04/01 05:56:10 INFO ResourceUtils: ============================================================== -26/04/01 05:56:10 INFO ResourceUtils: No custom resources configured for spark.driver. -26/04/01 05:56:10 INFO ResourceUtils: ============================================================== -26/04/01 05:56:10 INFO SparkContext: Submitted application: spark-offheap4g-q5 benchmark derived from tpch -26/04/01 05:56:10 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 4096, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) -26/04/01 05:56:10 INFO ResourceProfile: Limiting resource is cpu -26/04/01 05:56:10 INFO ResourceProfileManager: Added ResourceProfile id: 0 -26/04/01 05:56:10 INFO SecurityManager: Changing view acls to: andy -26/04/01 05:56:10 INFO SecurityManager: Changing modify acls to: andy -26/04/01 05:56:10 INFO SecurityManager: Changing view acls groups to: -26/04/01 05:56:10 INFO SecurityManager: Changing modify acls groups to: -26/04/01 05:56:10 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY -26/04/01 05:56:10 INFO Utils: Successfully started service 'sparkDriver' on port 58205. -26/04/01 05:56:10 INFO SparkEnv: Registering MapOutputTracker -26/04/01 05:56:10 INFO SparkEnv: Registering BlockManagerMaster -26/04/01 05:56:10 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information -26/04/01 05:56:10 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up -26/04/01 05:56:10 INFO SparkEnv: Registering BlockManagerMasterHeartbeat -26/04/01 05:56:10 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-3a98b41f-875a-442d-ba64-9adf3d2ffc14 -26/04/01 05:56:10 INFO MemoryStore: MemoryStore started with capacity 8.6 GiB -26/04/01 05:56:10 INFO SparkEnv: Registering OutputCommitCoordinator -26/04/01 05:56:10 INFO Executor: Starting executor ID driver on host 10.0.0.133 -26/04/01 05:56:10 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 05:56:10 INFO Executor: Java version 17.0.17 -26/04/01 05:56:10 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' -26/04/01 05:56:10 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@f860e41 for default. -26/04/01 05:56:10 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 58206. -26/04/01 05:56:10 INFO NettyBlockTransferService: Server created on 10.0.0.133:58206 -26/04/01 05:56:10 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy -26/04/01 05:56:10 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 58206, None) -26/04/01 05:56:10 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:58206 with 8.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 58206, None) -26/04/01 05:56:10 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 58206, None) -26/04/01 05:56:10 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 58206, None) -26/04/01 05:56:11 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. -26/04/01 05:56:11 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. -26/04/01 05:56:11 INFO InMemoryFileIndex: It took 18 ms to list leaf files for 1 paths. -26/04/01 05:56:11 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:56:11 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:56:11 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:56:11 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:56:11 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:11 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:56:11 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:56:11 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:56:11 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:11 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:11 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:56:11 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 -26/04/01 05:56:11 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) -26/04/01 05:56:11 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) -26/04/01 05:56:11 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 1998 bytes result sent to driver -26/04/01 05:56:11 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 118 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:56:11 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool -26/04/01 05:56:11 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.321 s -26/04/01 05:56:11 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:56:11 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished -26/04/01 05:56:11 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.332862 s -26/04/01 05:56:11 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. -26/04/01 05:56:12 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:56:12 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 -26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) -26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) -26/04/01 05:56:12 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2344 bytes result sent to driver -26/04/01 05:56:12 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 14 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool -26/04/01 05:56:12 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.023 s -26/04/01 05:56:12 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:56:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished -26/04/01 05:56:12 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.024672 s -26/04/01 05:56:12 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. -26/04/01 05:56:12 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:56:12 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 -26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) -26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) -26/04/01 05:56:12 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1760 bytes result sent to driver -26/04/01 05:56:12 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 7 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool -26/04/01 05:56:12 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s -26/04/01 05:56:12 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:56:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished -26/04/01 05:56:12 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.017571 s -26/04/01 05:56:12 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. -26/04/01 05:56:12 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:56:12 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 -26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) -26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) -26/04/01 05:56:12 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver -26/04/01 05:56:12 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 7 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool -26/04/01 05:56:12 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s -26/04/01 05:56:12 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:56:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished -26/04/01 05:56:12 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.016001 s -26/04/01 05:56:12 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. -26/04/01 05:56:12 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:56:12 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 -26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9134 bytes) -26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) -26/04/01 05:56:12 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver -26/04/01 05:56:12 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool -26/04/01 05:56:12 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s -26/04/01 05:56:12 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:56:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished -26/04/01 05:56:12 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.014413 s -26/04/01 05:56:12 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 05:56:12 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:56:12 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 -26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) -26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) -26/04/01 05:56:12 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver -26/04/01 05:56:12 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 7 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool -26/04/01 05:56:12 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 05:56:12 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:56:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished -26/04/01 05:56:12 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.014465 s -26/04/01 05:56:12 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. -26/04/01 05:56:12 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:56:12 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 -26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) -26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) -26/04/01 05:56:12 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver -26/04/01 05:56:12 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool -26/04/01 05:56:12 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.012 s -26/04/01 05:56:12 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:56:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished -26/04/01 05:56:12 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.013261 s -26/04/01 05:56:12 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 05:56:12 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 05:56:12 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) -26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:58206 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 -26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) -26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) -26/04/01 05:56:12 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver -26/04/01 05:56:12 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool -26/04/01 05:56:12 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.012 s -26/04/01 05:56:12 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:56:12 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished -26/04/01 05:56:12 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.013840 s -26/04/01 05:56:12 INFO FileSourceStrategy: Pushed Filters: IsNotNull(c_custkey),IsNotNull(c_nationkey) -26/04/01 05:56:12 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(c_custkey#0L),isnotnull(c_nationkey#3L) -26/04/01 05:56:12 INFO FileSourceStrategy: Pushed Filters: IsNotNull(o_orderdate),GreaterThanOrEqual(o_orderdate,1994-01-01),LessThan(o_orderdate,1995-01-01),IsNotNull(o_custkey),IsNotNull(o_orderkey) -26/04/01 05:56:12 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(o_orderdate#60),(o_orderdate#60 >= 1994-01-01),(o_orderdate#60 < 1995-01-01),isnotnull(o_custkey#57L),isnotnull(o_orderkey#56L) -26/04/01 05:56:12 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_orderkey),IsNotNull(l_suppkey) -26/04/01 05:56:12 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_orderkey#16L),isnotnull(l_suppkey#18L) -26/04/01 05:56:12 INFO FileSourceStrategy: Pushed Filters: IsNotNull(s_suppkey),IsNotNull(s_nationkey) -26/04/01 05:56:12 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(s_suppkey#108L),isnotnull(s_nationkey#111L) -26/04/01 05:56:12 INFO FileSourceStrategy: Pushed Filters: IsNotNull(n_nationkey),IsNotNull(n_regionkey) -26/04/01 05:56:12 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(n_nationkey#48L),isnotnull(n_regionkey#50L),might_contain(scalar-subquery#128 [], xxhash64(n_regionkey#50L, 42)) -26/04/01 05:56:12 INFO FileSourceStrategy: Pushed Filters: IsNotNull(r_name),EqualTo(r_name,AFRICA),IsNotNull(r_regionkey) -26/04/01 05:56:12 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(r_name#103),(r_name#103 = AFRICA),isnotnull(r_regionkey#102L) -26/04/01 05:56:12 INFO FileSourceStrategy: Pushed Filters: IsNotNull(r_name),EqualTo(r_name,AFRICA),IsNotNull(r_regionkey) -26/04/01 05:56:12 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(r_name#103),(r_name#103 = AFRICA),isnotnull(r_regionkey#102L) -26/04/01 05:56:12 INFO CodeGenerator: Code generated in 78.90275 ms -26/04/01 05:56:12 INFO CodeGenerator: Code generated in 79.198875 ms -26/04/01 05:56:12 INFO CodeGenerator: Code generated in 78.94675 ms -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:58206 (size: 35.0 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:58206 (size: 35.0 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:56:12 INFO SparkContext: Created broadcast 10 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:58206 (size: 35.0 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 9 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:56:12 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 05:56:12 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 05:56:12 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 05:56:12 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO DAGScheduler: Registering RDD 28 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 -26/04/01 05:56:12 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 16 output partitions -26/04/01 05:56:12 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:12 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:56:12 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:58206 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 17.7 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:58206 (size: 7.9 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 11 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:12 INFO DAGScheduler: Submitting 16 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[28] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 8.0 with 16 tasks resource profile 0 -26/04/01 05:56:12 INFO CodeGenerator: Code generated in 10.717584 ms -26/04/01 05:56:12 INFO DAGScheduler: Registering RDD 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 -26/04/01 05:56:12 INFO DAGScheduler: Got map stage job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 05:56:12 INFO DAGScheduler: Final stage: ShuffleMapStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:12 INFO DAGScheduler: Submitting ShuffleMapStage 9 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:56:12 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:12 INFO TaskSetManager: Starting task 1.0 in stage 8.0 (TID 9) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:12 INFO TaskSetManager: Starting task 2.0 in stage 8.0 (TID 10) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 201.9 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO TaskSetManager: Starting task 3.0 in stage 8.0 (TID 11) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:12 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) -26/04/01 05:56:12 INFO Executor: Running task 1.0 in stage 8.0 (TID 9) -26/04/01 05:56:12 INFO Executor: Running task 2.0 in stage 8.0 (TID 10) -26/04/01 05:56:12 INFO Executor: Running task 3.0 in stage 8.0 (TID 11) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 35.2 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:58206 (size: 35.2 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 12 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:56:12 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_13 stored as values in memory (estimated size 31.4 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_13_piece0 stored as bytes in memory (estimated size 14.0 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_13_piece0 in memory on 10.0.0.133:58206 (size: 14.0 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 13 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 9 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 9.0 with 1 tasks resource profile 0 -26/04/01 05:56:12 INFO DAGScheduler: Got job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 05:56:12 INFO DAGScheduler: Final stage: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:12 INFO DAGScheduler: Submitting ResultStage 10 (MapPartitionsRDD[25] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_14 stored as values in memory (estimated size 15.6 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_14_piece0 stored as bytes in memory (estimated size 6.7 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_14_piece0 in memory on 10.0.0.133:58206 (size: 6.7 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 14 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:12 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 10 (MapPartitionsRDD[25] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 10.0 with 1 tasks resource profile 0 -26/04/01 05:56:12 INFO DAGScheduler: Registering RDD 32 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 2 -26/04/01 05:56:12 INFO DAGScheduler: Got map stage job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 64 output partitions -26/04/01 05:56:12 INFO DAGScheduler: Final stage: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:12 INFO DAGScheduler: Submitting ShuffleMapStage 11 (MapPartitionsRDD[32] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:56:12 INFO CodeGenerator: Code generated in 7.381917 ms -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_15 stored as values in memory (estimated size 202.1 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_16 stored as values in memory (estimated size 19.2 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_16_piece0 stored as bytes in memory (estimated size 8.3 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_16_piece0 in memory on 10.0.0.133:58206 (size: 8.3 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 16 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:12 INFO DAGScheduler: Submitting 64 missing tasks from ShuffleMapStage 11 (MapPartitionsRDD[32] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 11.0 with 64 tasks resource profile 0 -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_15_piece0 stored as bytes in memory (estimated size 35.2 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_15_piece0 in memory on 10.0.0.133:58206 (size: 35.2 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO CodeGenerator: Code generated in 4.947959 ms -26/04/01 05:56:12 INFO SparkContext: Created broadcast 15 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:56:12 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 05:56:12 INFO DAGScheduler: Registering RDD 36 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 3 -26/04/01 05:56:12 INFO DAGScheduler: Got map stage job 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions -26/04/01 05:56:12 INFO DAGScheduler: Final stage: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:12 INFO DAGScheduler: Submitting ShuffleMapStage 12 (MapPartitionsRDD[36] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_17 stored as values in memory (estimated size 19.2 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_17_piece0 stored as bytes in memory (estimated size 8.3 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_17_piece0 in memory on 10.0.0.133:58206 (size: 8.3 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 17 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:12 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 12 (MapPartitionsRDD[36] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 12.0 with 208 tasks resource profile 0 -26/04/01 05:56:12 INFO CodeGenerator: Code generated in 5.984708 ms -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_18 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO CodeGenerator: Code generated in 6.960333 ms -26/04/01 05:56:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.28.parquet, range: 0-43421251, partition values: [empty row] -26/04/01 05:56:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.32.parquet, range: 0-43435385, partition values: [empty row] -26/04/01 05:56:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.1.parquet, range: 0-43414283, partition values: [empty row] -26/04/01 05:56:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.24.parquet, range: 0-43412266, partition values: [empty row] -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_18_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_18_piece0 in memory on 10.0.0.133:58206 (size: 35.0 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 18 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:56:12 INFO FileSourceScanExec: Planning scan with bin packing, max size: 55920126 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 05:56:12 INFO DAGScheduler: Registering RDD 40 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 4 -26/04/01 05:56:12 INFO DAGScheduler: Got map stage job 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 4 output partitions -26/04/01 05:56:12 INFO DAGScheduler: Final stage: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:56:12 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:56:12 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:12 INFO DAGScheduler: Submitting ShuffleMapStage 13 (MapPartitionsRDD[40] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_19 stored as values in memory (estimated size 17.8 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO MemoryStore: Block broadcast_19_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 8.6 GiB) -26/04/01 05:56:12 INFO BlockManagerInfo: Added broadcast_19_piece0 in memory on 10.0.0.133:58206 (size: 7.9 KiB, free: 8.6 GiB) -26/04/01 05:56:12 INFO SparkContext: Created broadcast 19 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:12 INFO DAGScheduler: Submitting 4 missing tasks from ShuffleMapStage 13 (MapPartitionsRDD[40] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3)) -26/04/01 05:56:12 INFO TaskSchedulerImpl: Adding task set 13.0 with 4 tasks resource profile 0 -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 05:56:13 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 05:56:13 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 05:56:13 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.7.parquet, range: 0-43422275, partition values: [empty row] -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.26.parquet, range: 0-43415615, partition values: [empty row] -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.22.parquet, range: 0-43412913, partition values: [empty row] -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.15.parquet, range: 0-43410292, partition values: [empty row] -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO Executor: Finished task 3.0 in stage 8.0 (TID 11). 2294 bytes result sent to driver -26/04/01 05:56:13 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 2294 bytes result sent to driver -26/04/01 05:56:13 INFO Executor: Finished task 1.0 in stage 8.0 (TID 9). 2294 bytes result sent to driver -26/04/01 05:56:13 INFO Executor: Finished task 2.0 in stage 8.0 (TID 10). 2294 bytes result sent to driver -26/04/01 05:56:13 INFO TaskSetManager: Starting task 4.0 in stage 8.0 (TID 12) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:13 INFO Executor: Running task 4.0 in stage 8.0 (TID 12) -26/04/01 05:56:13 INFO TaskSetManager: Starting task 5.0 in stage 8.0 (TID 13) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:13 INFO TaskSetManager: Finished task 3.0 in stage 8.0 (TID 11) in 669 ms on 10.0.0.133 (executor driver) (1/16) -26/04/01 05:56:13 INFO TaskSetManager: Starting task 6.0 in stage 8.0 (TID 14) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:13 INFO Executor: Running task 5.0 in stage 8.0 (TID 13) -26/04/01 05:56:13 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 672 ms on 10.0.0.133 (executor driver) (2/16) -26/04/01 05:56:13 INFO Executor: Running task 6.0 in stage 8.0 (TID 14) -26/04/01 05:56:13 INFO TaskSetManager: Finished task 1.0 in stage 8.0 (TID 9) in 670 ms on 10.0.0.133 (executor driver) (3/16) -26/04/01 05:56:13 INFO TaskSetManager: Starting task 7.0 in stage 8.0 (TID 15) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:13 INFO Executor: Running task 7.0 in stage 8.0 (TID 15) -26/04/01 05:56:13 INFO TaskSetManager: Finished task 2.0 in stage 8.0 (TID 10) in 670 ms on 10.0.0.133 (executor driver) (4/16) -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.9.parquet, range: 0-43409639, partition values: [empty row] -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.18.parquet, range: 0-43409175, partition values: [empty row] -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.30.parquet, range: 0-43407009, partition values: [empty row] -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.11.parquet, range: 0-43405290, partition values: [empty row] -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.5.parquet, range: 0-43408011, partition values: [empty row] -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.31.parquet, range: 0-43405268, partition values: [empty row] -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.20.parquet, range: 0-43409316, partition values: [empty row] -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.8.parquet, range: 0-43406367, partition values: [empty row] -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO Executor: Finished task 5.0 in stage 8.0 (TID 13). 2294 bytes result sent to driver -26/04/01 05:56:13 INFO Executor: Finished task 7.0 in stage 8.0 (TID 15). 2294 bytes result sent to driver -26/04/01 05:56:13 INFO TaskSetManager: Starting task 8.0 in stage 8.0 (TID 16) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:13 INFO Executor: Running task 8.0 in stage 8.0 (TID 16) -26/04/01 05:56:13 INFO TaskSetManager: Starting task 9.0 in stage 8.0 (TID 17) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:13 INFO TaskSetManager: Finished task 5.0 in stage 8.0 (TID 13) in 328 ms on 10.0.0.133 (executor driver) (5/16) -26/04/01 05:56:13 INFO TaskSetManager: Finished task 7.0 in stage 8.0 (TID 15) in 327 ms on 10.0.0.133 (executor driver) (6/16) -26/04/01 05:56:13 INFO Executor: Running task 9.0 in stage 8.0 (TID 17) -26/04/01 05:56:13 INFO Executor: Finished task 4.0 in stage 8.0 (TID 12). 2294 bytes result sent to driver -26/04/01 05:56:13 INFO TaskSetManager: Starting task 10.0 in stage 8.0 (TID 18) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:13 INFO Executor: Running task 10.0 in stage 8.0 (TID 18) -26/04/01 05:56:13 INFO TaskSetManager: Finished task 4.0 in stage 8.0 (TID 12) in 331 ms on 10.0.0.133 (executor driver) (7/16) -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.2.parquet, range: 0-43405025, partition values: [empty row] -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.19.parquet, range: 0-43404330, partition values: [empty row] -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.13.parquet, range: 0-43401020, partition values: [empty row] -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO Executor: Finished task 6.0 in stage 8.0 (TID 14). 2294 bytes result sent to driver -26/04/01 05:56:13 INFO TaskSetManager: Starting task 11.0 in stage 8.0 (TID 19) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:13 INFO TaskSetManager: Finished task 6.0 in stage 8.0 (TID 14) in 337 ms on 10.0.0.133 (executor driver) (8/16) -26/04/01 05:56:13 INFO Executor: Running task 11.0 in stage 8.0 (TID 19) -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.4.parquet, range: 0-43397542, partition values: [empty row] -26/04/01 05:56:13 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.23.parquet, range: 0-43398227, partition values: [empty row] -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.10.parquet, range: 0-43404438, partition values: [empty row] -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.6.parquet, range: 0-43402993, partition values: [empty row] -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.21.parquet, range: 0-43396532, partition values: [empty row] -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:14 INFO Executor: Finished task 10.0 in stage 8.0 (TID 18). 2294 bytes result sent to driver -26/04/01 05:56:14 INFO Executor: Finished task 8.0 in stage 8.0 (TID 16). 2294 bytes result sent to driver -26/04/01 05:56:14 INFO TaskSetManager: Starting task 12.0 in stage 8.0 (TID 20) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:14 INFO TaskSetManager: Finished task 10.0 in stage 8.0 (TID 18) in 230 ms on 10.0.0.133 (executor driver) (9/16) -26/04/01 05:56:14 INFO Executor: Running task 12.0 in stage 8.0 (TID 20) -26/04/01 05:56:14 INFO TaskSetManager: Starting task 13.0 in stage 8.0 (TID 21) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.12.parquet, range: 0-43396359, partition values: [empty row] -26/04/01 05:56:14 INFO TaskSetManager: Finished task 8.0 in stage 8.0 (TID 16) in 236 ms on 10.0.0.133 (executor driver) (10/16) -26/04/01 05:56:14 INFO Executor: Running task 13.0 in stage 8.0 (TID 21) -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.29.parquet, range: 0-43391813, partition values: [empty row] -26/04/01 05:56:14 INFO Executor: Finished task 9.0 in stage 8.0 (TID 17). 2294 bytes result sent to driver -26/04/01 05:56:14 INFO TaskSetManager: Starting task 14.0 in stage 8.0 (TID 22) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:14 INFO TaskSetManager: Finished task 9.0 in stage 8.0 (TID 17) in 238 ms on 10.0.0.133 (executor driver) (11/16) -26/04/01 05:56:14 INFO Executor: Running task 14.0 in stage 8.0 (TID 22) -26/04/01 05:56:14 INFO Executor: Finished task 11.0 in stage 8.0 (TID 19). 2251 bytes result sent to driver -26/04/01 05:56:14 INFO TaskSetManager: Starting task 15.0 in stage 8.0 (TID 23) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.16.parquet, range: 0-43385525, partition values: [empty row] -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:14 INFO TaskSetManager: Finished task 11.0 in stage 8.0 (TID 19) in 231 ms on 10.0.0.133 (executor driver) (12/16) -26/04/01 05:56:14 INFO Executor: Running task 15.0 in stage 8.0 (TID 23) -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.25.parquet, range: 0-43376403, partition values: [empty row] -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.17.parquet, range: 0-43381052, partition values: [empty row] -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.3.parquet, range: 0-43385934, partition values: [empty row] -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.27.parquet, range: 0-43393057, partition values: [empty row] -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/customer/customer.14.parquet, range: 0-43368555, partition values: [empty row] -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(noteq(c_custkey, null), noteq(c_nationkey, null)) -26/04/01 05:56:14 INFO Executor: Finished task 12.0 in stage 8.0 (TID 20). 2251 bytes result sent to driver -26/04/01 05:56:14 INFO TaskSetManager: Starting task 0.0 in stage 9.0 (TID 24) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:14 INFO Executor: Running task 0.0 in stage 9.0 (TID 24) -26/04/01 05:56:14 INFO TaskSetManager: Finished task 12.0 in stage 8.0 (TID 20) in 217 ms on 10.0.0.133 (executor driver) (13/16) -26/04/01 05:56:14 INFO Executor: Finished task 15.0 in stage 8.0 (TID 23). 2294 bytes result sent to driver -26/04/01 05:56:14 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 25) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9608 bytes) -26/04/01 05:56:14 INFO Executor: Finished task 13.0 in stage 8.0 (TID 21). 2251 bytes result sent to driver -26/04/01 05:56:14 INFO Executor: Running task 0.0 in stage 10.0 (TID 25) -26/04/01 05:56:14 INFO TaskSetManager: Starting task 0.0 in stage 11.0 (TID 26) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:14 INFO TaskSetManager: Finished task 15.0 in stage 8.0 (TID 23) in 214 ms on 10.0.0.133 (executor driver) (14/16) -26/04/01 05:56:14 INFO Executor: Running task 0.0 in stage 11.0 (TID 26) -26/04/01 05:56:14 INFO TaskSetManager: Finished task 13.0 in stage 8.0 (TID 21) in 222 ms on 10.0.0.133 (executor driver) (15/16) -26/04/01 05:56:14 INFO Executor: Finished task 14.0 in stage 8.0 (TID 22). 2294 bytes result sent to driver -26/04/01 05:56:14 INFO TaskSetManager: Starting task 1.0 in stage 11.0 (TID 27) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:14 INFO Executor: Running task 1.0 in stage 11.0 (TID 27) -26/04/01 05:56:14 INFO TaskSetManager: Finished task 14.0 in stage 8.0 (TID 22) in 218 ms on 10.0.0.133 (executor driver) (16/16) -26/04/01 05:56:14 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool -26/04/01 05:56:14 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 1.464 s -26/04/01 05:56:14 INFO DAGScheduler: looking for newly runnable stages -26/04/01 05:56:14 INFO DAGScheduler: running: Set(ShuffleMapStage 12, ShuffleMapStage 9, ShuffleMapStage 13, ResultStage 10, ShuffleMapStage 11) -26/04/01 05:56:14 INFO CodeGenerator: Code generated in 5.694291 ms -26/04/01 05:56:14 INFO DAGScheduler: waiting: Set() -26/04/01 05:56:14 INFO DAGScheduler: failed: Set() -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/region/region.1.parquet, range: 0-1227, partition values: [empty row] -26/04/01 05:56:14 INFO CodeGenerator: Code generated in 6.989792 ms -26/04/01 05:56:14 INFO CodeGenerator: Code generated in 5.905167 ms -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/region/region.1.parquet, range: 0-1227, partition values: [empty row] -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(r_name, null), eq(r_name, Binary{"AFRICA"})), noteq(r_regionkey, null)) -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(r_name, null), eq(r_name, Binary{"AFRICA"})), noteq(r_regionkey, null)) -26/04/01 05:56:14 INFO CodeGenerator: Code generated in 4.1855 ms -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.7.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.13.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:14 INFO Executor: Finished task 0.0 in stage 10.0 (TID 25). 1848 bytes result sent to driver -26/04/01 05:56:14 INFO TaskSetManager: Starting task 2.0 in stage 11.0 (TID 28) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:14 INFO Executor: Running task 2.0 in stage 11.0 (TID 28) -26/04/01 05:56:14 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 25) in 21 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:56:14 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool -26/04/01 05:56:14 INFO DAGScheduler: ResultStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 1.455 s -26/04/01 05:56:14 INFO DAGScheduler: Job 10 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:56:14 INFO TaskSchedulerImpl: Killing all running tasks in stage 10: Stage finished -26/04/01 05:56:14 INFO DAGScheduler: Job 10 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 1.479527 s -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.25.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:14 INFO CodeGenerator: Code generated in 2.792583 ms -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:14 INFO CodeGenerator: Code generated in 2.404583 ms -26/04/01 05:56:14 INFO MemoryStore: Block broadcast_20 stored as values in memory (estimated size 1024.0 KiB, free 8.6 GiB) -26/04/01 05:56:14 INFO MemoryStore: Block broadcast_20_piece0 stored as bytes in memory (estimated size 185.0 B, free 8.6 GiB) -26/04/01 05:56:14 INFO BlockManagerInfo: Added broadcast_20_piece0 in memory on 10.0.0.133:58206 (size: 185.0 B, free: 8.6 GiB) -26/04/01 05:56:14 INFO SparkContext: Created broadcast 20 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:56:14 INFO CodeGenerator: Code generated in 2.952166 ms -26/04/01 05:56:14 INFO CodeGenerator: Code generated in 4.225375 ms -26/04/01 05:56:14 INFO Executor: Finished task 0.0 in stage 9.0 (TID 24). 2580 bytes result sent to driver -26/04/01 05:56:14 INFO TaskSetManager: Starting task 3.0 in stage 11.0 (TID 29) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:14 INFO Executor: Running task 3.0 in stage 11.0 (TID 29) -26/04/01 05:56:14 INFO TaskSetManager: Finished task 0.0 in stage 9.0 (TID 24) in 56 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:56:14 INFO TaskSchedulerImpl: Removed TaskSet 9.0, whose tasks have all completed, from pool -26/04/01 05:56:14 INFO DAGScheduler: ShuffleMapStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 1.502 s -26/04/01 05:56:14 INFO DAGScheduler: looking for newly runnable stages -26/04/01 05:56:14 INFO DAGScheduler: running: Set(ShuffleMapStage 12, ShuffleMapStage 13, ShuffleMapStage 11) -26/04/01 05:56:14 INFO DAGScheduler: waiting: Set() -26/04/01 05:56:14 INFO DAGScheduler: failed: Set() -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.24.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:14 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:56:14 INFO DAGScheduler: Got job 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 05:56:14 INFO DAGScheduler: Final stage: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:56:14 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 14) -26/04/01 05:56:14 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:14 INFO DAGScheduler: Submitting ResultStage 15 (MapPartitionsRDD[43] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:56:14 INFO MemoryStore: Block broadcast_21 stored as values in memory (estimated size 32.0 KiB, free 8.6 GiB) -26/04/01 05:56:14 INFO MemoryStore: Block broadcast_21_piece0 stored as bytes in memory (estimated size 14.9 KiB, free 8.6 GiB) -26/04/01 05:56:14 INFO BlockManagerInfo: Added broadcast_21_piece0 in memory on 10.0.0.133:58206 (size: 14.9 KiB, free: 8.6 GiB) -26/04/01 05:56:14 INFO SparkContext: Created broadcast 21 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:14 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 15 (MapPartitionsRDD[43] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:56:14 INFO TaskSchedulerImpl: Adding task set 15.0 with 1 tasks resource profile 0 -26/04/01 05:56:14 INFO Executor: Finished task 3.0 in stage 11.0 (TID 29). 2294 bytes result sent to driver -26/04/01 05:56:14 INFO TaskSetManager: Starting task 4.0 in stage 11.0 (TID 30) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:14 INFO TaskSetManager: Finished task 3.0 in stage 11.0 (TID 29) in 305 ms on 10.0.0.133 (executor driver) (1/64) -26/04/01 05:56:14 INFO Executor: Running task 4.0 in stage 11.0 (TID 30) -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.12.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:14 INFO Executor: Finished task 2.0 in stage 11.0 (TID 28). 2251 bytes result sent to driver -26/04/01 05:56:14 INFO TaskSetManager: Starting task 5.0 in stage 11.0 (TID 31) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:14 INFO TaskSetManager: Finished task 2.0 in stage 11.0 (TID 28) in 338 ms on 10.0.0.133 (executor driver) (2/64) -26/04/01 05:56:14 INFO Executor: Running task 5.0 in stage 11.0 (TID 31) -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.6.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:14 INFO Executor: Finished task 1.0 in stage 11.0 (TID 27). 2251 bytes result sent to driver -26/04/01 05:56:14 INFO TaskSetManager: Starting task 6.0 in stage 11.0 (TID 32) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:14 INFO Executor: Running task 6.0 in stage 11.0 (TID 32) -26/04/01 05:56:14 INFO TaskSetManager: Finished task 1.0 in stage 11.0 (TID 27) in 361 ms on 10.0.0.133 (executor driver) (3/64) -26/04/01 05:56:14 INFO Executor: Finished task 0.0 in stage 11.0 (TID 26). 2251 bytes result sent to driver -26/04/01 05:56:14 INFO TaskSetManager: Starting task 7.0 in stage 11.0 (TID 33) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:14 INFO TaskSetManager: Finished task 0.0 in stage 11.0 (TID 26) in 364 ms on 10.0.0.133 (executor driver) (4/64) -26/04/01 05:56:14 INFO Executor: Running task 7.0 in stage 11.0 (TID 33) -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.19.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.26.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:14 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:14 INFO Executor: Finished task 4.0 in stage 11.0 (TID 30). 2251 bytes result sent to driver -26/04/01 05:56:14 INFO TaskSetManager: Starting task 8.0 in stage 11.0 (TID 34) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:15 INFO TaskSetManager: Finished task 4.0 in stage 11.0 (TID 30) in 260 ms on 10.0.0.133 (executor driver) (5/64) -26/04/01 05:56:15 INFO Executor: Running task 8.0 in stage 11.0 (TID 34) -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.4.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO Executor: Finished task 6.0 in stage 11.0 (TID 32). 2251 bytes result sent to driver -26/04/01 05:56:15 INFO TaskSetManager: Starting task 9.0 in stage 11.0 (TID 35) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:15 INFO Executor: Running task 9.0 in stage 11.0 (TID 35) -26/04/01 05:56:15 INFO TaskSetManager: Finished task 6.0 in stage 11.0 (TID 32) in 255 ms on 10.0.0.133 (executor driver) (6/64) -26/04/01 05:56:15 INFO Executor: Finished task 5.0 in stage 11.0 (TID 31). 2251 bytes result sent to driver -26/04/01 05:56:15 INFO TaskSetManager: Starting task 10.0 in stage 11.0 (TID 36) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:15 INFO TaskSetManager: Finished task 5.0 in stage 11.0 (TID 31) in 261 ms on 10.0.0.133 (executor driver) (7/64) -26/04/01 05:56:15 INFO Executor: Finished task 7.0 in stage 11.0 (TID 33). 2251 bytes result sent to driver -26/04/01 05:56:15 INFO Executor: Running task 10.0 in stage 11.0 (TID 36) -26/04/01 05:56:15 INFO TaskSetManager: Starting task 11.0 in stage 11.0 (TID 37) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.10.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO TaskSetManager: Finished task 7.0 in stage 11.0 (TID 33) in 256 ms on 10.0.0.133 (executor driver) (8/64) -26/04/01 05:56:15 INFO Executor: Running task 11.0 in stage 11.0 (TID 37) -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.5.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.11.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO Executor: Finished task 8.0 in stage 11.0 (TID 34). 2294 bytes result sent to driver -26/04/01 05:56:15 INFO TaskSetManager: Starting task 12.0 in stage 11.0 (TID 38) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:15 INFO TaskSetManager: Finished task 8.0 in stage 11.0 (TID 34) in 251 ms on 10.0.0.133 (executor driver) (9/64) -26/04/01 05:56:15 INFO Executor: Running task 12.0 in stage 11.0 (TID 38) -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.18.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO Executor: Finished task 10.0 in stage 11.0 (TID 36). 2251 bytes result sent to driver -26/04/01 05:56:15 INFO TaskSetManager: Starting task 13.0 in stage 11.0 (TID 39) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:15 INFO TaskSetManager: Finished task 10.0 in stage 11.0 (TID 36) in 258 ms on 10.0.0.133 (executor driver) (10/64) -26/04/01 05:56:15 INFO Executor: Running task 13.0 in stage 11.0 (TID 39) -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.27.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO Executor: Finished task 11.0 in stage 11.0 (TID 37). 2251 bytes result sent to driver -26/04/01 05:56:15 INFO TaskSetManager: Starting task 14.0 in stage 11.0 (TID 40) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:15 INFO Executor: Finished task 9.0 in stage 11.0 (TID 35). 2251 bytes result sent to driver -26/04/01 05:56:15 INFO TaskSetManager: Finished task 11.0 in stage 11.0 (TID 37) in 261 ms on 10.0.0.133 (executor driver) (11/64) -26/04/01 05:56:15 INFO Executor: Running task 14.0 in stage 11.0 (TID 40) -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO TaskSetManager: Starting task 15.0 in stage 11.0 (TID 41) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:15 INFO Executor: Running task 15.0 in stage 11.0 (TID 41) -26/04/01 05:56:15 INFO TaskSetManager: Finished task 9.0 in stage 11.0 (TID 35) in 264 ms on 10.0.0.133 (executor driver) (12/64) -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.32.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.22.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO Executor: Finished task 12.0 in stage 11.0 (TID 38). 2251 bytes result sent to driver -26/04/01 05:56:15 INFO TaskSetManager: Starting task 16.0 in stage 11.0 (TID 42) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:15 INFO TaskSetManager: Finished task 12.0 in stage 11.0 (TID 38) in 248 ms on 10.0.0.133 (executor driver) (13/64) -26/04/01 05:56:15 INFO Executor: Running task 16.0 in stage 11.0 (TID 42) -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.9.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO Executor: Finished task 13.0 in stage 11.0 (TID 39). 2251 bytes result sent to driver -26/04/01 05:56:15 INFO TaskSetManager: Starting task 17.0 in stage 11.0 (TID 43) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:15 INFO TaskSetManager: Finished task 13.0 in stage 11.0 (TID 39) in 251 ms on 10.0.0.133 (executor driver) (14/64) -26/04/01 05:56:15 INFO Executor: Running task 17.0 in stage 11.0 (TID 43) -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.14.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO Executor: Finished task 14.0 in stage 11.0 (TID 40). 2251 bytes result sent to driver -26/04/01 05:56:15 INFO TaskSetManager: Starting task 18.0 in stage 11.0 (TID 44) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:15 INFO TaskSetManager: Finished task 14.0 in stage 11.0 (TID 40) in 251 ms on 10.0.0.133 (executor driver) (15/64) -26/04/01 05:56:15 INFO Executor: Finished task 15.0 in stage 11.0 (TID 41). 2251 bytes result sent to driver -26/04/01 05:56:15 INFO Executor: Running task 18.0 in stage 11.0 (TID 44) -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO TaskSetManager: Starting task 19.0 in stage 11.0 (TID 45) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:15 INFO TaskSetManager: Finished task 15.0 in stage 11.0 (TID 41) in 250 ms on 10.0.0.133 (executor driver) (16/64) -26/04/01 05:56:15 INFO Executor: Running task 19.0 in stage 11.0 (TID 45) -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.1.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.15.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO Executor: Finished task 16.0 in stage 11.0 (TID 42). 2251 bytes result sent to driver -26/04/01 05:56:15 INFO TaskSetManager: Starting task 20.0 in stage 11.0 (TID 46) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:15 INFO Executor: Running task 20.0 in stage 11.0 (TID 46) -26/04/01 05:56:15 INFO TaskSetManager: Finished task 16.0 in stage 11.0 (TID 42) in 256 ms on 10.0.0.133 (executor driver) (17/64) -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.23.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO Executor: Finished task 17.0 in stage 11.0 (TID 43). 2251 bytes result sent to driver -26/04/01 05:56:15 INFO TaskSetManager: Starting task 21.0 in stage 11.0 (TID 47) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:15 INFO TaskSetManager: Finished task 17.0 in stage 11.0 (TID 43) in 262 ms on 10.0.0.133 (executor driver) (18/64) -26/04/01 05:56:15 INFO Executor: Running task 21.0 in stage 11.0 (TID 47) -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.8.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO Executor: Finished task 19.0 in stage 11.0 (TID 45). 2251 bytes result sent to driver -26/04/01 05:56:15 INFO TaskSetManager: Starting task 22.0 in stage 11.0 (TID 48) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:15 INFO Executor: Running task 22.0 in stage 11.0 (TID 48) -26/04/01 05:56:15 INFO TaskSetManager: Finished task 19.0 in stage 11.0 (TID 45) in 262 ms on 10.0.0.133 (executor driver) (19/64) -26/04/01 05:56:15 INFO Executor: Finished task 18.0 in stage 11.0 (TID 44). 2251 bytes result sent to driver -26/04/01 05:56:15 INFO TaskSetManager: Starting task 23.0 in stage 11.0 (TID 49) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:15 INFO TaskSetManager: Finished task 18.0 in stage 11.0 (TID 44) in 262 ms on 10.0.0.133 (executor driver) (20/64) -26/04/01 05:56:15 INFO Executor: Running task 23.0 in stage 11.0 (TID 49) -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.28.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.17.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:15 INFO Executor: Finished task 20.0 in stage 11.0 (TID 46). 2251 bytes result sent to driver -26/04/01 05:56:15 INFO TaskSetManager: Starting task 24.0 in stage 11.0 (TID 50) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:15 INFO TaskSetManager: Finished task 20.0 in stage 11.0 (TID 46) in 222 ms on 10.0.0.133 (executor driver) (21/64) -26/04/01 05:56:15 INFO Executor: Running task 24.0 in stage 11.0 (TID 50) -26/04/01 05:56:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.3.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:15 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO Executor: Finished task 22.0 in stage 11.0 (TID 48). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO TaskSetManager: Starting task 25.0 in stage 11.0 (TID 51) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:16 INFO Executor: Finished task 21.0 in stage 11.0 (TID 47). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO Executor: Running task 25.0 in stage 11.0 (TID 51) -26/04/01 05:56:16 INFO TaskSetManager: Starting task 26.0 in stage 11.0 (TID 52) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 22.0 in stage 11.0 (TID 48) in 238 ms on 10.0.0.133 (executor driver) (22/64) -26/04/01 05:56:16 INFO Executor: Running task 26.0 in stage 11.0 (TID 52) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 21.0 in stage 11.0 (TID 47) in 241 ms on 10.0.0.133 (executor driver) (23/64) -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.21.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.31.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:16 INFO Executor: Finished task 23.0 in stage 11.0 (TID 49). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO TaskSetManager: Starting task 27.0 in stage 11.0 (TID 53) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 23.0 in stage 11.0 (TID 49) in 239 ms on 10.0.0.133 (executor driver) (24/64) -26/04/01 05:56:16 INFO Executor: Running task 27.0 in stage 11.0 (TID 53) -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.20.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO Executor: Finished task 24.0 in stage 11.0 (TID 50). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO TaskSetManager: Starting task 28.0 in stage 11.0 (TID 54) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 24.0 in stage 11.0 (TID 50) in 229 ms on 10.0.0.133 (executor driver) (25/64) -26/04/01 05:56:16 INFO Executor: Running task 28.0 in stage 11.0 (TID 54) -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.30.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO Executor: Finished task 26.0 in stage 11.0 (TID 52). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO Executor: Finished task 27.0 in stage 11.0 (TID 53). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO TaskSetManager: Starting task 29.0 in stage 11.0 (TID 55) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 26.0 in stage 11.0 (TID 52) in 248 ms on 10.0.0.133 (executor driver) (26/64) -26/04/01 05:56:16 INFO Executor: Running task 29.0 in stage 11.0 (TID 55) -26/04/01 05:56:16 INFO TaskSetManager: Starting task 30.0 in stage 11.0 (TID 56) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 27.0 in stage 11.0 (TID 53) in 247 ms on 10.0.0.133 (executor driver) (27/64) -26/04/01 05:56:16 INFO Executor: Finished task 25.0 in stage 11.0 (TID 51). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO Executor: Running task 30.0 in stage 11.0 (TID 56) -26/04/01 05:56:16 INFO TaskSetManager: Starting task 31.0 in stage 11.0 (TID 57) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 25.0 in stage 11.0 (TID 51) in 249 ms on 10.0.0.133 (executor driver) (28/64) -26/04/01 05:56:16 INFO Executor: Running task 31.0 in stage 11.0 (TID 57) -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.29.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.16.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.2.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO Executor: Finished task 28.0 in stage 11.0 (TID 54). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO TaskSetManager: Starting task 32.0 in stage 11.0 (TID 58) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:16 INFO Executor: Running task 32.0 in stage 11.0 (TID 58) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 28.0 in stage 11.0 (TID 54) in 223 ms on 10.0.0.133 (executor driver) (29/64) -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.11.parquet, range: 134217728-233581632, partition values: [empty row] -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO Executor: Finished task 31.0 in stage 11.0 (TID 57). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO TaskSetManager: Starting task 33.0 in stage 11.0 (TID 59) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 31.0 in stage 11.0 (TID 57) in 250 ms on 10.0.0.133 (executor driver) (30/64) -26/04/01 05:56:16 INFO Executor: Running task 33.0 in stage 11.0 (TID 59) -26/04/01 05:56:16 INFO Executor: Finished task 30.0 in stage 11.0 (TID 56). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO TaskSetManager: Starting task 34.0 in stage 11.0 (TID 60) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.23.parquet, range: 134217728-233580600, partition values: [empty row] -26/04/01 05:56:16 INFO TaskSetManager: Finished task 30.0 in stage 11.0 (TID 56) in 252 ms on 10.0.0.133 (executor driver) (31/64) -26/04/01 05:56:16 INFO Executor: Running task 34.0 in stage 11.0 (TID 60) -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.30.parquet, range: 134217728-233577812, partition values: [empty row] -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO Executor: Finished task 29.0 in stage 11.0 (TID 55). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO TaskSetManager: Starting task 35.0 in stage 11.0 (TID 61) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 29.0 in stage 11.0 (TID 55) in 255 ms on 10.0.0.133 (executor driver) (32/64) -26/04/01 05:56:16 INFO Executor: Running task 35.0 in stage 11.0 (TID 61) -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.7.parquet, range: 134217728-233566955, partition values: [empty row] -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO Executor: Finished task 32.0 in stage 11.0 (TID 58). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO TaskSetManager: Starting task 36.0 in stage 11.0 (TID 62) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 32.0 in stage 11.0 (TID 58) in 169 ms on 10.0.0.133 (executor driver) (33/64) -26/04/01 05:56:16 INFO Executor: Running task 36.0 in stage 11.0 (TID 62) -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.12.parquet, range: 134217728-233565571, partition values: [empty row] -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO Executor: Finished task 34.0 in stage 11.0 (TID 60). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO TaskSetManager: Starting task 37.0 in stage 11.0 (TID 63) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:16 INFO Executor: Running task 37.0 in stage 11.0 (TID 63) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 34.0 in stage 11.0 (TID 60) in 195 ms on 10.0.0.133 (executor driver) (34/64) -26/04/01 05:56:16 INFO Executor: Finished task 33.0 in stage 11.0 (TID 59). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO TaskSetManager: Starting task 38.0 in stage 11.0 (TID 64) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:16 INFO Executor: Running task 38.0 in stage 11.0 (TID 64) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 33.0 in stage 11.0 (TID 59) in 199 ms on 10.0.0.133 (executor driver) (35/64) -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.16.parquet, range: 134217728-233564116, partition values: [empty row] -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.32.parquet, range: 134217728-233563384, partition values: [empty row] -26/04/01 05:56:16 INFO Executor: Finished task 35.0 in stage 11.0 (TID 61). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO TaskSetManager: Starting task 39.0 in stage 11.0 (TID 65) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:16 INFO Executor: Running task 39.0 in stage 11.0 (TID 65) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 35.0 in stage 11.0 (TID 61) in 196 ms on 10.0.0.133 (executor driver) (36/64) -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.14.parquet, range: 134217728-233562716, partition values: [empty row] -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO Executor: Finished task 36.0 in stage 11.0 (TID 62). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO TaskSetManager: Starting task 40.0 in stage 11.0 (TID 66) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 36.0 in stage 11.0 (TID 62) in 175 ms on 10.0.0.133 (executor driver) (37/64) -26/04/01 05:56:16 INFO Executor: Running task 40.0 in stage 11.0 (TID 66) -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.5.parquet, range: 134217728-233556822, partition values: [empty row] -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO Executor: Finished task 37.0 in stage 11.0 (TID 63). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO TaskSetManager: Starting task 41.0 in stage 11.0 (TID 67) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 37.0 in stage 11.0 (TID 63) in 203 ms on 10.0.0.133 (executor driver) (38/64) -26/04/01 05:56:16 INFO Executor: Running task 41.0 in stage 11.0 (TID 67) -26/04/01 05:56:16 INFO Executor: Finished task 39.0 in stage 11.0 (TID 65). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO TaskSetManager: Starting task 42.0 in stage 11.0 (TID 68) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:16 INFO Executor: Finished task 38.0 in stage 11.0 (TID 64). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO Executor: Running task 42.0 in stage 11.0 (TID 68) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 39.0 in stage 11.0 (TID 65) in 200 ms on 10.0.0.133 (executor driver) (39/64) -26/04/01 05:56:16 INFO TaskSetManager: Starting task 43.0 in stage 11.0 (TID 69) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 38.0 in stage 11.0 (TID 64) in 203 ms on 10.0.0.133 (executor driver) (40/64) -26/04/01 05:56:16 INFO Executor: Running task 43.0 in stage 11.0 (TID 69) -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.27.parquet, range: 134217728-233556807, partition values: [empty row] -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.19.parquet, range: 134217728-233556337, partition values: [empty row] -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.1.parquet, range: 134217728-233553544, partition values: [empty row] -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:16 INFO Executor: Finished task 40.0 in stage 11.0 (TID 66). 2251 bytes result sent to driver -26/04/01 05:56:16 INFO TaskSetManager: Starting task 44.0 in stage 11.0 (TID 70) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:16 INFO Executor: Running task 44.0 in stage 11.0 (TID 70) -26/04/01 05:56:16 INFO TaskSetManager: Finished task 40.0 in stage 11.0 (TID 66) in 185 ms on 10.0.0.133 (executor driver) (41/64) -26/04/01 05:56:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.6.parquet, range: 134217728-233549987, partition values: [empty row] -26/04/01 05:56:16 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO Executor: Finished task 43.0 in stage 11.0 (TID 69). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO TaskSetManager: Starting task 45.0 in stage 11.0 (TID 71) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:17 INFO Executor: Running task 45.0 in stage 11.0 (TID 71) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 43.0 in stage 11.0 (TID 69) in 196 ms on 10.0.0.133 (executor driver) (42/64) -26/04/01 05:56:17 INFO Executor: Finished task 42.0 in stage 11.0 (TID 68). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO TaskSetManager: Starting task 46.0 in stage 11.0 (TID 72) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 42.0 in stage 11.0 (TID 68) in 198 ms on 10.0.0.133 (executor driver) (43/64) -26/04/01 05:56:17 INFO Executor: Running task 46.0 in stage 11.0 (TID 72) -26/04/01 05:56:17 INFO Executor: Finished task 41.0 in stage 11.0 (TID 67). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO TaskSetManager: Starting task 47.0 in stage 11.0 (TID 73) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 41.0 in stage 11.0 (TID 67) in 200 ms on 10.0.0.133 (executor driver) (44/64) -26/04/01 05:56:17 INFO Executor: Running task 47.0 in stage 11.0 (TID 73) -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.21.parquet, range: 134217728-233548593, partition values: [empty row] -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.28.parquet, range: 134217728-233545450, partition values: [empty row] -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.4.parquet, range: 134217728-233545407, partition values: [empty row] -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO Executor: Finished task 44.0 in stage 11.0 (TID 70). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO TaskSetManager: Starting task 48.0 in stage 11.0 (TID 74) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 44.0 in stage 11.0 (TID 70) in 177 ms on 10.0.0.133 (executor driver) (45/64) -26/04/01 05:56:17 INFO Executor: Running task 48.0 in stage 11.0 (TID 74) -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.29.parquet, range: 134217728-233542751, partition values: [empty row] -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO Executor: Finished task 46.0 in stage 11.0 (TID 72). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO TaskSetManager: Starting task 49.0 in stage 11.0 (TID 75) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 46.0 in stage 11.0 (TID 72) in 202 ms on 10.0.0.133 (executor driver) (46/64) -26/04/01 05:56:17 INFO Executor: Running task 49.0 in stage 11.0 (TID 75) -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.24.parquet, range: 134217728-233541467, partition values: [empty row] -26/04/01 05:56:17 INFO Executor: Finished task 45.0 in stage 11.0 (TID 71). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO TaskSetManager: Starting task 50.0 in stage 11.0 (TID 76) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:17 INFO Executor: Running task 50.0 in stage 11.0 (TID 76) -26/04/01 05:56:17 INFO Executor: Finished task 48.0 in stage 11.0 (TID 74). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO TaskSetManager: Finished task 45.0 in stage 11.0 (TID 71) in 206 ms on 10.0.0.133 (executor driver) (47/64) -26/04/01 05:56:17 INFO TaskSetManager: Starting task 51.0 in stage 11.0 (TID 77) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 48.0 in stage 11.0 (TID 74) in 187 ms on 10.0.0.133 (executor driver) (48/64) -26/04/01 05:56:17 INFO Executor: Running task 51.0 in stage 11.0 (TID 77) -26/04/01 05:56:17 INFO Executor: Finished task 47.0 in stage 11.0 (TID 73). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO TaskSetManager: Starting task 52.0 in stage 11.0 (TID 78) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 47.0 in stage 11.0 (TID 73) in 206 ms on 10.0.0.133 (executor driver) (49/64) -26/04/01 05:56:17 INFO Executor: Running task 52.0 in stage 11.0 (TID 78) -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.20.parquet, range: 134217728-233539473, partition values: [empty row] -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.26.parquet, range: 134217728-233538804, partition values: [empty row] -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.10.parquet, range: 134217728-233538598, partition values: [empty row] -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO Executor: Finished task 50.0 in stage 11.0 (TID 76). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO Executor: Finished task 51.0 in stage 11.0 (TID 77). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO TaskSetManager: Starting task 53.0 in stage 11.0 (TID 79) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:17 INFO Executor: Running task 53.0 in stage 11.0 (TID 79) -26/04/01 05:56:17 INFO TaskSetManager: Starting task 54.0 in stage 11.0 (TID 80) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:17 INFO Executor: Finished task 49.0 in stage 11.0 (TID 75). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO Executor: Running task 54.0 in stage 11.0 (TID 80) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 50.0 in stage 11.0 (TID 76) in 225 ms on 10.0.0.133 (executor driver) (50/64) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 51.0 in stage 11.0 (TID 77) in 226 ms on 10.0.0.133 (executor driver) (51/64) -26/04/01 05:56:17 INFO TaskSetManager: Starting task 55.0 in stage 11.0 (TID 81) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 49.0 in stage 11.0 (TID 75) in 229 ms on 10.0.0.133 (executor driver) (52/64) -26/04/01 05:56:17 INFO Executor: Running task 55.0 in stage 11.0 (TID 81) -26/04/01 05:56:17 INFO Executor: Finished task 52.0 in stage 11.0 (TID 78). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO TaskSetManager: Starting task 56.0 in stage 11.0 (TID 82) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.13.parquet, range: 134217728-233537986, partition values: [empty row] -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.22.parquet, range: 134217728-233536356, partition values: [empty row] -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.8.parquet, range: 134217728-233536463, partition values: [empty row] -26/04/01 05:56:17 INFO TaskSetManager: Finished task 52.0 in stage 11.0 (TID 78) in 226 ms on 10.0.0.133 (executor driver) (53/64) -26/04/01 05:56:17 INFO Executor: Running task 56.0 in stage 11.0 (TID 82) -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.9.parquet, range: 134217728-233535871, partition values: [empty row] -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO Executor: Finished task 55.0 in stage 11.0 (TID 81). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO Executor: Finished task 54.0 in stage 11.0 (TID 80). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO TaskSetManager: Starting task 57.0 in stage 11.0 (TID 83) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:17 INFO Executor: Running task 57.0 in stage 11.0 (TID 83) -26/04/01 05:56:17 INFO TaskSetManager: Starting task 58.0 in stage 11.0 (TID 84) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 55.0 in stage 11.0 (TID 81) in 204 ms on 10.0.0.133 (executor driver) (54/64) -26/04/01 05:56:17 INFO Executor: Running task 58.0 in stage 11.0 (TID 84) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 54.0 in stage 11.0 (TID 80) in 205 ms on 10.0.0.133 (executor driver) (55/64) -26/04/01 05:56:17 INFO Executor: Finished task 53.0 in stage 11.0 (TID 79). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO Executor: Finished task 56.0 in stage 11.0 (TID 82). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO TaskSetManager: Starting task 59.0 in stage 11.0 (TID 85) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 53.0 in stage 11.0 (TID 79) in 206 ms on 10.0.0.133 (executor driver) (56/64) -26/04/01 05:56:17 INFO Executor: Running task 59.0 in stage 11.0 (TID 85) -26/04/01 05:56:17 INFO TaskSetManager: Starting task 60.0 in stage 11.0 (TID 86) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:17 INFO Executor: Running task 60.0 in stage 11.0 (TID 86) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 56.0 in stage 11.0 (TID 82) in 204 ms on 10.0.0.133 (executor driver) (57/64) -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.17.parquet, range: 134217728-233534488, partition values: [empty row] -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.31.parquet, range: 134217728-233535690, partition values: [empty row] -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.18.parquet, range: 134217728-233534883, partition values: [empty row] -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.15.parquet, range: 134217728-233533044, partition values: [empty row] -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO Executor: Finished task 57.0 in stage 11.0 (TID 83). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO TaskSetManager: Starting task 61.0 in stage 11.0 (TID 87) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9598 bytes) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 57.0 in stage 11.0 (TID 83) in 206 ms on 10.0.0.133 (executor driver) (58/64) -26/04/01 05:56:17 INFO Executor: Running task 61.0 in stage 11.0 (TID 87) -26/04/01 05:56:17 INFO Executor: Finished task 60.0 in stage 11.0 (TID 86). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO TaskSetManager: Starting task 62.0 in stage 11.0 (TID 88) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 60.0 in stage 11.0 (TID 86) in 206 ms on 10.0.0.133 (executor driver) (59/64) -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.25.parquet, range: 134217728-233532173, partition values: [empty row] -26/04/01 05:56:17 INFO Executor: Finished task 58.0 in stage 11.0 (TID 84). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO Executor: Running task 62.0 in stage 11.0 (TID 88) -26/04/01 05:56:17 INFO TaskSetManager: Starting task 63.0 in stage 11.0 (TID 89) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9597 bytes) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 58.0 in stage 11.0 (TID 84) in 208 ms on 10.0.0.133 (executor driver) (60/64) -26/04/01 05:56:17 INFO Executor: Running task 63.0 in stage 11.0 (TID 89) -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.3.parquet, range: 134217728-233524702, partition values: [empty row] -26/04/01 05:56:17 INFO Executor: Finished task 59.0 in stage 11.0 (TID 85). 2251 bytes result sent to driver -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.2.parquet, range: 134217728-233511585, partition values: [empty row] -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 90) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:17 INFO TaskSetManager: Finished task 59.0 in stage 11.0 (TID 85) in 209 ms on 10.0.0.133 (executor driver) (61/64) -26/04/01 05:56:17 INFO Executor: Running task 0.0 in stage 12.0 (TID 90) -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(and(and(and(noteq(o_orderdate, null), gteq(o_orderdate, 8766)), lt(o_orderdate, 9131)), noteq(o_custkey, null)), noteq(o_orderkey, null)) -26/04/01 05:56:17 INFO CodeGenerator: Code generated in 4.42975 ms -26/04/01 05:56:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:17 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:18 INFO Executor: Finished task 61.0 in stage 11.0 (TID 87). 2251 bytes result sent to driver -26/04/01 05:56:18 INFO Executor: Finished task 62.0 in stage 11.0 (TID 88). 2251 bytes result sent to driver -26/04/01 05:56:18 INFO TaskSetManager: Starting task 1.0 in stage 12.0 (TID 91) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:18 INFO Executor: Running task 1.0 in stage 12.0 (TID 91) -26/04/01 05:56:18 INFO TaskSetManager: Starting task 2.0 in stage 12.0 (TID 92) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:18 INFO TaskSetManager: Finished task 61.0 in stage 11.0 (TID 87) in 211 ms on 10.0.0.133 (executor driver) (62/64) -26/04/01 05:56:18 INFO Executor: Running task 2.0 in stage 12.0 (TID 92) -26/04/01 05:56:18 INFO TaskSetManager: Finished task 62.0 in stage 11.0 (TID 88) in 209 ms on 10.0.0.133 (executor driver) (63/64) -26/04/01 05:56:18 INFO Executor: Finished task 63.0 in stage 11.0 (TID 89). 2251 bytes result sent to driver -26/04/01 05:56:18 INFO TaskSetManager: Starting task 3.0 in stage 12.0 (TID 93) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:18 INFO Executor: Running task 3.0 in stage 12.0 (TID 93) -26/04/01 05:56:18 INFO TaskSetManager: Finished task 63.0 in stage 11.0 (TID 89) in 210 ms on 10.0.0.133 (executor driver) (64/64) -26/04/01 05:56:18 INFO TaskSchedulerImpl: Removed TaskSet 11.0, whose tasks have all completed, from pool -26/04/01 05:56:18 INFO DAGScheduler: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 5.209 s -26/04/01 05:56:18 INFO DAGScheduler: looking for newly runnable stages -26/04/01 05:56:18 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 12, ShuffleMapStage 13) -26/04/01 05:56:18 INFO DAGScheduler: waiting: Set() -26/04/01 05:56:18 INFO DAGScheduler: failed: Set() -26/04/01 05:56:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:18 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:18 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:18 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:18 INFO ShufflePartitionsUtil: For shuffle(0, 2), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 05:56:18 INFO CodeGenerator: Code generated in 9.73425 ms -26/04/01 05:56:18 INFO CodeGenerator: Code generated in 4.046291 ms -26/04/01 05:56:18 INFO CodeGenerator: Code generated in 7.183792 ms -26/04/01 05:56:18 INFO DAGScheduler: Registering RDD 50 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 5 -26/04/01 05:56:18 INFO DAGScheduler: Got map stage job 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 7 output partitions -26/04/01 05:56:18 INFO DAGScheduler: Final stage: ShuffleMapStage 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:56:18 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 16, ShuffleMapStage 17) -26/04/01 05:56:18 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:18 INFO DAGScheduler: Submitting ShuffleMapStage 18 (MapPartitionsRDD[50] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:56:18 INFO MemoryStore: Block broadcast_22 stored as values in memory (estimated size 47.7 KiB, free 8.6 GiB) -26/04/01 05:56:18 INFO MemoryStore: Block broadcast_22_piece0 stored as bytes in memory (estimated size 22.2 KiB, free 8.6 GiB) -26/04/01 05:56:18 INFO BlockManagerInfo: Added broadcast_22_piece0 in memory on 10.0.0.133:58206 (size: 22.2 KiB, free: 8.6 GiB) -26/04/01 05:56:18 INFO SparkContext: Created broadcast 22 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:18 INFO DAGScheduler: Submitting 7 missing tasks from ShuffleMapStage 18 (MapPartitionsRDD[50] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6)) -26/04/01 05:56:18 INFO TaskSchedulerImpl: Adding task set 18.0 with 7 tasks resource profile 0 -26/04/01 05:56:18 INFO Executor: Finished task 0.0 in stage 12.0 (TID 90). 2251 bytes result sent to driver -26/04/01 05:56:18 INFO TaskSetManager: Starting task 4.0 in stage 12.0 (TID 94) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:18 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 90) in 774 ms on 10.0.0.133 (executor driver) (1/208) -26/04/01 05:56:18 INFO Executor: Running task 4.0 in stage 12.0 (TID 94) -26/04/01 05:56:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:18 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:18 INFO Executor: Finished task 3.0 in stage 12.0 (TID 93). 2251 bytes result sent to driver -26/04/01 05:56:18 INFO TaskSetManager: Starting task 5.0 in stage 12.0 (TID 95) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:18 INFO TaskSetManager: Finished task 3.0 in stage 12.0 (TID 93) in 805 ms on 10.0.0.133 (executor driver) (2/208) -26/04/01 05:56:18 INFO Executor: Running task 5.0 in stage 12.0 (TID 95) -26/04/01 05:56:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:18 INFO Executor: Finished task 2.0 in stage 12.0 (TID 92). 2251 bytes result sent to driver -26/04/01 05:56:18 INFO TaskSetManager: Starting task 6.0 in stage 12.0 (TID 96) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:18 INFO TaskSetManager: Finished task 2.0 in stage 12.0 (TID 92) in 809 ms on 10.0.0.133 (executor driver) (3/208) -26/04/01 05:56:18 INFO Executor: Running task 6.0 in stage 12.0 (TID 96) -26/04/01 05:56:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:18 INFO Executor: Finished task 1.0 in stage 12.0 (TID 91). 2251 bytes result sent to driver -26/04/01 05:56:18 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:18 INFO TaskSetManager: Starting task 7.0 in stage 12.0 (TID 97) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:18 INFO TaskSetManager: Finished task 1.0 in stage 12.0 (TID 91) in 811 ms on 10.0.0.133 (executor driver) (4/208) -26/04/01 05:56:18 INFO Executor: Running task 7.0 in stage 12.0 (TID 97) -26/04/01 05:56:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:18 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:18 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:19 INFO Executor: Finished task 4.0 in stage 12.0 (TID 94). 2251 bytes result sent to driver -26/04/01 05:56:19 INFO TaskSetManager: Starting task 8.0 in stage 12.0 (TID 98) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:19 INFO TaskSetManager: Finished task 4.0 in stage 12.0 (TID 94) in 821 ms on 10.0.0.133 (executor driver) (5/208) -26/04/01 05:56:19 INFO Executor: Running task 8.0 in stage 12.0 (TID 98) -26/04/01 05:56:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:19 INFO BlockManagerInfo: Removed broadcast_11_piece0 on 10.0.0.133:58206 in memory (size: 7.9 KiB, free: 8.6 GiB) -26/04/01 05:56:19 INFO BlockManagerInfo: Removed broadcast_16_piece0 on 10.0.0.133:58206 in memory (size: 8.3 KiB, free: 8.6 GiB) -26/04/01 05:56:19 INFO BlockManagerInfo: Removed broadcast_13_piece0 on 10.0.0.133:58206 in memory (size: 14.0 KiB, free: 8.6 GiB) -26/04/01 05:56:19 INFO BlockManagerInfo: Removed broadcast_14_piece0 on 10.0.0.133:58206 in memory (size: 6.7 KiB, free: 8.6 GiB) -26/04/01 05:56:19 INFO Executor: Finished task 6.0 in stage 12.0 (TID 96). 2251 bytes result sent to driver -26/04/01 05:56:19 INFO Executor: Finished task 7.0 in stage 12.0 (TID 97). 2251 bytes result sent to driver -26/04/01 05:56:19 INFO TaskSetManager: Starting task 9.0 in stage 12.0 (TID 99) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:19 INFO TaskSetManager: Finished task 6.0 in stage 12.0 (TID 96) in 908 ms on 10.0.0.133 (executor driver) (6/208) -26/04/01 05:56:19 INFO Executor: Running task 9.0 in stage 12.0 (TID 99) -26/04/01 05:56:19 INFO TaskSetManager: Starting task 10.0 in stage 12.0 (TID 100) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:19 INFO TaskSetManager: Finished task 7.0 in stage 12.0 (TID 97) in 906 ms on 10.0.0.133 (executor driver) (7/208) -26/04/01 05:56:19 INFO Executor: Running task 10.0 in stage 12.0 (TID 100) -26/04/01 05:56:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:19 INFO Executor: Finished task 5.0 in stage 12.0 (TID 95). 2251 bytes result sent to driver -26/04/01 05:56:19 INFO TaskSetManager: Starting task 11.0 in stage 12.0 (TID 101) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:19 INFO TaskSetManager: Finished task 5.0 in stage 12.0 (TID 95) in 934 ms on 10.0.0.133 (executor driver) (8/208) -26/04/01 05:56:19 INFO Executor: Running task 11.0 in stage 12.0 (TID 101) -26/04/01 05:56:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:19 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:20 INFO Executor: Finished task 8.0 in stage 12.0 (TID 98). 2251 bytes result sent to driver -26/04/01 05:56:20 INFO TaskSetManager: Starting task 12.0 in stage 12.0 (TID 102) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:20 INFO TaskSetManager: Finished task 8.0 in stage 12.0 (TID 98) in 848 ms on 10.0.0.133 (executor driver) (9/208) -26/04/01 05:56:20 INFO Executor: Running task 12.0 in stage 12.0 (TID 102) -26/04/01 05:56:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:20 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:20 INFO Executor: Finished task 11.0 in stage 12.0 (TID 101). 2251 bytes result sent to driver -26/04/01 05:56:20 INFO TaskSetManager: Starting task 13.0 in stage 12.0 (TID 103) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:20 INFO TaskSetManager: Finished task 11.0 in stage 12.0 (TID 101) in 763 ms on 10.0.0.133 (executor driver) (10/208) -26/04/01 05:56:20 INFO Executor: Running task 13.0 in stage 12.0 (TID 103) -26/04/01 05:56:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:20 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:20 INFO Executor: Finished task 9.0 in stage 12.0 (TID 99). 2251 bytes result sent to driver -26/04/01 05:56:20 INFO TaskSetManager: Starting task 14.0 in stage 12.0 (TID 104) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:20 INFO TaskSetManager: Finished task 9.0 in stage 12.0 (TID 99) in 794 ms on 10.0.0.133 (executor driver) (11/208) -26/04/01 05:56:20 INFO Executor: Running task 14.0 in stage 12.0 (TID 104) -26/04/01 05:56:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:20 INFO Executor: Finished task 10.0 in stage 12.0 (TID 100). 2251 bytes result sent to driver -26/04/01 05:56:20 INFO TaskSetManager: Starting task 15.0 in stage 12.0 (TID 105) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:20 INFO Executor: Running task 15.0 in stage 12.0 (TID 105) -26/04/01 05:56:20 INFO TaskSetManager: Finished task 10.0 in stage 12.0 (TID 100) in 796 ms on 10.0.0.133 (executor driver) (12/208) -26/04/01 05:56:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:20 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:20 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:21 INFO Executor: Finished task 12.0 in stage 12.0 (TID 102). 2294 bytes result sent to driver -26/04/01 05:56:21 INFO TaskSetManager: Starting task 16.0 in stage 12.0 (TID 106) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:21 INFO TaskSetManager: Finished task 12.0 in stage 12.0 (TID 102) in 748 ms on 10.0.0.133 (executor driver) (13/208) -26/04/01 05:56:21 INFO Executor: Running task 16.0 in stage 12.0 (TID 106) -26/04/01 05:56:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:21 INFO Executor: Finished task 15.0 in stage 12.0 (TID 105). 2251 bytes result sent to driver -26/04/01 05:56:21 INFO Executor: Finished task 14.0 in stage 12.0 (TID 104). 2251 bytes result sent to driver -26/04/01 05:56:21 INFO TaskSetManager: Starting task 17.0 in stage 12.0 (TID 107) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:21 INFO Executor: Running task 17.0 in stage 12.0 (TID 107) -26/04/01 05:56:21 INFO TaskSetManager: Starting task 18.0 in stage 12.0 (TID 108) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:21 INFO TaskSetManager: Finished task 15.0 in stage 12.0 (TID 105) in 887 ms on 10.0.0.133 (executor driver) (14/208) -26/04/01 05:56:21 INFO Executor: Running task 18.0 in stage 12.0 (TID 108) -26/04/01 05:56:21 INFO TaskSetManager: Finished task 14.0 in stage 12.0 (TID 104) in 889 ms on 10.0.0.133 (executor driver) (15/208) -26/04/01 05:56:21 INFO Executor: Finished task 13.0 in stage 12.0 (TID 103). 2251 bytes result sent to driver -26/04/01 05:56:21 INFO TaskSetManager: Starting task 19.0 in stage 12.0 (TID 109) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:21 INFO TaskSetManager: Finished task 13.0 in stage 12.0 (TID 103) in 899 ms on 10.0.0.133 (executor driver) (16/208) -26/04/01 05:56:21 INFO Executor: Running task 19.0 in stage 12.0 (TID 109) -26/04/01 05:56:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:21 INFO Executor: Finished task 16.0 in stage 12.0 (TID 106). 2251 bytes result sent to driver -26/04/01 05:56:21 INFO TaskSetManager: Starting task 20.0 in stage 12.0 (TID 110) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:21 INFO TaskSetManager: Finished task 16.0 in stage 12.0 (TID 106) in 760 ms on 10.0.0.133 (executor driver) (17/208) -26/04/01 05:56:21 INFO Executor: Running task 20.0 in stage 12.0 (TID 110) -26/04/01 05:56:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:21 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:22 INFO Executor: Finished task 17.0 in stage 12.0 (TID 107). 2251 bytes result sent to driver -26/04/01 05:56:22 INFO TaskSetManager: Starting task 21.0 in stage 12.0 (TID 111) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:22 INFO TaskSetManager: Finished task 17.0 in stage 12.0 (TID 107) in 771 ms on 10.0.0.133 (executor driver) (18/208) -26/04/01 05:56:22 INFO Executor: Running task 21.0 in stage 12.0 (TID 111) -26/04/01 05:56:22 INFO Executor: Finished task 18.0 in stage 12.0 (TID 108). 2251 bytes result sent to driver -26/04/01 05:56:22 INFO TaskSetManager: Starting task 22.0 in stage 12.0 (TID 112) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:22 INFO TaskSetManager: Finished task 18.0 in stage 12.0 (TID 108) in 771 ms on 10.0.0.133 (executor driver) (19/208) -26/04/01 05:56:22 INFO Executor: Running task 22.0 in stage 12.0 (TID 112) -26/04/01 05:56:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:22 INFO Executor: Finished task 19.0 in stage 12.0 (TID 109). 2251 bytes result sent to driver -26/04/01 05:56:22 INFO TaskSetManager: Starting task 23.0 in stage 12.0 (TID 113) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:22 INFO TaskSetManager: Finished task 19.0 in stage 12.0 (TID 109) in 780 ms on 10.0.0.133 (executor driver) (20/208) -26/04/01 05:56:22 INFO Executor: Running task 23.0 in stage 12.0 (TID 113) -26/04/01 05:56:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:22 INFO Executor: Finished task 20.0 in stage 12.0 (TID 110). 2251 bytes result sent to driver -26/04/01 05:56:22 INFO TaskSetManager: Starting task 24.0 in stage 12.0 (TID 114) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:22 INFO TaskSetManager: Finished task 20.0 in stage 12.0 (TID 110) in 754 ms on 10.0.0.133 (executor driver) (21/208) -26/04/01 05:56:22 INFO Executor: Running task 24.0 in stage 12.0 (TID 114) -26/04/01 05:56:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:22 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:23 INFO Executor: Finished task 22.0 in stage 12.0 (TID 112). 2251 bytes result sent to driver -26/04/01 05:56:23 INFO TaskSetManager: Starting task 25.0 in stage 12.0 (TID 115) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:23 INFO TaskSetManager: Finished task 22.0 in stage 12.0 (TID 112) in 754 ms on 10.0.0.133 (executor driver) (22/208) -26/04/01 05:56:23 INFO Executor: Running task 25.0 in stage 12.0 (TID 115) -26/04/01 05:56:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:23 INFO Executor: Finished task 21.0 in stage 12.0 (TID 111). 2251 bytes result sent to driver -26/04/01 05:56:23 INFO TaskSetManager: Starting task 26.0 in stage 12.0 (TID 116) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:23 INFO TaskSetManager: Finished task 21.0 in stage 12.0 (TID 111) in 757 ms on 10.0.0.133 (executor driver) (23/208) -26/04/01 05:56:23 INFO Executor: Running task 26.0 in stage 12.0 (TID 116) -26/04/01 05:56:23 INFO Executor: Finished task 23.0 in stage 12.0 (TID 113). 2251 bytes result sent to driver -26/04/01 05:56:23 INFO TaskSetManager: Starting task 27.0 in stage 12.0 (TID 117) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:23 INFO TaskSetManager: Finished task 23.0 in stage 12.0 (TID 113) in 747 ms on 10.0.0.133 (executor driver) (24/208) -26/04/01 05:56:23 INFO Executor: Running task 27.0 in stage 12.0 (TID 117) -26/04/01 05:56:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:23 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:23 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:23 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:23 INFO Executor: Finished task 24.0 in stage 12.0 (TID 114). 2251 bytes result sent to driver -26/04/01 05:56:23 INFO TaskSetManager: Starting task 28.0 in stage 12.0 (TID 118) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:23 INFO TaskSetManager: Finished task 24.0 in stage 12.0 (TID 114) in 735 ms on 10.0.0.133 (executor driver) (25/208) -26/04/01 05:56:23 INFO Executor: Running task 28.0 in stage 12.0 (TID 118) -26/04/01 05:56:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:23 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:23 INFO Executor: Finished task 26.0 in stage 12.0 (TID 116). 2251 bytes result sent to driver -26/04/01 05:56:23 INFO TaskSetManager: Starting task 29.0 in stage 12.0 (TID 119) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:23 INFO TaskSetManager: Finished task 26.0 in stage 12.0 (TID 116) in 789 ms on 10.0.0.133 (executor driver) (26/208) -26/04/01 05:56:23 INFO Executor: Running task 29.0 in stage 12.0 (TID 119) -26/04/01 05:56:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:23 INFO Executor: Finished task 27.0 in stage 12.0 (TID 117). 2251 bytes result sent to driver -26/04/01 05:56:23 INFO TaskSetManager: Starting task 30.0 in stage 12.0 (TID 120) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:23 INFO TaskSetManager: Finished task 27.0 in stage 12.0 (TID 117) in 791 ms on 10.0.0.133 (executor driver) (27/208) -26/04/01 05:56:23 INFO Executor: Running task 30.0 in stage 12.0 (TID 120) -26/04/01 05:56:23 INFO Executor: Finished task 25.0 in stage 12.0 (TID 115). 2251 bytes result sent to driver -26/04/01 05:56:23 INFO TaskSetManager: Starting task 31.0 in stage 12.0 (TID 121) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:23 INFO TaskSetManager: Finished task 25.0 in stage 12.0 (TID 115) in 795 ms on 10.0.0.133 (executor driver) (28/208) -26/04/01 05:56:23 INFO Executor: Running task 31.0 in stage 12.0 (TID 121) -26/04/01 05:56:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:23 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:23 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:23 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:24 INFO Executor: Finished task 28.0 in stage 12.0 (TID 118). 2251 bytes result sent to driver -26/04/01 05:56:24 INFO TaskSetManager: Starting task 32.0 in stage 12.0 (TID 122) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:24 INFO Executor: Running task 32.0 in stage 12.0 (TID 122) -26/04/01 05:56:24 INFO TaskSetManager: Finished task 28.0 in stage 12.0 (TID 118) in 767 ms on 10.0.0.133 (executor driver) (29/208) -26/04/01 05:56:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:24 INFO Executor: Finished task 31.0 in stage 12.0 (TID 121). 2251 bytes result sent to driver -26/04/01 05:56:24 INFO TaskSetManager: Starting task 33.0 in stage 12.0 (TID 123) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:24 INFO TaskSetManager: Finished task 31.0 in stage 12.0 (TID 121) in 743 ms on 10.0.0.133 (executor driver) (30/208) -26/04/01 05:56:24 INFO Executor: Running task 33.0 in stage 12.0 (TID 123) -26/04/01 05:56:24 INFO Executor: Finished task 30.0 in stage 12.0 (TID 120). 2251 bytes result sent to driver -26/04/01 05:56:24 INFO TaskSetManager: Starting task 34.0 in stage 12.0 (TID 124) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:24 INFO TaskSetManager: Finished task 30.0 in stage 12.0 (TID 120) in 744 ms on 10.0.0.133 (executor driver) (31/208) -26/04/01 05:56:24 INFO Executor: Running task 34.0 in stage 12.0 (TID 124) -26/04/01 05:56:24 INFO Executor: Finished task 29.0 in stage 12.0 (TID 119). 2251 bytes result sent to driver -26/04/01 05:56:24 INFO TaskSetManager: Starting task 35.0 in stage 12.0 (TID 125) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:24 INFO TaskSetManager: Finished task 29.0 in stage 12.0 (TID 119) in 747 ms on 10.0.0.133 (executor driver) (32/208) -26/04/01 05:56:24 INFO Executor: Running task 35.0 in stage 12.0 (TID 125) -26/04/01 05:56:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:24 INFO Executor: Finished task 32.0 in stage 12.0 (TID 122). 2251 bytes result sent to driver -26/04/01 05:56:24 INFO TaskSetManager: Starting task 36.0 in stage 12.0 (TID 126) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:24 INFO TaskSetManager: Finished task 32.0 in stage 12.0 (TID 122) in 719 ms on 10.0.0.133 (executor driver) (33/208) -26/04/01 05:56:24 INFO Executor: Running task 36.0 in stage 12.0 (TID 126) -26/04/01 05:56:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:24 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:25 INFO Executor: Finished task 35.0 in stage 12.0 (TID 125). 2251 bytes result sent to driver -26/04/01 05:56:25 INFO TaskSetManager: Starting task 37.0 in stage 12.0 (TID 127) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:25 INFO Executor: Finished task 34.0 in stage 12.0 (TID 124). 2251 bytes result sent to driver -26/04/01 05:56:25 INFO TaskSetManager: Finished task 35.0 in stage 12.0 (TID 125) in 742 ms on 10.0.0.133 (executor driver) (34/208) -26/04/01 05:56:25 INFO Executor: Running task 37.0 in stage 12.0 (TID 127) -26/04/01 05:56:25 INFO TaskSetManager: Starting task 38.0 in stage 12.0 (TID 128) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:25 INFO TaskSetManager: Finished task 34.0 in stage 12.0 (TID 124) in 743 ms on 10.0.0.133 (executor driver) (35/208) -26/04/01 05:56:25 INFO Executor: Running task 38.0 in stage 12.0 (TID 128) -26/04/01 05:56:25 INFO Executor: Finished task 33.0 in stage 12.0 (TID 123). 2251 bytes result sent to driver -26/04/01 05:56:25 INFO TaskSetManager: Starting task 39.0 in stage 12.0 (TID 129) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:25 INFO Executor: Running task 39.0 in stage 12.0 (TID 129) -26/04/01 05:56:25 INFO TaskSetManager: Finished task 33.0 in stage 12.0 (TID 123) in 745 ms on 10.0.0.133 (executor driver) (36/208) -26/04/01 05:56:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:25 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:25 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:25 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:25 INFO Executor: Finished task 36.0 in stage 12.0 (TID 126). 2251 bytes result sent to driver -26/04/01 05:56:25 INFO TaskSetManager: Starting task 40.0 in stage 12.0 (TID 130) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:25 INFO Executor: Running task 40.0 in stage 12.0 (TID 130) -26/04/01 05:56:25 INFO TaskSetManager: Finished task 36.0 in stage 12.0 (TID 126) in 757 ms on 10.0.0.133 (executor driver) (37/208) -26/04/01 05:56:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:25 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:26 INFO Executor: Finished task 37.0 in stage 12.0 (TID 127). 2251 bytes result sent to driver -26/04/01 05:56:26 INFO TaskSetManager: Starting task 41.0 in stage 12.0 (TID 131) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:26 INFO Executor: Running task 41.0 in stage 12.0 (TID 131) -26/04/01 05:56:26 INFO TaskSetManager: Finished task 37.0 in stage 12.0 (TID 127) in 769 ms on 10.0.0.133 (executor driver) (38/208) -26/04/01 05:56:26 INFO Executor: Finished task 39.0 in stage 12.0 (TID 129). 2251 bytes result sent to driver -26/04/01 05:56:26 INFO TaskSetManager: Starting task 42.0 in stage 12.0 (TID 132) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:26 INFO Executor: Running task 42.0 in stage 12.0 (TID 132) -26/04/01 05:56:26 INFO TaskSetManager: Finished task 39.0 in stage 12.0 (TID 129) in 769 ms on 10.0.0.133 (executor driver) (39/208) -26/04/01 05:56:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:26 INFO Executor: Finished task 38.0 in stage 12.0 (TID 128). 2251 bytes result sent to driver -26/04/01 05:56:26 INFO TaskSetManager: Starting task 43.0 in stage 12.0 (TID 133) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:26 INFO TaskSetManager: Finished task 38.0 in stage 12.0 (TID 128) in 771 ms on 10.0.0.133 (executor driver) (40/208) -26/04/01 05:56:26 INFO Executor: Running task 43.0 in stage 12.0 (TID 133) -26/04/01 05:56:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:26 INFO Executor: Finished task 40.0 in stage 12.0 (TID 130). 2251 bytes result sent to driver -26/04/01 05:56:26 INFO TaskSetManager: Starting task 44.0 in stage 12.0 (TID 134) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:26 INFO TaskSetManager: Finished task 40.0 in stage 12.0 (TID 130) in 703 ms on 10.0.0.133 (executor driver) (41/208) -26/04/01 05:56:26 INFO Executor: Running task 44.0 in stage 12.0 (TID 134) -26/04/01 05:56:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:26 INFO Executor: Finished task 42.0 in stage 12.0 (TID 132). 2251 bytes result sent to driver -26/04/01 05:56:26 INFO TaskSetManager: Starting task 45.0 in stage 12.0 (TID 135) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:26 INFO TaskSetManager: Finished task 42.0 in stage 12.0 (TID 132) in 747 ms on 10.0.0.133 (executor driver) (42/208) -26/04/01 05:56:26 INFO Executor: Running task 45.0 in stage 12.0 (TID 135) -26/04/01 05:56:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:26 INFO Executor: Finished task 43.0 in stage 12.0 (TID 133). 2251 bytes result sent to driver -26/04/01 05:56:26 INFO TaskSetManager: Starting task 46.0 in stage 12.0 (TID 136) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:26 INFO TaskSetManager: Finished task 43.0 in stage 12.0 (TID 133) in 776 ms on 10.0.0.133 (executor driver) (43/208) -26/04/01 05:56:26 INFO Executor: Running task 46.0 in stage 12.0 (TID 136) -26/04/01 05:56:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:26 INFO Executor: Finished task 41.0 in stage 12.0 (TID 131). 2251 bytes result sent to driver -26/04/01 05:56:26 INFO TaskSetManager: Starting task 47.0 in stage 12.0 (TID 137) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:26 INFO TaskSetManager: Finished task 41.0 in stage 12.0 (TID 131) in 789 ms on 10.0.0.133 (executor driver) (44/208) -26/04/01 05:56:26 INFO Executor: Running task 47.0 in stage 12.0 (TID 137) -26/04/01 05:56:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:26 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:27 INFO Executor: Finished task 44.0 in stage 12.0 (TID 134). 2251 bytes result sent to driver -26/04/01 05:56:27 INFO TaskSetManager: Starting task 48.0 in stage 12.0 (TID 138) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:27 INFO Executor: Running task 48.0 in stage 12.0 (TID 138) -26/04/01 05:56:27 INFO TaskSetManager: Finished task 44.0 in stage 12.0 (TID 134) in 737 ms on 10.0.0.133 (executor driver) (45/208) -26/04/01 05:56:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:27 INFO Executor: Finished task 45.0 in stage 12.0 (TID 135). 2251 bytes result sent to driver -26/04/01 05:56:27 INFO TaskSetManager: Starting task 49.0 in stage 12.0 (TID 139) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:27 INFO TaskSetManager: Finished task 45.0 in stage 12.0 (TID 135) in 726 ms on 10.0.0.133 (executor driver) (46/208) -26/04/01 05:56:27 INFO Executor: Running task 49.0 in stage 12.0 (TID 139) -26/04/01 05:56:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:27 INFO Executor: Finished task 46.0 in stage 12.0 (TID 136). 2251 bytes result sent to driver -26/04/01 05:56:27 INFO TaskSetManager: Starting task 50.0 in stage 12.0 (TID 140) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:27 INFO Executor: Running task 50.0 in stage 12.0 (TID 140) -26/04/01 05:56:27 INFO TaskSetManager: Finished task 46.0 in stage 12.0 (TID 136) in 788 ms on 10.0.0.133 (executor driver) (47/208) -26/04/01 05:56:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:27 INFO Executor: Finished task 47.0 in stage 12.0 (TID 137). 2251 bytes result sent to driver -26/04/01 05:56:27 INFO TaskSetManager: Starting task 51.0 in stage 12.0 (TID 141) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:27 INFO TaskSetManager: Finished task 47.0 in stage 12.0 (TID 137) in 822 ms on 10.0.0.133 (executor driver) (48/208) -26/04/01 05:56:27 INFO Executor: Running task 51.0 in stage 12.0 (TID 141) -26/04/01 05:56:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:27 INFO Executor: Finished task 48.0 in stage 12.0 (TID 138). 2251 bytes result sent to driver -26/04/01 05:56:27 INFO TaskSetManager: Starting task 52.0 in stage 12.0 (TID 142) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:27 INFO TaskSetManager: Finished task 48.0 in stage 12.0 (TID 138) in 760 ms on 10.0.0.133 (executor driver) (49/208) -26/04/01 05:56:27 INFO Executor: Running task 52.0 in stage 12.0 (TID 142) -26/04/01 05:56:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:27 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:28 INFO Executor: Finished task 49.0 in stage 12.0 (TID 139). 2251 bytes result sent to driver -26/04/01 05:56:28 INFO TaskSetManager: Starting task 53.0 in stage 12.0 (TID 143) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:28 INFO TaskSetManager: Finished task 49.0 in stage 12.0 (TID 139) in 732 ms on 10.0.0.133 (executor driver) (50/208) -26/04/01 05:56:28 INFO Executor: Running task 53.0 in stage 12.0 (TID 143) -26/04/01 05:56:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:28 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:28 INFO Executor: Finished task 50.0 in stage 12.0 (TID 140). 2251 bytes result sent to driver -26/04/01 05:56:28 INFO TaskSetManager: Starting task 54.0 in stage 12.0 (TID 144) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:28 INFO TaskSetManager: Finished task 50.0 in stage 12.0 (TID 140) in 742 ms on 10.0.0.133 (executor driver) (51/208) -26/04/01 05:56:28 INFO Executor: Running task 54.0 in stage 12.0 (TID 144) -26/04/01 05:56:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:28 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:28 INFO Executor: Finished task 51.0 in stage 12.0 (TID 141). 2251 bytes result sent to driver -26/04/01 05:56:28 INFO TaskSetManager: Starting task 55.0 in stage 12.0 (TID 145) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:28 INFO TaskSetManager: Finished task 51.0 in stage 12.0 (TID 141) in 725 ms on 10.0.0.133 (executor driver) (52/208) -26/04/01 05:56:28 INFO Executor: Running task 55.0 in stage 12.0 (TID 145) -26/04/01 05:56:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:28 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:28 INFO Executor: Finished task 52.0 in stage 12.0 (TID 142). 2251 bytes result sent to driver -26/04/01 05:56:28 INFO TaskSetManager: Starting task 56.0 in stage 12.0 (TID 146) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:28 INFO TaskSetManager: Finished task 52.0 in stage 12.0 (TID 142) in 738 ms on 10.0.0.133 (executor driver) (53/208) -26/04/01 05:56:28 INFO Executor: Running task 56.0 in stage 12.0 (TID 146) -26/04/01 05:56:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:28 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:29 INFO Executor: Finished task 53.0 in stage 12.0 (TID 143). 2251 bytes result sent to driver -26/04/01 05:56:29 INFO TaskSetManager: Starting task 57.0 in stage 12.0 (TID 147) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:29 INFO Executor: Running task 57.0 in stage 12.0 (TID 147) -26/04/01 05:56:29 INFO TaskSetManager: Finished task 53.0 in stage 12.0 (TID 143) in 727 ms on 10.0.0.133 (executor driver) (54/208) -26/04/01 05:56:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:29 INFO Executor: Finished task 54.0 in stage 12.0 (TID 144). 2251 bytes result sent to driver -26/04/01 05:56:29 INFO TaskSetManager: Starting task 58.0 in stage 12.0 (TID 148) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:29 INFO TaskSetManager: Finished task 54.0 in stage 12.0 (TID 144) in 774 ms on 10.0.0.133 (executor driver) (55/208) -26/04/01 05:56:29 INFO Executor: Running task 58.0 in stage 12.0 (TID 148) -26/04/01 05:56:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:29 INFO Executor: Finished task 55.0 in stage 12.0 (TID 145). 2251 bytes result sent to driver -26/04/01 05:56:29 INFO TaskSetManager: Starting task 59.0 in stage 12.0 (TID 149) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:29 INFO TaskSetManager: Finished task 55.0 in stage 12.0 (TID 145) in 756 ms on 10.0.0.133 (executor driver) (56/208) -26/04/01 05:56:29 INFO Executor: Running task 59.0 in stage 12.0 (TID 149) -26/04/01 05:56:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:29 INFO Executor: Finished task 56.0 in stage 12.0 (TID 146). 2251 bytes result sent to driver -26/04/01 05:56:29 INFO TaskSetManager: Starting task 60.0 in stage 12.0 (TID 150) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:29 INFO TaskSetManager: Finished task 56.0 in stage 12.0 (TID 146) in 767 ms on 10.0.0.133 (executor driver) (57/208) -26/04/01 05:56:29 INFO Executor: Running task 60.0 in stage 12.0 (TID 150) -26/04/01 05:56:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:29 INFO Executor: Finished task 57.0 in stage 12.0 (TID 147). 2251 bytes result sent to driver -26/04/01 05:56:29 INFO TaskSetManager: Starting task 61.0 in stage 12.0 (TID 151) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:29 INFO TaskSetManager: Finished task 57.0 in stage 12.0 (TID 147) in 757 ms on 10.0.0.133 (executor driver) (58/208) -26/04/01 05:56:29 INFO Executor: Running task 61.0 in stage 12.0 (TID 151) -26/04/01 05:56:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:29 INFO Executor: Finished task 58.0 in stage 12.0 (TID 148). 2251 bytes result sent to driver -26/04/01 05:56:29 INFO TaskSetManager: Starting task 62.0 in stage 12.0 (TID 152) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:29 INFO TaskSetManager: Finished task 58.0 in stage 12.0 (TID 148) in 761 ms on 10.0.0.133 (executor driver) (59/208) -26/04/01 05:56:29 INFO Executor: Running task 62.0 in stage 12.0 (TID 152) -26/04/01 05:56:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:29 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:30 INFO Executor: Finished task 59.0 in stage 12.0 (TID 149). 2251 bytes result sent to driver -26/04/01 05:56:30 INFO TaskSetManager: Starting task 63.0 in stage 12.0 (TID 153) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:30 INFO TaskSetManager: Finished task 59.0 in stage 12.0 (TID 149) in 782 ms on 10.0.0.133 (executor driver) (60/208) -26/04/01 05:56:30 INFO Executor: Running task 63.0 in stage 12.0 (TID 153) -26/04/01 05:56:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:30 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:30 INFO Executor: Finished task 60.0 in stage 12.0 (TID 150). 2251 bytes result sent to driver -26/04/01 05:56:30 INFO TaskSetManager: Starting task 64.0 in stage 12.0 (TID 154) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:30 INFO TaskSetManager: Finished task 60.0 in stage 12.0 (TID 150) in 765 ms on 10.0.0.133 (executor driver) (61/208) -26/04/01 05:56:30 INFO Executor: Running task 64.0 in stage 12.0 (TID 154) -26/04/01 05:56:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:30 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:30 INFO Executor: Finished task 61.0 in stage 12.0 (TID 151). 2251 bytes result sent to driver -26/04/01 05:56:30 INFO TaskSetManager: Starting task 65.0 in stage 12.0 (TID 155) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:30 INFO TaskSetManager: Finished task 61.0 in stage 12.0 (TID 151) in 755 ms on 10.0.0.133 (executor driver) (62/208) -26/04/01 05:56:30 INFO Executor: Running task 65.0 in stage 12.0 (TID 155) -26/04/01 05:56:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:30 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:30 INFO Executor: Finished task 62.0 in stage 12.0 (TID 152). 2251 bytes result sent to driver -26/04/01 05:56:30 INFO TaskSetManager: Starting task 66.0 in stage 12.0 (TID 156) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:30 INFO TaskSetManager: Finished task 62.0 in stage 12.0 (TID 152) in 760 ms on 10.0.0.133 (executor driver) (63/208) -26/04/01 05:56:30 INFO Executor: Running task 66.0 in stage 12.0 (TID 156) -26/04/01 05:56:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:30 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:30 INFO Executor: Finished task 63.0 in stage 12.0 (TID 153). 2251 bytes result sent to driver -26/04/01 05:56:30 INFO TaskSetManager: Starting task 67.0 in stage 12.0 (TID 157) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:30 INFO TaskSetManager: Finished task 63.0 in stage 12.0 (TID 153) in 742 ms on 10.0.0.133 (executor driver) (64/208) -26/04/01 05:56:30 INFO Executor: Running task 67.0 in stage 12.0 (TID 157) -26/04/01 05:56:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:30 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:30 INFO Executor: Finished task 64.0 in stage 12.0 (TID 154). 2251 bytes result sent to driver -26/04/01 05:56:30 INFO TaskSetManager: Starting task 68.0 in stage 12.0 (TID 158) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:30 INFO TaskSetManager: Finished task 64.0 in stage 12.0 (TID 154) in 726 ms on 10.0.0.133 (executor driver) (65/208) -26/04/01 05:56:30 INFO Executor: Running task 68.0 in stage 12.0 (TID 158) -26/04/01 05:56:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:30 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:31 INFO Executor: Finished task 65.0 in stage 12.0 (TID 155). 2251 bytes result sent to driver -26/04/01 05:56:31 INFO TaskSetManager: Starting task 69.0 in stage 12.0 (TID 159) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:31 INFO Executor: Running task 69.0 in stage 12.0 (TID 159) -26/04/01 05:56:31 INFO TaskSetManager: Finished task 65.0 in stage 12.0 (TID 155) in 721 ms on 10.0.0.133 (executor driver) (66/208) -26/04/01 05:56:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:31 INFO Executor: Finished task 66.0 in stage 12.0 (TID 156). 2251 bytes result sent to driver -26/04/01 05:56:31 INFO TaskSetManager: Starting task 70.0 in stage 12.0 (TID 160) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:31 INFO TaskSetManager: Finished task 66.0 in stage 12.0 (TID 156) in 754 ms on 10.0.0.133 (executor driver) (67/208) -26/04/01 05:56:31 INFO Executor: Running task 70.0 in stage 12.0 (TID 160) -26/04/01 05:56:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:31 INFO Executor: Finished task 67.0 in stage 12.0 (TID 157). 2251 bytes result sent to driver -26/04/01 05:56:31 INFO TaskSetManager: Starting task 71.0 in stage 12.0 (TID 161) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:31 INFO TaskSetManager: Finished task 67.0 in stage 12.0 (TID 157) in 772 ms on 10.0.0.133 (executor driver) (68/208) -26/04/01 05:56:31 INFO Executor: Running task 71.0 in stage 12.0 (TID 161) -26/04/01 05:56:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:31 INFO Executor: Finished task 68.0 in stage 12.0 (TID 158). 2251 bytes result sent to driver -26/04/01 05:56:31 INFO TaskSetManager: Starting task 72.0 in stage 12.0 (TID 162) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:31 INFO TaskSetManager: Finished task 68.0 in stage 12.0 (TID 158) in 776 ms on 10.0.0.133 (executor driver) (69/208) -26/04/01 05:56:31 INFO Executor: Running task 72.0 in stage 12.0 (TID 162) -26/04/01 05:56:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:31 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:32 INFO Executor: Finished task 69.0 in stage 12.0 (TID 159). 2251 bytes result sent to driver -26/04/01 05:56:32 INFO TaskSetManager: Starting task 73.0 in stage 12.0 (TID 163) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:32 INFO TaskSetManager: Finished task 69.0 in stage 12.0 (TID 159) in 769 ms on 10.0.0.133 (executor driver) (70/208) -26/04/01 05:56:32 INFO Executor: Running task 73.0 in stage 12.0 (TID 163) -26/04/01 05:56:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:32 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:32 INFO Executor: Finished task 71.0 in stage 12.0 (TID 161). 2251 bytes result sent to driver -26/04/01 05:56:32 INFO TaskSetManager: Starting task 74.0 in stage 12.0 (TID 164) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:32 INFO TaskSetManager: Finished task 71.0 in stage 12.0 (TID 161) in 722 ms on 10.0.0.133 (executor driver) (71/208) -26/04/01 05:56:32 INFO Executor: Running task 74.0 in stage 12.0 (TID 164) -26/04/01 05:56:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:32 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:32 INFO Executor: Finished task 72.0 in stage 12.0 (TID 162). 2251 bytes result sent to driver -26/04/01 05:56:32 INFO TaskSetManager: Starting task 75.0 in stage 12.0 (TID 165) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:32 INFO TaskSetManager: Finished task 72.0 in stage 12.0 (TID 162) in 724 ms on 10.0.0.133 (executor driver) (72/208) -26/04/01 05:56:32 INFO Executor: Running task 75.0 in stage 12.0 (TID 165) -26/04/01 05:56:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:32 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:32 INFO Executor: Finished task 70.0 in stage 12.0 (TID 160). 2251 bytes result sent to driver -26/04/01 05:56:32 INFO TaskSetManager: Starting task 76.0 in stage 12.0 (TID 166) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:32 INFO TaskSetManager: Finished task 70.0 in stage 12.0 (TID 160) in 854 ms on 10.0.0.133 (executor driver) (73/208) -26/04/01 05:56:32 INFO Executor: Running task 76.0 in stage 12.0 (TID 166) -26/04/01 05:56:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:32 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:32 INFO Executor: Finished task 73.0 in stage 12.0 (TID 163). 2251 bytes result sent to driver -26/04/01 05:56:32 INFO TaskSetManager: Starting task 77.0 in stage 12.0 (TID 167) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:32 INFO TaskSetManager: Finished task 73.0 in stage 12.0 (TID 163) in 736 ms on 10.0.0.133 (executor driver) (74/208) -26/04/01 05:56:32 INFO Executor: Running task 77.0 in stage 12.0 (TID 167) -26/04/01 05:56:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:32 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:32 INFO Executor: Finished task 74.0 in stage 12.0 (TID 164). 2251 bytes result sent to driver -26/04/01 05:56:32 INFO TaskSetManager: Starting task 78.0 in stage 12.0 (TID 168) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:32 INFO TaskSetManager: Finished task 74.0 in stage 12.0 (TID 164) in 716 ms on 10.0.0.133 (executor driver) (75/208) -26/04/01 05:56:32 INFO Executor: Running task 78.0 in stage 12.0 (TID 168) -26/04/01 05:56:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:32 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:33 INFO Executor: Finished task 75.0 in stage 12.0 (TID 165). 2251 bytes result sent to driver -26/04/01 05:56:33 INFO TaskSetManager: Starting task 79.0 in stage 12.0 (TID 169) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:33 INFO TaskSetManager: Finished task 75.0 in stage 12.0 (TID 165) in 736 ms on 10.0.0.133 (executor driver) (76/208) -26/04/01 05:56:33 INFO Executor: Running task 79.0 in stage 12.0 (TID 169) -26/04/01 05:56:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:33 INFO Executor: Finished task 76.0 in stage 12.0 (TID 166). 2251 bytes result sent to driver -26/04/01 05:56:33 INFO TaskSetManager: Starting task 80.0 in stage 12.0 (TID 170) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:33 INFO TaskSetManager: Finished task 76.0 in stage 12.0 (TID 166) in 730 ms on 10.0.0.133 (executor driver) (77/208) -26/04/01 05:56:33 INFO Executor: Running task 80.0 in stage 12.0 (TID 170) -26/04/01 05:56:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:33 INFO Executor: Finished task 77.0 in stage 12.0 (TID 167). 2251 bytes result sent to driver -26/04/01 05:56:33 INFO TaskSetManager: Starting task 81.0 in stage 12.0 (TID 171) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:33 INFO TaskSetManager: Finished task 77.0 in stage 12.0 (TID 167) in 710 ms on 10.0.0.133 (executor driver) (78/208) -26/04/01 05:56:33 INFO Executor: Running task 81.0 in stage 12.0 (TID 171) -26/04/01 05:56:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:33 INFO Executor: Finished task 78.0 in stage 12.0 (TID 168). 2251 bytes result sent to driver -26/04/01 05:56:33 INFO TaskSetManager: Starting task 82.0 in stage 12.0 (TID 172) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:33 INFO TaskSetManager: Finished task 78.0 in stage 12.0 (TID 168) in 733 ms on 10.0.0.133 (executor driver) (79/208) -26/04/01 05:56:33 INFO Executor: Running task 82.0 in stage 12.0 (TID 172) -26/04/01 05:56:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:33 INFO Executor: Finished task 80.0 in stage 12.0 (TID 170). 2251 bytes result sent to driver -26/04/01 05:56:33 INFO TaskSetManager: Starting task 83.0 in stage 12.0 (TID 173) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:33 INFO TaskSetManager: Finished task 80.0 in stage 12.0 (TID 170) in 720 ms on 10.0.0.133 (executor driver) (80/208) -26/04/01 05:56:33 INFO Executor: Running task 83.0 in stage 12.0 (TID 173) -26/04/01 05:56:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:33 INFO Executor: Finished task 79.0 in stage 12.0 (TID 169). 2251 bytes result sent to driver -26/04/01 05:56:33 INFO TaskSetManager: Starting task 84.0 in stage 12.0 (TID 174) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:33 INFO Executor: Running task 84.0 in stage 12.0 (TID 174) -26/04/01 05:56:33 INFO TaskSetManager: Finished task 79.0 in stage 12.0 (TID 169) in 737 ms on 10.0.0.133 (executor driver) (81/208) -26/04/01 05:56:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:33 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:34 INFO Executor: Finished task 81.0 in stage 12.0 (TID 171). 2251 bytes result sent to driver -26/04/01 05:56:34 INFO TaskSetManager: Starting task 85.0 in stage 12.0 (TID 175) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:34 INFO TaskSetManager: Finished task 81.0 in stage 12.0 (TID 171) in 728 ms on 10.0.0.133 (executor driver) (82/208) -26/04/01 05:56:34 INFO Executor: Running task 85.0 in stage 12.0 (TID 175) -26/04/01 05:56:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:34 INFO Executor: Finished task 82.0 in stage 12.0 (TID 172). 2251 bytes result sent to driver -26/04/01 05:56:34 INFO TaskSetManager: Starting task 86.0 in stage 12.0 (TID 176) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:34 INFO TaskSetManager: Finished task 82.0 in stage 12.0 (TID 172) in 697 ms on 10.0.0.133 (executor driver) (83/208) -26/04/01 05:56:34 INFO Executor: Running task 86.0 in stage 12.0 (TID 176) -26/04/01 05:56:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:34 INFO Executor: Finished task 83.0 in stage 12.0 (TID 173). 2251 bytes result sent to driver -26/04/01 05:56:34 INFO TaskSetManager: Starting task 87.0 in stage 12.0 (TID 177) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:34 INFO TaskSetManager: Finished task 83.0 in stage 12.0 (TID 173) in 707 ms on 10.0.0.133 (executor driver) (84/208) -26/04/01 05:56:34 INFO Executor: Running task 87.0 in stage 12.0 (TID 177) -26/04/01 05:56:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:34 INFO Executor: Finished task 84.0 in stage 12.0 (TID 174). 2251 bytes result sent to driver -26/04/01 05:56:34 INFO TaskSetManager: Starting task 88.0 in stage 12.0 (TID 178) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:34 INFO Executor: Running task 88.0 in stage 12.0 (TID 178) -26/04/01 05:56:34 INFO TaskSetManager: Finished task 84.0 in stage 12.0 (TID 174) in 708 ms on 10.0.0.133 (executor driver) (85/208) -26/04/01 05:56:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:34 INFO Executor: Finished task 85.0 in stage 12.0 (TID 175). 2251 bytes result sent to driver -26/04/01 05:56:34 INFO TaskSetManager: Starting task 89.0 in stage 12.0 (TID 179) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:34 INFO TaskSetManager: Finished task 85.0 in stage 12.0 (TID 175) in 701 ms on 10.0.0.133 (executor driver) (86/208) -26/04/01 05:56:34 INFO Executor: Running task 89.0 in stage 12.0 (TID 179) -26/04/01 05:56:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:34 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:35 INFO Executor: Finished task 86.0 in stage 12.0 (TID 176). 2251 bytes result sent to driver -26/04/01 05:56:35 INFO TaskSetManager: Starting task 90.0 in stage 12.0 (TID 180) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:35 INFO TaskSetManager: Finished task 86.0 in stage 12.0 (TID 176) in 710 ms on 10.0.0.133 (executor driver) (87/208) -26/04/01 05:56:35 INFO Executor: Running task 90.0 in stage 12.0 (TID 180) -26/04/01 05:56:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:35 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:35 INFO Executor: Finished task 87.0 in stage 12.0 (TID 177). 2251 bytes result sent to driver -26/04/01 05:56:35 INFO TaskSetManager: Starting task 91.0 in stage 12.0 (TID 181) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:35 INFO TaskSetManager: Finished task 87.0 in stage 12.0 (TID 177) in 735 ms on 10.0.0.133 (executor driver) (88/208) -26/04/01 05:56:35 INFO Executor: Running task 91.0 in stage 12.0 (TID 181) -26/04/01 05:56:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:35 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:35 INFO Executor: Finished task 88.0 in stage 12.0 (TID 178). 2251 bytes result sent to driver -26/04/01 05:56:35 INFO TaskSetManager: Starting task 92.0 in stage 12.0 (TID 182) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:35 INFO Executor: Running task 92.0 in stage 12.0 (TID 182) -26/04/01 05:56:35 INFO TaskSetManager: Finished task 88.0 in stage 12.0 (TID 178) in 782 ms on 10.0.0.133 (executor driver) (89/208) -26/04/01 05:56:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:35 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:35 INFO Executor: Finished task 89.0 in stage 12.0 (TID 179). 2251 bytes result sent to driver -26/04/01 05:56:35 INFO TaskSetManager: Starting task 93.0 in stage 12.0 (TID 183) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:35 INFO Executor: Running task 93.0 in stage 12.0 (TID 183) -26/04/01 05:56:35 INFO TaskSetManager: Finished task 89.0 in stage 12.0 (TID 179) in 743 ms on 10.0.0.133 (executor driver) (90/208) -26/04/01 05:56:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:35 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:35 INFO Executor: Finished task 90.0 in stage 12.0 (TID 180). 2251 bytes result sent to driver -26/04/01 05:56:35 INFO TaskSetManager: Starting task 94.0 in stage 12.0 (TID 184) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:35 INFO TaskSetManager: Finished task 90.0 in stage 12.0 (TID 180) in 720 ms on 10.0.0.133 (executor driver) (91/208) -26/04/01 05:56:35 INFO Executor: Running task 94.0 in stage 12.0 (TID 184) -26/04/01 05:56:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:35 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:35 INFO Executor: Finished task 91.0 in stage 12.0 (TID 181). 2251 bytes result sent to driver -26/04/01 05:56:35 INFO TaskSetManager: Starting task 95.0 in stage 12.0 (TID 185) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:35 INFO Executor: Running task 95.0 in stage 12.0 (TID 185) -26/04/01 05:56:35 INFO TaskSetManager: Finished task 91.0 in stage 12.0 (TID 181) in 745 ms on 10.0.0.133 (executor driver) (92/208) -26/04/01 05:56:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:35 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:36 INFO Executor: Finished task 92.0 in stage 12.0 (TID 182). 2251 bytes result sent to driver -26/04/01 05:56:36 INFO TaskSetManager: Starting task 96.0 in stage 12.0 (TID 186) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:36 INFO TaskSetManager: Finished task 92.0 in stage 12.0 (TID 182) in 723 ms on 10.0.0.133 (executor driver) (93/208) -26/04/01 05:56:36 INFO Executor: Running task 96.0 in stage 12.0 (TID 186) -26/04/01 05:56:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:36 INFO Executor: Finished task 93.0 in stage 12.0 (TID 183). 2251 bytes result sent to driver -26/04/01 05:56:36 INFO TaskSetManager: Starting task 97.0 in stage 12.0 (TID 187) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:36 INFO TaskSetManager: Finished task 93.0 in stage 12.0 (TID 183) in 714 ms on 10.0.0.133 (executor driver) (94/208) -26/04/01 05:56:36 INFO Executor: Running task 97.0 in stage 12.0 (TID 187) -26/04/01 05:56:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:36 INFO Executor: Finished task 94.0 in stage 12.0 (TID 184). 2251 bytes result sent to driver -26/04/01 05:56:36 INFO TaskSetManager: Starting task 98.0 in stage 12.0 (TID 188) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:36 INFO TaskSetManager: Finished task 94.0 in stage 12.0 (TID 184) in 700 ms on 10.0.0.133 (executor driver) (95/208) -26/04/01 05:56:36 INFO Executor: Running task 98.0 in stage 12.0 (TID 188) -26/04/01 05:56:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:36 INFO Executor: Finished task 95.0 in stage 12.0 (TID 185). 2251 bytes result sent to driver -26/04/01 05:56:36 INFO TaskSetManager: Starting task 99.0 in stage 12.0 (TID 189) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:36 INFO TaskSetManager: Finished task 95.0 in stage 12.0 (TID 185) in 720 ms on 10.0.0.133 (executor driver) (96/208) -26/04/01 05:56:36 INFO Executor: Running task 99.0 in stage 12.0 (TID 189) -26/04/01 05:56:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:36 INFO Executor: Finished task 96.0 in stage 12.0 (TID 186). 2251 bytes result sent to driver -26/04/01 05:56:36 INFO TaskSetManager: Starting task 100.0 in stage 12.0 (TID 190) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:36 INFO TaskSetManager: Finished task 96.0 in stage 12.0 (TID 186) in 747 ms on 10.0.0.133 (executor driver) (97/208) -26/04/01 05:56:36 INFO Executor: Running task 100.0 in stage 12.0 (TID 190) -26/04/01 05:56:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:36 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:37 INFO Executor: Finished task 97.0 in stage 12.0 (TID 187). 2251 bytes result sent to driver -26/04/01 05:56:37 INFO TaskSetManager: Starting task 101.0 in stage 12.0 (TID 191) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:37 INFO TaskSetManager: Finished task 97.0 in stage 12.0 (TID 187) in 764 ms on 10.0.0.133 (executor driver) (98/208) -26/04/01 05:56:37 INFO Executor: Running task 101.0 in stage 12.0 (TID 191) -26/04/01 05:56:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:37 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:37 INFO Executor: Finished task 98.0 in stage 12.0 (TID 188). 2251 bytes result sent to driver -26/04/01 05:56:37 INFO TaskSetManager: Starting task 102.0 in stage 12.0 (TID 192) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:37 INFO TaskSetManager: Finished task 98.0 in stage 12.0 (TID 188) in 806 ms on 10.0.0.133 (executor driver) (99/208) -26/04/01 05:56:37 INFO Executor: Running task 102.0 in stage 12.0 (TID 192) -26/04/01 05:56:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:37 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:37 INFO Executor: Finished task 99.0 in stage 12.0 (TID 189). 2251 bytes result sent to driver -26/04/01 05:56:37 INFO TaskSetManager: Starting task 103.0 in stage 12.0 (TID 193) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:37 INFO TaskSetManager: Finished task 99.0 in stage 12.0 (TID 189) in 762 ms on 10.0.0.133 (executor driver) (100/208) -26/04/01 05:56:37 INFO Executor: Running task 103.0 in stage 12.0 (TID 193) -26/04/01 05:56:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:37 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:37 INFO Executor: Finished task 100.0 in stage 12.0 (TID 190). 2251 bytes result sent to driver -26/04/01 05:56:37 INFO TaskSetManager: Starting task 104.0 in stage 12.0 (TID 194) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:37 INFO TaskSetManager: Finished task 100.0 in stage 12.0 (TID 190) in 752 ms on 10.0.0.133 (executor driver) (101/208) -26/04/01 05:56:37 INFO Executor: Running task 104.0 in stage 12.0 (TID 194) -26/04/01 05:56:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:37 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:37 INFO Executor: Finished task 101.0 in stage 12.0 (TID 191). 2251 bytes result sent to driver -26/04/01 05:56:37 INFO TaskSetManager: Starting task 105.0 in stage 12.0 (TID 195) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:37 INFO TaskSetManager: Finished task 101.0 in stage 12.0 (TID 191) in 756 ms on 10.0.0.133 (executor driver) (102/208) -26/04/01 05:56:37 INFO Executor: Running task 105.0 in stage 12.0 (TID 195) -26/04/01 05:56:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:37 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:38 INFO Executor: Finished task 102.0 in stage 12.0 (TID 192). 2251 bytes result sent to driver -26/04/01 05:56:38 INFO TaskSetManager: Starting task 106.0 in stage 12.0 (TID 196) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:38 INFO TaskSetManager: Finished task 102.0 in stage 12.0 (TID 192) in 740 ms on 10.0.0.133 (executor driver) (103/208) -26/04/01 05:56:38 INFO Executor: Running task 106.0 in stage 12.0 (TID 196) -26/04/01 05:56:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:38 INFO Executor: Finished task 103.0 in stage 12.0 (TID 193). 2251 bytes result sent to driver -26/04/01 05:56:38 INFO TaskSetManager: Starting task 107.0 in stage 12.0 (TID 197) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:38 INFO TaskSetManager: Finished task 103.0 in stage 12.0 (TID 193) in 728 ms on 10.0.0.133 (executor driver) (104/208) -26/04/01 05:56:38 INFO Executor: Running task 107.0 in stage 12.0 (TID 197) -26/04/01 05:56:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:38 INFO Executor: Finished task 104.0 in stage 12.0 (TID 194). 2251 bytes result sent to driver -26/04/01 05:56:38 INFO TaskSetManager: Starting task 108.0 in stage 12.0 (TID 198) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:38 INFO TaskSetManager: Finished task 104.0 in stage 12.0 (TID 194) in 739 ms on 10.0.0.133 (executor driver) (105/208) -26/04/01 05:56:38 INFO Executor: Running task 108.0 in stage 12.0 (TID 198) -26/04/01 05:56:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:38 INFO Executor: Finished task 105.0 in stage 12.0 (TID 195). 2251 bytes result sent to driver -26/04/01 05:56:38 INFO TaskSetManager: Starting task 109.0 in stage 12.0 (TID 199) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:38 INFO TaskSetManager: Finished task 105.0 in stage 12.0 (TID 195) in 762 ms on 10.0.0.133 (executor driver) (106/208) -26/04/01 05:56:38 INFO Executor: Running task 109.0 in stage 12.0 (TID 199) -26/04/01 05:56:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:38 INFO Executor: Finished task 106.0 in stage 12.0 (TID 196). 2251 bytes result sent to driver -26/04/01 05:56:38 INFO TaskSetManager: Starting task 110.0 in stage 12.0 (TID 200) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:38 INFO TaskSetManager: Finished task 106.0 in stage 12.0 (TID 196) in 770 ms on 10.0.0.133 (executor driver) (107/208) -26/04/01 05:56:38 INFO Executor: Running task 110.0 in stage 12.0 (TID 200) -26/04/01 05:56:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:38 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:38 INFO Executor: Finished task 107.0 in stage 12.0 (TID 197). 2251 bytes result sent to driver -26/04/01 05:56:38 INFO TaskSetManager: Starting task 111.0 in stage 12.0 (TID 201) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:38 INFO TaskSetManager: Finished task 107.0 in stage 12.0 (TID 197) in 806 ms on 10.0.0.133 (executor driver) (108/208) -26/04/01 05:56:38 INFO Executor: Running task 111.0 in stage 12.0 (TID 201) -26/04/01 05:56:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:39 INFO Executor: Finished task 108.0 in stage 12.0 (TID 198). 2251 bytes result sent to driver -26/04/01 05:56:39 INFO TaskSetManager: Starting task 112.0 in stage 12.0 (TID 202) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:39 INFO TaskSetManager: Finished task 108.0 in stage 12.0 (TID 198) in 788 ms on 10.0.0.133 (executor driver) (109/208) -26/04/01 05:56:39 INFO Executor: Running task 112.0 in stage 12.0 (TID 202) -26/04/01 05:56:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:39 INFO Executor: Finished task 109.0 in stage 12.0 (TID 199). 2251 bytes result sent to driver -26/04/01 05:56:39 INFO TaskSetManager: Starting task 113.0 in stage 12.0 (TID 203) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:39 INFO TaskSetManager: Finished task 109.0 in stage 12.0 (TID 199) in 765 ms on 10.0.0.133 (executor driver) (110/208) -26/04/01 05:56:39 INFO Executor: Running task 113.0 in stage 12.0 (TID 203) -26/04/01 05:56:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:39 INFO Executor: Finished task 110.0 in stage 12.0 (TID 200). 2251 bytes result sent to driver -26/04/01 05:56:39 INFO TaskSetManager: Starting task 114.0 in stage 12.0 (TID 204) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:39 INFO TaskSetManager: Finished task 110.0 in stage 12.0 (TID 200) in 742 ms on 10.0.0.133 (executor driver) (111/208) -26/04/01 05:56:39 INFO Executor: Running task 114.0 in stage 12.0 (TID 204) -26/04/01 05:56:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:39 INFO Executor: Finished task 111.0 in stage 12.0 (TID 201). 2251 bytes result sent to driver -26/04/01 05:56:39 INFO TaskSetManager: Starting task 115.0 in stage 12.0 (TID 205) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:39 INFO TaskSetManager: Finished task 111.0 in stage 12.0 (TID 201) in 700 ms on 10.0.0.133 (executor driver) (112/208) -26/04/01 05:56:39 INFO Executor: Running task 115.0 in stage 12.0 (TID 205) -26/04/01 05:56:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:39 INFO Executor: Finished task 112.0 in stage 12.0 (TID 202). 2251 bytes result sent to driver -26/04/01 05:56:39 INFO TaskSetManager: Starting task 116.0 in stage 12.0 (TID 206) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:39 INFO TaskSetManager: Finished task 112.0 in stage 12.0 (TID 202) in 736 ms on 10.0.0.133 (executor driver) (113/208) -26/04/01 05:56:39 INFO Executor: Running task 116.0 in stage 12.0 (TID 206) -26/04/01 05:56:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:39 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:40 INFO Executor: Finished task 113.0 in stage 12.0 (TID 203). 2251 bytes result sent to driver -26/04/01 05:56:40 INFO TaskSetManager: Starting task 117.0 in stage 12.0 (TID 207) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:40 INFO TaskSetManager: Finished task 113.0 in stage 12.0 (TID 203) in 727 ms on 10.0.0.133 (executor driver) (114/208) -26/04/01 05:56:40 INFO Executor: Running task 117.0 in stage 12.0 (TID 207) -26/04/01 05:56:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:40 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:40 INFO Executor: Finished task 114.0 in stage 12.0 (TID 204). 2251 bytes result sent to driver -26/04/01 05:56:40 INFO TaskSetManager: Starting task 118.0 in stage 12.0 (TID 208) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:40 INFO TaskSetManager: Finished task 114.0 in stage 12.0 (TID 204) in 731 ms on 10.0.0.133 (executor driver) (115/208) -26/04/01 05:56:40 INFO Executor: Running task 118.0 in stage 12.0 (TID 208) -26/04/01 05:56:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:40 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:40 INFO Executor: Finished task 115.0 in stage 12.0 (TID 205). 2251 bytes result sent to driver -26/04/01 05:56:40 INFO TaskSetManager: Starting task 119.0 in stage 12.0 (TID 209) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:40 INFO TaskSetManager: Finished task 115.0 in stage 12.0 (TID 205) in 736 ms on 10.0.0.133 (executor driver) (116/208) -26/04/01 05:56:40 INFO Executor: Running task 119.0 in stage 12.0 (TID 209) -26/04/01 05:56:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:40 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:40 INFO Executor: Finished task 116.0 in stage 12.0 (TID 206). 2251 bytes result sent to driver -26/04/01 05:56:40 INFO TaskSetManager: Starting task 120.0 in stage 12.0 (TID 210) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:40 INFO TaskSetManager: Finished task 116.0 in stage 12.0 (TID 206) in 724 ms on 10.0.0.133 (executor driver) (117/208) -26/04/01 05:56:40 INFO Executor: Running task 120.0 in stage 12.0 (TID 210) -26/04/01 05:56:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:40 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:40 INFO Executor: Finished task 117.0 in stage 12.0 (TID 207). 2251 bytes result sent to driver -26/04/01 05:56:40 INFO TaskSetManager: Starting task 121.0 in stage 12.0 (TID 211) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:40 INFO Executor: Running task 121.0 in stage 12.0 (TID 211) -26/04/01 05:56:40 INFO TaskSetManager: Finished task 117.0 in stage 12.0 (TID 207) in 752 ms on 10.0.0.133 (executor driver) (118/208) -26/04/01 05:56:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:40 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:41 INFO Executor: Finished task 118.0 in stage 12.0 (TID 208). 2251 bytes result sent to driver -26/04/01 05:56:41 INFO TaskSetManager: Starting task 122.0 in stage 12.0 (TID 212) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:41 INFO TaskSetManager: Finished task 118.0 in stage 12.0 (TID 208) in 750 ms on 10.0.0.133 (executor driver) (119/208) -26/04/01 05:56:41 INFO Executor: Running task 122.0 in stage 12.0 (TID 212) -26/04/01 05:56:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:41 INFO Executor: Finished task 119.0 in stage 12.0 (TID 209). 2251 bytes result sent to driver -26/04/01 05:56:41 INFO TaskSetManager: Starting task 123.0 in stage 12.0 (TID 213) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:41 INFO Executor: Running task 123.0 in stage 12.0 (TID 213) -26/04/01 05:56:41 INFO TaskSetManager: Finished task 119.0 in stage 12.0 (TID 209) in 725 ms on 10.0.0.133 (executor driver) (120/208) -26/04/01 05:56:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:41 INFO Executor: Finished task 120.0 in stage 12.0 (TID 210). 2251 bytes result sent to driver -26/04/01 05:56:41 INFO TaskSetManager: Starting task 124.0 in stage 12.0 (TID 214) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:41 INFO TaskSetManager: Finished task 120.0 in stage 12.0 (TID 210) in 736 ms on 10.0.0.133 (executor driver) (121/208) -26/04/01 05:56:41 INFO Executor: Running task 124.0 in stage 12.0 (TID 214) -26/04/01 05:56:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:41 INFO Executor: Finished task 121.0 in stage 12.0 (TID 211). 2251 bytes result sent to driver -26/04/01 05:56:41 INFO TaskSetManager: Starting task 125.0 in stage 12.0 (TID 215) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:41 INFO TaskSetManager: Finished task 121.0 in stage 12.0 (TID 211) in 705 ms on 10.0.0.133 (executor driver) (122/208) -26/04/01 05:56:41 INFO Executor: Running task 125.0 in stage 12.0 (TID 215) -26/04/01 05:56:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:41 INFO Executor: Finished task 122.0 in stage 12.0 (TID 212). 2251 bytes result sent to driver -26/04/01 05:56:41 INFO TaskSetManager: Starting task 126.0 in stage 12.0 (TID 216) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:41 INFO Executor: Running task 126.0 in stage 12.0 (TID 216) -26/04/01 05:56:41 INFO TaskSetManager: Finished task 122.0 in stage 12.0 (TID 212) in 704 ms on 10.0.0.133 (executor driver) (123/208) -26/04/01 05:56:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:41 INFO Executor: Finished task 123.0 in stage 12.0 (TID 213). 2251 bytes result sent to driver -26/04/01 05:56:41 INFO TaskSetManager: Starting task 127.0 in stage 12.0 (TID 217) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:41 INFO TaskSetManager: Finished task 123.0 in stage 12.0 (TID 213) in 699 ms on 10.0.0.133 (executor driver) (124/208) -26/04/01 05:56:41 INFO Executor: Running task 127.0 in stage 12.0 (TID 217) -26/04/01 05:56:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:41 INFO Executor: Finished task 124.0 in stage 12.0 (TID 214). 2251 bytes result sent to driver -26/04/01 05:56:41 INFO TaskSetManager: Starting task 128.0 in stage 12.0 (TID 218) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:41 INFO TaskSetManager: Finished task 124.0 in stage 12.0 (TID 214) in 714 ms on 10.0.0.133 (executor driver) (125/208) -26/04/01 05:56:41 INFO Executor: Running task 128.0 in stage 12.0 (TID 218) -26/04/01 05:56:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:41 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:42 INFO Executor: Finished task 125.0 in stage 12.0 (TID 215). 2251 bytes result sent to driver -26/04/01 05:56:42 INFO TaskSetManager: Starting task 129.0 in stage 12.0 (TID 219) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:42 INFO TaskSetManager: Finished task 125.0 in stage 12.0 (TID 215) in 710 ms on 10.0.0.133 (executor driver) (126/208) -26/04/01 05:56:42 INFO Executor: Running task 129.0 in stage 12.0 (TID 219) -26/04/01 05:56:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:42 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:42 INFO Executor: Finished task 126.0 in stage 12.0 (TID 216). 2251 bytes result sent to driver -26/04/01 05:56:42 INFO TaskSetManager: Starting task 130.0 in stage 12.0 (TID 220) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:42 INFO TaskSetManager: Finished task 126.0 in stage 12.0 (TID 216) in 717 ms on 10.0.0.133 (executor driver) (127/208) -26/04/01 05:56:42 INFO Executor: Running task 130.0 in stage 12.0 (TID 220) -26/04/01 05:56:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:42 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:42 INFO Executor: Finished task 127.0 in stage 12.0 (TID 217). 2251 bytes result sent to driver -26/04/01 05:56:42 INFO TaskSetManager: Starting task 131.0 in stage 12.0 (TID 221) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:42 INFO Executor: Running task 131.0 in stage 12.0 (TID 221) -26/04/01 05:56:42 INFO TaskSetManager: Finished task 127.0 in stage 12.0 (TID 217) in 739 ms on 10.0.0.133 (executor driver) (128/208) -26/04/01 05:56:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:42 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:42 INFO Executor: Finished task 128.0 in stage 12.0 (TID 218). 2251 bytes result sent to driver -26/04/01 05:56:42 INFO TaskSetManager: Starting task 132.0 in stage 12.0 (TID 222) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:42 INFO TaskSetManager: Finished task 128.0 in stage 12.0 (TID 218) in 710 ms on 10.0.0.133 (executor driver) (129/208) -26/04/01 05:56:42 INFO Executor: Running task 132.0 in stage 12.0 (TID 222) -26/04/01 05:56:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:42 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:43 INFO Executor: Finished task 129.0 in stage 12.0 (TID 219). 2251 bytes result sent to driver -26/04/01 05:56:43 INFO TaskSetManager: Starting task 133.0 in stage 12.0 (TID 223) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:43 INFO TaskSetManager: Finished task 129.0 in stage 12.0 (TID 219) in 730 ms on 10.0.0.133 (executor driver) (130/208) -26/04/01 05:56:43 INFO Executor: Running task 133.0 in stage 12.0 (TID 223) -26/04/01 05:56:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:43 INFO Executor: Finished task 130.0 in stage 12.0 (TID 220). 2251 bytes result sent to driver -26/04/01 05:56:43 INFO TaskSetManager: Starting task 134.0 in stage 12.0 (TID 224) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:43 INFO TaskSetManager: Finished task 130.0 in stage 12.0 (TID 220) in 702 ms on 10.0.0.133 (executor driver) (131/208) -26/04/01 05:56:43 INFO Executor: Running task 134.0 in stage 12.0 (TID 224) -26/04/01 05:56:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:43 INFO Executor: Finished task 131.0 in stage 12.0 (TID 221). 2251 bytes result sent to driver -26/04/01 05:56:43 INFO TaskSetManager: Starting task 135.0 in stage 12.0 (TID 225) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:43 INFO TaskSetManager: Finished task 131.0 in stage 12.0 (TID 221) in 696 ms on 10.0.0.133 (executor driver) (132/208) -26/04/01 05:56:43 INFO Executor: Running task 135.0 in stage 12.0 (TID 225) -26/04/01 05:56:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:43 INFO Executor: Finished task 132.0 in stage 12.0 (TID 222). 2251 bytes result sent to driver -26/04/01 05:56:43 INFO TaskSetManager: Starting task 136.0 in stage 12.0 (TID 226) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:43 INFO TaskSetManager: Finished task 132.0 in stage 12.0 (TID 222) in 709 ms on 10.0.0.133 (executor driver) (133/208) -26/04/01 05:56:43 INFO Executor: Running task 136.0 in stage 12.0 (TID 226) -26/04/01 05:56:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:43 INFO Executor: Finished task 133.0 in stage 12.0 (TID 223). 2251 bytes result sent to driver -26/04/01 05:56:43 INFO TaskSetManager: Starting task 137.0 in stage 12.0 (TID 227) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:43 INFO TaskSetManager: Finished task 133.0 in stage 12.0 (TID 223) in 714 ms on 10.0.0.133 (executor driver) (134/208) -26/04/01 05:56:43 INFO Executor: Running task 137.0 in stage 12.0 (TID 227) -26/04/01 05:56:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:43 INFO Executor: Finished task 134.0 in stage 12.0 (TID 224). 2251 bytes result sent to driver -26/04/01 05:56:43 INFO TaskSetManager: Starting task 138.0 in stage 12.0 (TID 228) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:43 INFO TaskSetManager: Finished task 134.0 in stage 12.0 (TID 224) in 708 ms on 10.0.0.133 (executor driver) (135/208) -26/04/01 05:56:43 INFO Executor: Running task 138.0 in stage 12.0 (TID 228) -26/04/01 05:56:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:43 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:44 INFO Executor: Finished task 135.0 in stage 12.0 (TID 225). 2251 bytes result sent to driver -26/04/01 05:56:44 INFO TaskSetManager: Starting task 139.0 in stage 12.0 (TID 229) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:44 INFO TaskSetManager: Finished task 135.0 in stage 12.0 (TID 225) in 708 ms on 10.0.0.133 (executor driver) (136/208) -26/04/01 05:56:44 INFO Executor: Running task 139.0 in stage 12.0 (TID 229) -26/04/01 05:56:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:44 INFO Executor: Finished task 136.0 in stage 12.0 (TID 226). 2251 bytes result sent to driver -26/04/01 05:56:44 INFO TaskSetManager: Starting task 140.0 in stage 12.0 (TID 230) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:44 INFO TaskSetManager: Finished task 136.0 in stage 12.0 (TID 226) in 744 ms on 10.0.0.133 (executor driver) (137/208) -26/04/01 05:56:44 INFO Executor: Running task 140.0 in stage 12.0 (TID 230) -26/04/01 05:56:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:44 INFO Executor: Finished task 137.0 in stage 12.0 (TID 227). 2251 bytes result sent to driver -26/04/01 05:56:44 INFO TaskSetManager: Starting task 141.0 in stage 12.0 (TID 231) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:44 INFO Executor: Running task 141.0 in stage 12.0 (TID 231) -26/04/01 05:56:44 INFO TaskSetManager: Finished task 137.0 in stage 12.0 (TID 227) in 730 ms on 10.0.0.133 (executor driver) (138/208) -26/04/01 05:56:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:44 INFO Executor: Finished task 138.0 in stage 12.0 (TID 228). 2251 bytes result sent to driver -26/04/01 05:56:44 INFO TaskSetManager: Starting task 142.0 in stage 12.0 (TID 232) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:44 INFO TaskSetManager: Finished task 138.0 in stage 12.0 (TID 228) in 745 ms on 10.0.0.133 (executor driver) (139/208) -26/04/01 05:56:44 INFO Executor: Running task 142.0 in stage 12.0 (TID 232) -26/04/01 05:56:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:44 INFO Executor: Finished task 139.0 in stage 12.0 (TID 229). 2251 bytes result sent to driver -26/04/01 05:56:44 INFO TaskSetManager: Starting task 143.0 in stage 12.0 (TID 233) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:44 INFO Executor: Running task 143.0 in stage 12.0 (TID 233) -26/04/01 05:56:44 INFO TaskSetManager: Finished task 139.0 in stage 12.0 (TID 229) in 714 ms on 10.0.0.133 (executor driver) (140/208) -26/04/01 05:56:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:44 INFO Executor: Finished task 140.0 in stage 12.0 (TID 230). 2251 bytes result sent to driver -26/04/01 05:56:44 INFO TaskSetManager: Starting task 144.0 in stage 12.0 (TID 234) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:44 INFO TaskSetManager: Finished task 140.0 in stage 12.0 (TID 230) in 714 ms on 10.0.0.133 (executor driver) (141/208) -26/04/01 05:56:44 INFO Executor: Running task 144.0 in stage 12.0 (TID 234) -26/04/01 05:56:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:44 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:45 INFO Executor: Finished task 141.0 in stage 12.0 (TID 231). 2251 bytes result sent to driver -26/04/01 05:56:45 INFO TaskSetManager: Starting task 145.0 in stage 12.0 (TID 235) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:45 INFO TaskSetManager: Finished task 141.0 in stage 12.0 (TID 231) in 723 ms on 10.0.0.133 (executor driver) (142/208) -26/04/01 05:56:45 INFO Executor: Running task 145.0 in stage 12.0 (TID 235) -26/04/01 05:56:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:45 INFO Executor: Finished task 142.0 in stage 12.0 (TID 232). 2251 bytes result sent to driver -26/04/01 05:56:45 INFO TaskSetManager: Starting task 146.0 in stage 12.0 (TID 236) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:45 INFO TaskSetManager: Finished task 142.0 in stage 12.0 (TID 232) in 733 ms on 10.0.0.133 (executor driver) (143/208) -26/04/01 05:56:45 INFO Executor: Running task 146.0 in stage 12.0 (TID 236) -26/04/01 05:56:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:45 INFO Executor: Finished task 143.0 in stage 12.0 (TID 233). 2251 bytes result sent to driver -26/04/01 05:56:45 INFO TaskSetManager: Starting task 147.0 in stage 12.0 (TID 237) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:45 INFO TaskSetManager: Finished task 143.0 in stage 12.0 (TID 233) in 739 ms on 10.0.0.133 (executor driver) (144/208) -26/04/01 05:56:45 INFO Executor: Running task 147.0 in stage 12.0 (TID 237) -26/04/01 05:56:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:45 INFO Executor: Finished task 144.0 in stage 12.0 (TID 234). 2251 bytes result sent to driver -26/04/01 05:56:45 INFO TaskSetManager: Starting task 148.0 in stage 12.0 (TID 238) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:45 INFO TaskSetManager: Finished task 144.0 in stage 12.0 (TID 234) in 717 ms on 10.0.0.133 (executor driver) (145/208) -26/04/01 05:56:45 INFO Executor: Running task 148.0 in stage 12.0 (TID 238) -26/04/01 05:56:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:45 INFO Executor: Finished task 145.0 in stage 12.0 (TID 235). 2251 bytes result sent to driver -26/04/01 05:56:45 INFO TaskSetManager: Starting task 149.0 in stage 12.0 (TID 239) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:45 INFO TaskSetManager: Finished task 145.0 in stage 12.0 (TID 235) in 745 ms on 10.0.0.133 (executor driver) (146/208) -26/04/01 05:56:45 INFO Executor: Running task 149.0 in stage 12.0 (TID 239) -26/04/01 05:56:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:45 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:46 INFO Executor: Finished task 146.0 in stage 12.0 (TID 236). 2251 bytes result sent to driver -26/04/01 05:56:46 INFO TaskSetManager: Starting task 150.0 in stage 12.0 (TID 240) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:46 INFO TaskSetManager: Finished task 146.0 in stage 12.0 (TID 236) in 737 ms on 10.0.0.133 (executor driver) (147/208) -26/04/01 05:56:46 INFO Executor: Running task 150.0 in stage 12.0 (TID 240) -26/04/01 05:56:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:46 INFO Executor: Finished task 147.0 in stage 12.0 (TID 237). 2251 bytes result sent to driver -26/04/01 05:56:46 INFO TaskSetManager: Starting task 151.0 in stage 12.0 (TID 241) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:46 INFO TaskSetManager: Finished task 147.0 in stage 12.0 (TID 237) in 744 ms on 10.0.0.133 (executor driver) (148/208) -26/04/01 05:56:46 INFO Executor: Running task 151.0 in stage 12.0 (TID 241) -26/04/01 05:56:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:46 INFO Executor: Finished task 148.0 in stage 12.0 (TID 238). 2251 bytes result sent to driver -26/04/01 05:56:46 INFO TaskSetManager: Starting task 152.0 in stage 12.0 (TID 242) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:46 INFO TaskSetManager: Finished task 148.0 in stage 12.0 (TID 238) in 723 ms on 10.0.0.133 (executor driver) (149/208) -26/04/01 05:56:46 INFO Executor: Running task 152.0 in stage 12.0 (TID 242) -26/04/01 05:56:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:46 INFO Executor: Finished task 149.0 in stage 12.0 (TID 239). 2251 bytes result sent to driver -26/04/01 05:56:46 INFO TaskSetManager: Starting task 153.0 in stage 12.0 (TID 243) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:46 INFO TaskSetManager: Finished task 149.0 in stage 12.0 (TID 239) in 704 ms on 10.0.0.133 (executor driver) (150/208) -26/04/01 05:56:46 INFO Executor: Running task 153.0 in stage 12.0 (TID 243) -26/04/01 05:56:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:46 INFO Executor: Finished task 150.0 in stage 12.0 (TID 240). 2251 bytes result sent to driver -26/04/01 05:56:46 INFO TaskSetManager: Starting task 154.0 in stage 12.0 (TID 244) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:46 INFO TaskSetManager: Finished task 150.0 in stage 12.0 (TID 240) in 704 ms on 10.0.0.133 (executor driver) (151/208) -26/04/01 05:56:46 INFO Executor: Running task 154.0 in stage 12.0 (TID 244) -26/04/01 05:56:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:46 INFO Executor: Finished task 151.0 in stage 12.0 (TID 241). 2251 bytes result sent to driver -26/04/01 05:56:46 INFO TaskSetManager: Starting task 155.0 in stage 12.0 (TID 245) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:46 INFO TaskSetManager: Finished task 151.0 in stage 12.0 (TID 241) in 699 ms on 10.0.0.133 (executor driver) (152/208) -26/04/01 05:56:46 INFO Executor: Running task 155.0 in stage 12.0 (TID 245) -26/04/01 05:56:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:46 INFO Executor: Finished task 152.0 in stage 12.0 (TID 242). 2251 bytes result sent to driver -26/04/01 05:56:46 INFO TaskSetManager: Starting task 156.0 in stage 12.0 (TID 246) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:46 INFO TaskSetManager: Finished task 152.0 in stage 12.0 (TID 242) in 705 ms on 10.0.0.133 (executor driver) (153/208) -26/04/01 05:56:46 INFO Executor: Running task 156.0 in stage 12.0 (TID 246) -26/04/01 05:56:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:46 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:47 INFO Executor: Finished task 153.0 in stage 12.0 (TID 243). 2251 bytes result sent to driver -26/04/01 05:56:47 INFO TaskSetManager: Starting task 157.0 in stage 12.0 (TID 247) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:47 INFO TaskSetManager: Finished task 153.0 in stage 12.0 (TID 243) in 735 ms on 10.0.0.133 (executor driver) (154/208) -26/04/01 05:56:47 INFO Executor: Running task 157.0 in stage 12.0 (TID 247) -26/04/01 05:56:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:47 INFO Executor: Finished task 154.0 in stage 12.0 (TID 244). 2251 bytes result sent to driver -26/04/01 05:56:47 INFO TaskSetManager: Starting task 158.0 in stage 12.0 (TID 248) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:47 INFO TaskSetManager: Finished task 154.0 in stage 12.0 (TID 244) in 724 ms on 10.0.0.133 (executor driver) (155/208) -26/04/01 05:56:47 INFO Executor: Running task 158.0 in stage 12.0 (TID 248) -26/04/01 05:56:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:47 INFO Executor: Finished task 155.0 in stage 12.0 (TID 245). 2251 bytes result sent to driver -26/04/01 05:56:47 INFO TaskSetManager: Starting task 159.0 in stage 12.0 (TID 249) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:47 INFO TaskSetManager: Finished task 155.0 in stage 12.0 (TID 245) in 743 ms on 10.0.0.133 (executor driver) (156/208) -26/04/01 05:56:47 INFO Executor: Running task 159.0 in stage 12.0 (TID 249) -26/04/01 05:56:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:47 INFO Executor: Finished task 156.0 in stage 12.0 (TID 246). 2251 bytes result sent to driver -26/04/01 05:56:47 INFO TaskSetManager: Starting task 160.0 in stage 12.0 (TID 250) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:47 INFO Executor: Running task 160.0 in stage 12.0 (TID 250) -26/04/01 05:56:47 INFO TaskSetManager: Finished task 156.0 in stage 12.0 (TID 246) in 759 ms on 10.0.0.133 (executor driver) (157/208) -26/04/01 05:56:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:47 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:48 INFO Executor: Finished task 157.0 in stage 12.0 (TID 247). 2251 bytes result sent to driver -26/04/01 05:56:48 INFO TaskSetManager: Starting task 161.0 in stage 12.0 (TID 251) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:48 INFO TaskSetManager: Finished task 157.0 in stage 12.0 (TID 247) in 733 ms on 10.0.0.133 (executor driver) (158/208) -26/04/01 05:56:48 INFO Executor: Running task 161.0 in stage 12.0 (TID 251) -26/04/01 05:56:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:48 INFO Executor: Finished task 158.0 in stage 12.0 (TID 248). 2251 bytes result sent to driver -26/04/01 05:56:48 INFO TaskSetManager: Starting task 162.0 in stage 12.0 (TID 252) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:48 INFO TaskSetManager: Finished task 158.0 in stage 12.0 (TID 248) in 733 ms on 10.0.0.133 (executor driver) (159/208) -26/04/01 05:56:48 INFO Executor: Running task 162.0 in stage 12.0 (TID 252) -26/04/01 05:56:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:48 INFO Executor: Finished task 159.0 in stage 12.0 (TID 249). 2251 bytes result sent to driver -26/04/01 05:56:48 INFO TaskSetManager: Starting task 163.0 in stage 12.0 (TID 253) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:48 INFO TaskSetManager: Finished task 159.0 in stage 12.0 (TID 249) in 684 ms on 10.0.0.133 (executor driver) (160/208) -26/04/01 05:56:48 INFO Executor: Running task 163.0 in stage 12.0 (TID 253) -26/04/01 05:56:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:48 INFO Executor: Finished task 160.0 in stage 12.0 (TID 250). 2251 bytes result sent to driver -26/04/01 05:56:48 INFO TaskSetManager: Starting task 164.0 in stage 12.0 (TID 254) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:48 INFO TaskSetManager: Finished task 160.0 in stage 12.0 (TID 250) in 705 ms on 10.0.0.133 (executor driver) (161/208) -26/04/01 05:56:48 INFO Executor: Running task 164.0 in stage 12.0 (TID 254) -26/04/01 05:56:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:48 INFO Executor: Finished task 161.0 in stage 12.0 (TID 251). 2251 bytes result sent to driver -26/04/01 05:56:48 INFO TaskSetManager: Starting task 165.0 in stage 12.0 (TID 255) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:48 INFO Executor: Running task 165.0 in stage 12.0 (TID 255) -26/04/01 05:56:48 INFO TaskSetManager: Finished task 161.0 in stage 12.0 (TID 251) in 703 ms on 10.0.0.133 (executor driver) (162/208) -26/04/01 05:56:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:48 INFO Executor: Finished task 162.0 in stage 12.0 (TID 252). 2251 bytes result sent to driver -26/04/01 05:56:48 INFO TaskSetManager: Starting task 166.0 in stage 12.0 (TID 256) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:48 INFO Executor: Running task 166.0 in stage 12.0 (TID 256) -26/04/01 05:56:48 INFO TaskSetManager: Finished task 162.0 in stage 12.0 (TID 252) in 704 ms on 10.0.0.133 (executor driver) (163/208) -26/04/01 05:56:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:48 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:49 INFO Executor: Finished task 163.0 in stage 12.0 (TID 253). 2251 bytes result sent to driver -26/04/01 05:56:49 INFO TaskSetManager: Starting task 167.0 in stage 12.0 (TID 257) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:49 INFO TaskSetManager: Finished task 163.0 in stage 12.0 (TID 253) in 732 ms on 10.0.0.133 (executor driver) (164/208) -26/04/01 05:56:49 INFO Executor: Running task 167.0 in stage 12.0 (TID 257) -26/04/01 05:56:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:49 INFO Executor: Finished task 164.0 in stage 12.0 (TID 254). 2251 bytes result sent to driver -26/04/01 05:56:49 INFO TaskSetManager: Starting task 168.0 in stage 12.0 (TID 258) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:49 INFO Executor: Running task 168.0 in stage 12.0 (TID 258) -26/04/01 05:56:49 INFO TaskSetManager: Finished task 164.0 in stage 12.0 (TID 254) in 726 ms on 10.0.0.133 (executor driver) (165/208) -26/04/01 05:56:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:49 INFO Executor: Finished task 165.0 in stage 12.0 (TID 255). 2251 bytes result sent to driver -26/04/01 05:56:49 INFO TaskSetManager: Starting task 169.0 in stage 12.0 (TID 259) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:49 INFO Executor: Running task 169.0 in stage 12.0 (TID 259) -26/04/01 05:56:49 INFO TaskSetManager: Finished task 165.0 in stage 12.0 (TID 255) in 722 ms on 10.0.0.133 (executor driver) (166/208) -26/04/01 05:56:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:49 INFO Executor: Finished task 166.0 in stage 12.0 (TID 256). 2251 bytes result sent to driver -26/04/01 05:56:49 INFO TaskSetManager: Starting task 170.0 in stage 12.0 (TID 260) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:49 INFO TaskSetManager: Finished task 166.0 in stage 12.0 (TID 256) in 705 ms on 10.0.0.133 (executor driver) (167/208) -26/04/01 05:56:49 INFO Executor: Running task 170.0 in stage 12.0 (TID 260) -26/04/01 05:56:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:49 INFO Executor: Finished task 167.0 in stage 12.0 (TID 257). 2251 bytes result sent to driver -26/04/01 05:56:49 INFO TaskSetManager: Starting task 171.0 in stage 12.0 (TID 261) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:49 INFO Executor: Running task 171.0 in stage 12.0 (TID 261) -26/04/01 05:56:49 INFO TaskSetManager: Finished task 167.0 in stage 12.0 (TID 257) in 695 ms on 10.0.0.133 (executor driver) (168/208) -26/04/01 05:56:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:49 INFO Executor: Finished task 168.0 in stage 12.0 (TID 258). 2251 bytes result sent to driver -26/04/01 05:56:49 INFO TaskSetManager: Starting task 172.0 in stage 12.0 (TID 262) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:49 INFO TaskSetManager: Finished task 168.0 in stage 12.0 (TID 258) in 812 ms on 10.0.0.133 (executor driver) (169/208) -26/04/01 05:56:49 INFO Executor: Running task 172.0 in stage 12.0 (TID 262) -26/04/01 05:56:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:49 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:50 INFO Executor: Finished task 169.0 in stage 12.0 (TID 259). 2251 bytes result sent to driver -26/04/01 05:56:50 INFO TaskSetManager: Starting task 173.0 in stage 12.0 (TID 263) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:50 INFO TaskSetManager: Finished task 169.0 in stage 12.0 (TID 259) in 714 ms on 10.0.0.133 (executor driver) (170/208) -26/04/01 05:56:50 INFO Executor: Running task 173.0 in stage 12.0 (TID 263) -26/04/01 05:56:50 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:50 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:50 INFO Executor: Finished task 170.0 in stage 12.0 (TID 260). 2251 bytes result sent to driver -26/04/01 05:56:50 INFO TaskSetManager: Starting task 174.0 in stage 12.0 (TID 264) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:50 INFO TaskSetManager: Finished task 170.0 in stage 12.0 (TID 260) in 719 ms on 10.0.0.133 (executor driver) (171/208) -26/04/01 05:56:50 INFO Executor: Running task 174.0 in stage 12.0 (TID 264) -26/04/01 05:56:50 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:50 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:50 INFO Executor: Finished task 171.0 in stage 12.0 (TID 261). 2251 bytes result sent to driver -26/04/01 05:56:50 INFO TaskSetManager: Starting task 175.0 in stage 12.0 (TID 265) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:50 INFO Executor: Running task 175.0 in stage 12.0 (TID 265) -26/04/01 05:56:50 INFO TaskSetManager: Finished task 171.0 in stage 12.0 (TID 261) in 727 ms on 10.0.0.133 (executor driver) (172/208) -26/04/01 05:56:50 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:50 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:50 INFO Executor: Finished task 172.0 in stage 12.0 (TID 262). 2251 bytes result sent to driver -26/04/01 05:56:50 INFO TaskSetManager: Starting task 176.0 in stage 12.0 (TID 266) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:50 INFO TaskSetManager: Finished task 172.0 in stage 12.0 (TID 262) in 704 ms on 10.0.0.133 (executor driver) (173/208) -26/04/01 05:56:50 INFO Executor: Running task 176.0 in stage 12.0 (TID 266) -26/04/01 05:56:50 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:50 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:50 INFO Executor: Finished task 173.0 in stage 12.0 (TID 263). 2251 bytes result sent to driver -26/04/01 05:56:50 INFO TaskSetManager: Starting task 177.0 in stage 12.0 (TID 267) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:50 INFO Executor: Running task 177.0 in stage 12.0 (TID 267) -26/04/01 05:56:50 INFO TaskSetManager: Finished task 173.0 in stage 12.0 (TID 263) in 703 ms on 10.0.0.133 (executor driver) (174/208) -26/04/01 05:56:50 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:50 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:51 INFO Executor: Finished task 174.0 in stage 12.0 (TID 264). 2251 bytes result sent to driver -26/04/01 05:56:51 INFO TaskSetManager: Starting task 178.0 in stage 12.0 (TID 268) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:51 INFO Executor: Running task 178.0 in stage 12.0 (TID 268) -26/04/01 05:56:51 INFO TaskSetManager: Finished task 174.0 in stage 12.0 (TID 264) in 728 ms on 10.0.0.133 (executor driver) (175/208) -26/04/01 05:56:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:51 INFO Executor: Finished task 175.0 in stage 12.0 (TID 265). 2251 bytes result sent to driver -26/04/01 05:56:51 INFO TaskSetManager: Starting task 179.0 in stage 12.0 (TID 269) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9601 bytes) -26/04/01 05:56:51 INFO TaskSetManager: Finished task 175.0 in stage 12.0 (TID 265) in 736 ms on 10.0.0.133 (executor driver) (176/208) -26/04/01 05:56:51 INFO Executor: Running task 179.0 in stage 12.0 (TID 269) -26/04/01 05:56:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:51 INFO Executor: Finished task 176.0 in stage 12.0 (TID 266). 2251 bytes result sent to driver -26/04/01 05:56:51 INFO TaskSetManager: Starting task 180.0 in stage 12.0 (TID 270) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:51 INFO TaskSetManager: Finished task 176.0 in stage 12.0 (TID 266) in 754 ms on 10.0.0.133 (executor driver) (177/208) -26/04/01 05:56:51 INFO Executor: Running task 180.0 in stage 12.0 (TID 270) -26/04/01 05:56:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:51 INFO Executor: Finished task 177.0 in stage 12.0 (TID 267). 2251 bytes result sent to driver -26/04/01 05:56:51 INFO TaskSetManager: Starting task 181.0 in stage 12.0 (TID 271) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:51 INFO Executor: Running task 181.0 in stage 12.0 (TID 271) -26/04/01 05:56:51 INFO TaskSetManager: Finished task 177.0 in stage 12.0 (TID 267) in 726 ms on 10.0.0.133 (executor driver) (178/208) -26/04/01 05:56:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:51 INFO Executor: Finished task 178.0 in stage 12.0 (TID 268). 2251 bytes result sent to driver -26/04/01 05:56:51 INFO TaskSetManager: Starting task 182.0 in stage 12.0 (TID 272) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:51 INFO Executor: Running task 182.0 in stage 12.0 (TID 272) -26/04/01 05:56:51 INFO TaskSetManager: Finished task 178.0 in stage 12.0 (TID 268) in 731 ms on 10.0.0.133 (executor driver) (179/208) -26/04/01 05:56:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:51 INFO Executor: Finished task 179.0 in stage 12.0 (TID 269). 2251 bytes result sent to driver -26/04/01 05:56:51 INFO TaskSetManager: Starting task 183.0 in stage 12.0 (TID 273) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:51 INFO TaskSetManager: Finished task 179.0 in stage 12.0 (TID 269) in 725 ms on 10.0.0.133 (executor driver) (180/208) -26/04/01 05:56:51 INFO Executor: Running task 183.0 in stage 12.0 (TID 273) -26/04/01 05:56:51 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:51 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:52 INFO Executor: Finished task 180.0 in stage 12.0 (TID 270). 2251 bytes result sent to driver -26/04/01 05:56:52 INFO TaskSetManager: Starting task 184.0 in stage 12.0 (TID 274) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:52 INFO TaskSetManager: Finished task 180.0 in stage 12.0 (TID 270) in 707 ms on 10.0.0.133 (executor driver) (181/208) -26/04/01 05:56:52 INFO Executor: Running task 184.0 in stage 12.0 (TID 274) -26/04/01 05:56:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:52 INFO Executor: Finished task 181.0 in stage 12.0 (TID 271). 2251 bytes result sent to driver -26/04/01 05:56:52 INFO TaskSetManager: Starting task 185.0 in stage 12.0 (TID 275) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:52 INFO TaskSetManager: Finished task 181.0 in stage 12.0 (TID 271) in 741 ms on 10.0.0.133 (executor driver) (182/208) -26/04/01 05:56:52 INFO Executor: Running task 185.0 in stage 12.0 (TID 275) -26/04/01 05:56:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:52 INFO Executor: Finished task 182.0 in stage 12.0 (TID 272). 2251 bytes result sent to driver -26/04/01 05:56:52 INFO TaskSetManager: Starting task 186.0 in stage 12.0 (TID 276) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:52 INFO TaskSetManager: Finished task 182.0 in stage 12.0 (TID 272) in 717 ms on 10.0.0.133 (executor driver) (183/208) -26/04/01 05:56:52 INFO Executor: Running task 186.0 in stage 12.0 (TID 276) -26/04/01 05:56:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 05:56:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:52 INFO Executor: Finished task 183.0 in stage 12.0 (TID 273). 2251 bytes result sent to driver -26/04/01 05:56:52 INFO TaskSetManager: Starting task 187.0 in stage 12.0 (TID 277) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:52 INFO Executor: Running task 187.0 in stage 12.0 (TID 277) -26/04/01 05:56:52 INFO TaskSetManager: Finished task 183.0 in stage 12.0 (TID 273) in 754 ms on 10.0.0.133 (executor driver) (184/208) -26/04/01 05:56:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 05:56:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:52 INFO Executor: Finished task 184.0 in stage 12.0 (TID 274). 2251 bytes result sent to driver -26/04/01 05:56:52 INFO TaskSetManager: Starting task 188.0 in stage 12.0 (TID 278) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:52 INFO TaskSetManager: Finished task 184.0 in stage 12.0 (TID 274) in 738 ms on 10.0.0.133 (executor driver) (185/208) -26/04/01 05:56:52 INFO Executor: Running task 188.0 in stage 12.0 (TID 278) -26/04/01 05:56:52 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 05:56:52 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:53 INFO Executor: Finished task 185.0 in stage 12.0 (TID 275). 2251 bytes result sent to driver -26/04/01 05:56:53 INFO TaskSetManager: Starting task 189.0 in stage 12.0 (TID 279) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:53 INFO TaskSetManager: Finished task 185.0 in stage 12.0 (TID 275) in 724 ms on 10.0.0.133 (executor driver) (186/208) -26/04/01 05:56:53 INFO Executor: Running task 189.0 in stage 12.0 (TID 279) -26/04/01 05:56:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 05:56:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:53 INFO Executor: Finished task 186.0 in stage 12.0 (TID 276). 2251 bytes result sent to driver -26/04/01 05:56:53 INFO TaskSetManager: Starting task 190.0 in stage 12.0 (TID 280) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:53 INFO Executor: Running task 190.0 in stage 12.0 (TID 280) -26/04/01 05:56:53 INFO TaskSetManager: Finished task 186.0 in stage 12.0 (TID 276) in 725 ms on 10.0.0.133 (executor driver) (187/208) -26/04/01 05:56:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 05:56:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:53 INFO Executor: Finished task 187.0 in stage 12.0 (TID 277). 2251 bytes result sent to driver -26/04/01 05:56:53 INFO TaskSetManager: Starting task 191.0 in stage 12.0 (TID 281) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9602 bytes) -26/04/01 05:56:53 INFO TaskSetManager: Finished task 187.0 in stage 12.0 (TID 277) in 708 ms on 10.0.0.133 (executor driver) (188/208) -26/04/01 05:56:53 INFO Executor: Running task 191.0 in stage 12.0 (TID 281) -26/04/01 05:56:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 05:56:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:53 INFO Executor: Finished task 188.0 in stage 12.0 (TID 278). 2251 bytes result sent to driver -26/04/01 05:56:53 INFO TaskSetManager: Starting task 192.0 in stage 12.0 (TID 282) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:53 INFO TaskSetManager: Finished task 188.0 in stage 12.0 (TID 278) in 716 ms on 10.0.0.133 (executor driver) (189/208) -26/04/01 05:56:53 INFO Executor: Running task 192.0 in stage 12.0 (TID 282) -26/04/01 05:56:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 805306368-848649288, partition values: [empty row] -26/04/01 05:56:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 805306368-848594494, partition values: [empty row] -26/04/01 05:56:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:53 INFO Executor: Finished task 189.0 in stage 12.0 (TID 279). 2251 bytes result sent to driver -26/04/01 05:56:53 INFO TaskSetManager: Starting task 193.0 in stage 12.0 (TID 283) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:53 INFO TaskSetManager: Finished task 189.0 in stage 12.0 (TID 279) in 728 ms on 10.0.0.133 (executor driver) (190/208) -26/04/01 05:56:53 INFO Executor: Running task 193.0 in stage 12.0 (TID 283) -26/04/01 05:56:53 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 805306368-848563442, partition values: [empty row] -26/04/01 05:56:53 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO Executor: Finished task 190.0 in stage 12.0 (TID 280). 2251 bytes result sent to driver -26/04/01 05:56:54 INFO TaskSetManager: Starting task 194.0 in stage 12.0 (TID 284) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:54 INFO TaskSetManager: Finished task 190.0 in stage 12.0 (TID 280) in 715 ms on 10.0.0.133 (executor driver) (191/208) -26/04/01 05:56:54 INFO Executor: Running task 194.0 in stage 12.0 (TID 284) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 805306368-848496433, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO Executor: Finished task 192.0 in stage 12.0 (TID 282). 2251 bytes result sent to driver -26/04/01 05:56:54 INFO TaskSetManager: Starting task 195.0 in stage 12.0 (TID 285) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:54 INFO TaskSetManager: Finished task 192.0 in stage 12.0 (TID 282) in 467 ms on 10.0.0.133 (executor driver) (192/208) -26/04/01 05:56:54 INFO Executor: Running task 195.0 in stage 12.0 (TID 285) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 805306368-848463796, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO Executor: Finished task 191.0 in stage 12.0 (TID 281). 2251 bytes result sent to driver -26/04/01 05:56:54 INFO TaskSetManager: Starting task 196.0 in stage 12.0 (TID 286) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:54 INFO TaskSetManager: Finished task 191.0 in stage 12.0 (TID 281) in 732 ms on 10.0.0.133 (executor driver) (193/208) -26/04/01 05:56:54 INFO Executor: Running task 196.0 in stage 12.0 (TID 286) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 805306368-848432523, partition values: [empty row] -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 805306368-848521656, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 805306368-848489219, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 805306368-848442878, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 805306368-848423659, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO Executor: Finished task 193.0 in stage 12.0 (TID 283). 2251 bytes result sent to driver -26/04/01 05:56:54 INFO TaskSetManager: Starting task 197.0 in stage 12.0 (TID 287) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:54 INFO TaskSetManager: Finished task 193.0 in stage 12.0 (TID 283) in 469 ms on 10.0.0.133 (executor driver) (194/208) -26/04/01 05:56:54 INFO Executor: Running task 197.0 in stage 12.0 (TID 287) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 805306368-848422348, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO Executor: Finished task 194.0 in stage 12.0 (TID 284). 2251 bytes result sent to driver -26/04/01 05:56:54 INFO TaskSetManager: Starting task 198.0 in stage 12.0 (TID 288) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:54 INFO TaskSetManager: Finished task 194.0 in stage 12.0 (TID 284) in 484 ms on 10.0.0.133 (executor driver) (195/208) -26/04/01 05:56:54 INFO Executor: Running task 198.0 in stage 12.0 (TID 288) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 805306368-848399844, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO Executor: Finished task 195.0 in stage 12.0 (TID 285). 2251 bytes result sent to driver -26/04/01 05:56:54 INFO TaskSetManager: Starting task 199.0 in stage 12.0 (TID 289) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:54 INFO TaskSetManager: Finished task 195.0 in stage 12.0 (TID 285) in 481 ms on 10.0.0.133 (executor driver) (196/208) -26/04/01 05:56:54 INFO Executor: Running task 199.0 in stage 12.0 (TID 289) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 805306368-848378208, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 805306368-848411809, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO Executor: Finished task 196.0 in stage 12.0 (TID 286). 2251 bytes result sent to driver -26/04/01 05:56:54 INFO TaskSetManager: Starting task 200.0 in stage 12.0 (TID 290) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9708 bytes) -26/04/01 05:56:54 INFO TaskSetManager: Finished task 196.0 in stage 12.0 (TID 286) in 478 ms on 10.0.0.133 (executor driver) (197/208) -26/04/01 05:56:54 INFO Executor: Running task 200.0 in stage 12.0 (TID 290) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 805306368-848344114, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 805306368-848379799, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 805306368-848348606, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 805306368-848338051, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO Executor: Finished task 197.0 in stage 12.0 (TID 287). 2251 bytes result sent to driver -26/04/01 05:56:54 INFO TaskSetManager: Starting task 201.0 in stage 12.0 (TID 291) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:54 INFO TaskSetManager: Finished task 197.0 in stage 12.0 (TID 287) in 440 ms on 10.0.0.133 (executor driver) (198/208) -26/04/01 05:56:54 INFO Executor: Running task 201.0 in stage 12.0 (TID 291) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 805306368-848329366, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO Executor: Finished task 198.0 in stage 12.0 (TID 288). 2251 bytes result sent to driver -26/04/01 05:56:54 INFO TaskSetManager: Starting task 202.0 in stage 12.0 (TID 292) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:54 INFO TaskSetManager: Finished task 198.0 in stage 12.0 (TID 288) in 441 ms on 10.0.0.133 (executor driver) (199/208) -26/04/01 05:56:54 INFO Executor: Running task 202.0 in stage 12.0 (TID 292) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 805306368-848328570, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:54 INFO Executor: Finished task 199.0 in stage 12.0 (TID 289). 2251 bytes result sent to driver -26/04/01 05:56:54 INFO TaskSetManager: Starting task 203.0 in stage 12.0 (TID 293) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:54 INFO TaskSetManager: Finished task 199.0 in stage 12.0 (TID 289) in 446 ms on 10.0.0.133 (executor driver) (200/208) -26/04/01 05:56:54 INFO Executor: Running task 203.0 in stage 12.0 (TID 293) -26/04/01 05:56:54 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 805306368-848301737, partition values: [empty row] -26/04/01 05:56:54 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 805306368-848329327, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:55 INFO Executor: Finished task 200.0 in stage 12.0 (TID 290). 2251 bytes result sent to driver -26/04/01 05:56:55 INFO TaskSetManager: Starting task 204.0 in stage 12.0 (TID 294) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:55 INFO TaskSetManager: Finished task 200.0 in stage 12.0 (TID 290) in 441 ms on 10.0.0.133 (executor driver) (201/208) -26/04/01 05:56:55 INFO Executor: Running task 204.0 in stage 12.0 (TID 294) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 805306368-848263904, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 805306368-848306153, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 805306368-848278425, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 805306368-848247851, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:55 INFO Executor: Finished task 201.0 in stage 12.0 (TID 291). 2251 bytes result sent to driver -26/04/01 05:56:55 INFO TaskSetManager: Starting task 205.0 in stage 12.0 (TID 295) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9710 bytes) -26/04/01 05:56:55 INFO TaskSetManager: Finished task 201.0 in stage 12.0 (TID 291) in 439 ms on 10.0.0.133 (executor driver) (202/208) -26/04/01 05:56:55 INFO Executor: Running task 205.0 in stage 12.0 (TID 295) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 805306368-848235021, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:55 INFO Executor: Finished task 202.0 in stage 12.0 (TID 292). 2251 bytes result sent to driver -26/04/01 05:56:55 INFO TaskSetManager: Starting task 206.0 in stage 12.0 (TID 296) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:55 INFO TaskSetManager: Finished task 202.0 in stage 12.0 (TID 292) in 447 ms on 10.0.0.133 (executor driver) (203/208) -26/04/01 05:56:55 INFO Executor: Running task 206.0 in stage 12.0 (TID 296) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 805306368-848107384, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:55 INFO Executor: Finished task 203.0 in stage 12.0 (TID 293). 2251 bytes result sent to driver -26/04/01 05:56:55 INFO TaskSetManager: Starting task 207.0 in stage 12.0 (TID 297) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9709 bytes) -26/04/01 05:56:55 INFO TaskSetManager: Finished task 203.0 in stage 12.0 (TID 293) in 459 ms on 10.0.0.133 (executor driver) (204/208) -26/04/01 05:56:55 INFO Executor: Running task 207.0 in stage 12.0 (TID 297) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 805306368-848005421, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 805306368-848234776, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:55 INFO Executor: Finished task 204.0 in stage 12.0 (TID 294). 2251 bytes result sent to driver -26/04/01 05:56:55 INFO TaskSetManager: Starting task 0.0 in stage 13.0 (TID 298) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 10355 bytes) -26/04/01 05:56:55 INFO Executor: Running task 0.0 in stage 13.0 (TID 298) -26/04/01 05:56:55 INFO TaskSetManager: Finished task 204.0 in stage 12.0 (TID 294) in 468 ms on 10.0.0.133 (executor driver) (205/208) -26/04/01 05:56:55 INFO CodeGenerator: Code generated in 3.538958 ms -26/04/01 05:56:55 INFO CodeGenerator: Code generated in 2.368917 ms -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.9.parquet, range: 0-2801305, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.21.parquet, range: 0-2800748, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.13.parquet, range: 0-2800604, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.1.parquet, range: 0-2799999, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.31.parquet, range: 0-2799730, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.30.parquet, range: 0-2799509, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.25.parquet, range: 0-2798499, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.6.parquet, range: 0-2798414, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 805306368-848050794, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:55 INFO Executor: Finished task 0.0 in stage 13.0 (TID 298). 2208 bytes result sent to driver -26/04/01 05:56:55 INFO TaskSetManager: Starting task 1.0 in stage 13.0 (TID 299) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 10357 bytes) -26/04/01 05:56:55 INFO TaskSetManager: Finished task 0.0 in stage 13.0 (TID 298) in 88 ms on 10.0.0.133 (executor driver) (1/4) -26/04/01 05:56:55 INFO Executor: Running task 1.0 in stage 13.0 (TID 299) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.11.parquet, range: 0-2798396, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.18.parquet, range: 0-2797632, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.2.parquet, range: 0-2797502, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.12.parquet, range: 0-2797316, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.26.parquet, range: 0-2797291, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.14.parquet, range: 0-2797103, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.22.parquet, range: 0-2797014, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.17.parquet, range: 0-2795731, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 805306368-847699715, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(l_orderkey, null), noteq(l_suppkey, null)) -26/04/01 05:56:55 INFO Executor: Finished task 1.0 in stage 13.0 (TID 299). 2208 bytes result sent to driver -26/04/01 05:56:55 INFO TaskSetManager: Starting task 2.0 in stage 13.0 (TID 300) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 10357 bytes) -26/04/01 05:56:55 INFO TaskSetManager: Finished task 1.0 in stage 13.0 (TID 299) in 89 ms on 10.0.0.133 (executor driver) (2/4) -26/04/01 05:56:55 INFO Executor: Running task 2.0 in stage 13.0 (TID 300) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.27.parquet, range: 0-2795177, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.10.parquet, range: 0-2795129, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO Executor: Finished task 205.0 in stage 12.0 (TID 295). 2251 bytes result sent to driver -26/04/01 05:56:55 INFO TaskSetManager: Starting task 3.0 in stage 13.0 (TID 301) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 10354 bytes) -26/04/01 05:56:55 INFO TaskSetManager: Finished task 205.0 in stage 12.0 (TID 295) in 456 ms on 10.0.0.133 (executor driver) (206/208) -26/04/01 05:56:55 INFO Executor: Running task 3.0 in stage 13.0 (TID 301) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.32.parquet, range: 0-2792572, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.19.parquet, range: 0-2794808, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.15.parquet, range: 0-2794714, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.3.parquet, range: 0-2794406, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.16.parquet, range: 0-2794298, partition values: [empty row] -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.5.parquet, range: 0-2792421, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.24.parquet, range: 0-2793929, partition values: [empty row] -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.7.parquet, range: 0-2791953, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.23.parquet, range: 0-2792937, partition values: [empty row] -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.29.parquet, range: 0-2791555, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.20.parquet, range: 0-2791117, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.28.parquet, range: 0-2790847, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.8.parquet, range: 0-2790608, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.4.parquet, range: 0-2789514, partition values: [empty row] -26/04/01 05:56:55 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 05:56:55 INFO Executor: Finished task 2.0 in stage 13.0 (TID 300). 2251 bytes result sent to driver -26/04/01 05:56:55 INFO TaskSetManager: Starting task 0.0 in stage 15.0 (TID 302) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8999 bytes) -26/04/01 05:56:55 INFO Executor: Running task 0.0 in stage 15.0 (TID 302) -26/04/01 05:56:55 INFO TaskSetManager: Finished task 2.0 in stage 13.0 (TID 300) in 93 ms on 10.0.0.133 (executor driver) (3/4) -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 1 (5.2 KiB) non-empty blocks including 1 (5.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms -26/04/01 05:56:55 INFO Executor: Finished task 3.0 in stage 13.0 (TID 301). 2251 bytes result sent to driver -26/04/01 05:56:55 INFO TaskSetManager: Starting task 0.0 in stage 18.0 (TID 303) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9270 bytes) -26/04/01 05:56:55 INFO TaskSetManager: Finished task 3.0 in stage 13.0 (TID 301) in 87 ms on 10.0.0.133 (executor driver) (4/4) -26/04/01 05:56:55 INFO Executor: Running task 0.0 in stage 18.0 (TID 303) -26/04/01 05:56:55 INFO TaskSchedulerImpl: Removed TaskSet 13.0, whose tasks have all completed, from pool -26/04/01 05:56:55 INFO DAGScheduler: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 42.814 s -26/04/01 05:56:55 INFO DAGScheduler: looking for newly runnable stages -26/04/01 05:56:55 INFO DAGScheduler: running: Set(ResultStage 15, ShuffleMapStage 12, ShuffleMapStage 18) -26/04/01 05:56:55 INFO DAGScheduler: waiting: Set() -26/04/01 05:56:55 INFO DAGScheduler: failed: Set() -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 16 (20.5 MiB) non-empty blocks including 16 (20.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:55 INFO Executor: Finished task 0.0 in stage 15.0 (TID 302). 10188 bytes result sent to driver -26/04/01 05:56:55 INFO TaskSetManager: Starting task 1.0 in stage 18.0 (TID 304) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9270 bytes) -26/04/01 05:56:55 INFO Executor: Running task 1.0 in stage 18.0 (TID 304) -26/04/01 05:56:55 INFO TaskSetManager: Finished task 0.0 in stage 15.0 (TID 302) in 26 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:56:55 INFO TaskSchedulerImpl: Removed TaskSet 15.0, whose tasks have all completed, from pool -26/04/01 05:56:55 INFO DAGScheduler: ResultStage 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 41.361 s -26/04/01 05:56:55 INFO DAGScheduler: Job 14 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:56:55 INFO TaskSchedulerImpl: Killing all running tasks in stage 15: Stage finished -26/04/01 05:56:55 INFO DAGScheduler: Job 14 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 41.364141 s -26/04/01 05:56:55 INFO CodeGenerator: Code generated in 2.945959 ms -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 16 (20.5 MiB) non-empty blocks including 16 (20.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:55 INFO CodeGenerator: Code generated in 3.59775 ms -26/04/01 05:56:55 INFO CodeGenerator: Code generated in 3.850333 ms -26/04/01 05:56:55 INFO MemoryStore: Block broadcast_23 stored as values in memory (estimated size 201.9 KiB, free 8.6 GiB) -26/04/01 05:56:55 INFO CodeGenerator: Code generated in 1.692125 ms -26/04/01 05:56:55 INFO MemoryStore: Block broadcast_23_piece0 stored as bytes in memory (estimated size 35.2 KiB, free 8.6 GiB) -26/04/01 05:56:55 INFO BlockManagerInfo: Added broadcast_23_piece0 in memory on 10.0.0.133:58206 (size: 35.2 KiB, free: 8.6 GiB) -26/04/01 05:56:55 INFO SparkContext: Created broadcast 23 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 64 (42.8 MiB) non-empty blocks including 64 (42.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms -26/04/01 05:56:55 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 05:56:55 INFO CodeGenerator: Code generated in 2.804208 ms -26/04/01 05:56:55 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:56:55 INFO DAGScheduler: Got job 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 05:56:55 INFO DAGScheduler: Final stage: ResultStage 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:56:55 INFO DAGScheduler: Parents of final stage: List() -26/04/01 05:56:55 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:55 INFO DAGScheduler: Submitting ResultStage 19 (MapPartitionsRDD[54] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:56:55 INFO CodeGenerator: Code generated in 1.525541 ms -26/04/01 05:56:55 WARN DAGScheduler: Broadcasting large task binary with size 1045.3 KiB -26/04/01 05:56:55 INFO MemoryStore: Block broadcast_24 stored as values in memory (estimated size 1045.3 KiB, free 8.6 GiB) -26/04/01 05:56:55 INFO MemoryStore: Block broadcast_24_piece0 stored as bytes in memory (estimated size 12.7 KiB, free 8.6 GiB) -26/04/01 05:56:55 INFO BlockManagerInfo: Added broadcast_24_piece0 in memory on 10.0.0.133:58206 (size: 12.7 KiB, free: 8.6 GiB) -26/04/01 05:56:55 INFO SparkContext: Created broadcast 24 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:55 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 19 (MapPartitionsRDD[54] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:56:55 INFO TaskSchedulerImpl: Adding task set 19.0 with 1 tasks resource profile 0 -26/04/01 05:56:55 INFO CodeGenerator: Code generated in 1.738541 ms -26/04/01 05:56:55 INFO CodeGenerator: Code generated in 3.485708 ms -26/04/01 05:56:55 INFO Executor: Finished task 206.0 in stage 12.0 (TID 296). 2251 bytes result sent to driver -26/04/01 05:56:55 INFO TaskSetManager: Starting task 2.0 in stage 18.0 (TID 305) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9270 bytes) -26/04/01 05:56:55 INFO TaskSetManager: Finished task 206.0 in stage 12.0 (TID 296) in 476 ms on 10.0.0.133 (executor driver) (207/208) -26/04/01 05:56:55 INFO Executor: Running task 2.0 in stage 18.0 (TID 305) -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 16 (20.5 MiB) non-empty blocks including 16 (20.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:55 INFO Executor: Finished task 207.0 in stage 12.0 (TID 297). 2251 bytes result sent to driver -26/04/01 05:56:55 INFO TaskSetManager: Starting task 3.0 in stage 18.0 (TID 306) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9270 bytes) -26/04/01 05:56:55 INFO TaskSetManager: Finished task 207.0 in stage 12.0 (TID 297) in 463 ms on 10.0.0.133 (executor driver) (208/208) -26/04/01 05:56:55 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool -26/04/01 05:56:55 INFO Executor: Running task 3.0 in stage 18.0 (TID 306) -26/04/01 05:56:55 INFO DAGScheduler: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 42.935 s -26/04/01 05:56:55 INFO DAGScheduler: looking for newly runnable stages -26/04/01 05:56:55 INFO DAGScheduler: running: Set(ResultStage 19, ShuffleMapStage 18) -26/04/01 05:56:55 INFO DAGScheduler: waiting: Set() -26/04/01 05:56:55 INFO DAGScheduler: failed: Set() -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 16 (20.5 MiB) non-empty blocks including 16 (20.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:57 INFO Executor: Finished task 2.0 in stage 18.0 (TID 305). 7309 bytes result sent to driver -26/04/01 05:56:57 INFO TaskSetManager: Starting task 4.0 in stage 18.0 (TID 307) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9270 bytes) -26/04/01 05:56:57 INFO Executor: Running task 4.0 in stage 18.0 (TID 307) -26/04/01 05:56:57 INFO TaskSetManager: Finished task 2.0 in stage 18.0 (TID 305) in 1804 ms on 10.0.0.133 (executor driver) (1/7) -26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Getting 16 (20.5 MiB) non-empty blocks including 16 (20.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:57 INFO Executor: Finished task 3.0 in stage 18.0 (TID 306). 7309 bytes result sent to driver -26/04/01 05:56:57 INFO TaskSetManager: Starting task 5.0 in stage 18.0 (TID 308) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9270 bytes) -26/04/01 05:56:57 INFO TaskSetManager: Finished task 3.0 in stage 18.0 (TID 306) in 1746 ms on 10.0.0.133 (executor driver) (2/7) -26/04/01 05:56:57 INFO Executor: Running task 5.0 in stage 18.0 (TID 308) -26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Getting 64 (42.6 MiB) non-empty blocks including 64 (42.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Getting 16 (20.5 MiB) non-empty blocks including 16 (20.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Getting 64 (42.7 MiB) non-empty blocks including 64 (42.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:57 INFO Executor: Finished task 0.0 in stage 18.0 (TID 303). 7309 bytes result sent to driver -26/04/01 05:56:57 INFO TaskSetManager: Starting task 6.0 in stage 18.0 (TID 309) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9270 bytes) -26/04/01 05:56:57 INFO TaskSetManager: Finished task 0.0 in stage 18.0 (TID 303) in 1862 ms on 10.0.0.133 (executor driver) (3/7) -26/04/01 05:56:57 INFO Executor: Running task 6.0 in stage 18.0 (TID 309) -26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Getting 16 (9.2 MiB) non-empty blocks including 16 (9.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Getting 64 (19.3 MiB) non-empty blocks including 64 (19.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:57 INFO Executor: Finished task 1.0 in stage 18.0 (TID 304). 7266 bytes result sent to driver -26/04/01 05:56:57 INFO TaskSetManager: Starting task 0.0 in stage 19.0 (TID 310) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9608 bytes) -26/04/01 05:56:57 INFO TaskSetManager: Finished task 1.0 in stage 18.0 (TID 304) in 1860 ms on 10.0.0.133 (executor driver) (4/7) -26/04/01 05:56:57 INFO Executor: Running task 0.0 in stage 19.0 (TID 310) -26/04/01 05:56:57 INFO CodeGenerator: Code generated in 3.867541 ms -26/04/01 05:56:57 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/nation/nation.1.parquet, range: 0-2670, partition values: [empty row] -26/04/01 05:56:57 INFO FilterCompat: Filtering using predicate: and(noteq(n_nationkey, null), noteq(n_regionkey, null)) -26/04/01 05:56:57 INFO Executor: Finished task 0.0 in stage 19.0 (TID 310). 1921 bytes result sent to driver -26/04/01 05:56:57 INFO TaskSetManager: Finished task 0.0 in stage 19.0 (TID 310) in 9 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:56:57 INFO TaskSchedulerImpl: Removed TaskSet 19.0, whose tasks have all completed, from pool -26/04/01 05:56:57 INFO DAGScheduler: ResultStage 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 1.844 s -26/04/01 05:56:57 INFO DAGScheduler: Job 16 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:56:57 INFO TaskSchedulerImpl: Killing all running tasks in stage 19: Stage finished -26/04/01 05:56:57 INFO DAGScheduler: Job 16 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 1.844655 s -26/04/01 05:56:57 INFO CodeGenerator: Code generated in 1.84025 ms -26/04/01 05:56:57 INFO MemoryStore: Block broadcast_25 stored as values in memory (estimated size 1024.2 KiB, free 8.5 GiB) -26/04/01 05:56:57 INFO MemoryStore: Block broadcast_25_piece0 stored as bytes in memory (estimated size 326.0 B, free 8.5 GiB) -26/04/01 05:56:57 INFO BlockManagerInfo: Added broadcast_25_piece0 in memory on 10.0.0.133:58206 (size: 326.0 B, free: 8.6 GiB) -26/04/01 05:56:57 INFO SparkContext: Created broadcast 25 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:56:58 INFO BlockManagerInfo: Removed broadcast_24_piece0 on 10.0.0.133:58206 in memory (size: 12.7 KiB, free: 8.6 GiB) -26/04/01 05:56:58 INFO Executor: Finished task 6.0 in stage 18.0 (TID 309). 7266 bytes result sent to driver -26/04/01 05:56:58 INFO TaskSetManager: Finished task 6.0 in stage 18.0 (TID 309) in 687 ms on 10.0.0.133 (executor driver) (5/7) -26/04/01 05:56:59 INFO Executor: Finished task 4.0 in stage 18.0 (TID 307). 7309 bytes result sent to driver -26/04/01 05:56:59 INFO TaskSetManager: Finished task 4.0 in stage 18.0 (TID 307) in 1510 ms on 10.0.0.133 (executor driver) (6/7) -26/04/01 05:56:59 INFO Executor: Finished task 5.0 in stage 18.0 (TID 308). 7266 bytes result sent to driver -26/04/01 05:56:59 INFO TaskSetManager: Finished task 5.0 in stage 18.0 (TID 308) in 1509 ms on 10.0.0.133 (executor driver) (7/7) -26/04/01 05:56:59 INFO TaskSchedulerImpl: Removed TaskSet 18.0, whose tasks have all completed, from pool -26/04/01 05:56:59 INFO DAGScheduler: ShuffleMapStage 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 40.936 s -26/04/01 05:56:59 INFO DAGScheduler: looking for newly runnable stages -26/04/01 05:56:59 INFO DAGScheduler: running: Set() -26/04/01 05:56:59 INFO DAGScheduler: waiting: Set() -26/04/01 05:56:59 INFO DAGScheduler: failed: Set() -26/04/01 05:56:59 INFO ShufflePartitionsUtil: For shuffle(5, 3), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 05:56:59 INFO CodeGenerator: Code generated in 3.7285 ms -26/04/01 05:56:59 INFO CodeGenerator: Code generated in 1.978708 ms -26/04/01 05:56:59 INFO CodeGenerator: Code generated in 2.039917 ms -26/04/01 05:56:59 INFO DAGScheduler: Registering RDD 61 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 6 -26/04/01 05:56:59 INFO DAGScheduler: Got map stage job 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions -26/04/01 05:56:59 INFO DAGScheduler: Final stage: ShuffleMapStage 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:56:59 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 22, ShuffleMapStage 23) -26/04/01 05:56:59 INFO DAGScheduler: Missing parents: List() -26/04/01 05:56:59 INFO DAGScheduler: Submitting ShuffleMapStage 24 (MapPartitionsRDD[61] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:56:59 INFO MemoryStore: Block broadcast_26 stored as values in memory (estimated size 64.1 KiB, free 8.6 GiB) -26/04/01 05:56:59 INFO MemoryStore: Block broadcast_26_piece0 stored as bytes in memory (estimated size 29.5 KiB, free 8.6 GiB) -26/04/01 05:56:59 INFO BlockManagerInfo: Added broadcast_26_piece0 in memory on 10.0.0.133:58206 (size: 29.5 KiB, free: 8.6 GiB) -26/04/01 05:56:59 INFO SparkContext: Created broadcast 26 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:56:59 INFO DAGScheduler: Submitting 200 missing tasks from ShuffleMapStage 24 (MapPartitionsRDD[61] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 05:56:59 INFO TaskSchedulerImpl: Adding task set 24.0 with 200 tasks resource profile 0 -26/04/01 05:56:59 INFO TaskSetManager: Starting task 0.0 in stage 24.0 (TID 311) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9270 bytes) -26/04/01 05:56:59 INFO TaskSetManager: Starting task 1.0 in stage 24.0 (TID 312) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9270 bytes) -26/04/01 05:56:59 INFO TaskSetManager: Starting task 2.0 in stage 24.0 (TID 313) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9270 bytes) -26/04/01 05:56:59 INFO TaskSetManager: Starting task 3.0 in stage 24.0 (TID 314) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9270 bytes) -26/04/01 05:56:59 INFO Executor: Running task 0.0 in stage 24.0 (TID 311) -26/04/01 05:56:59 INFO Executor: Running task 2.0 in stage 24.0 (TID 313) -26/04/01 05:56:59 INFO Executor: Running task 1.0 in stage 24.0 (TID 312) -26/04/01 05:56:59 INFO Executor: Running task 3.0 in stage 24.0 (TID 314) -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:59 INFO CodeGenerator: Code generated in 2.124834 ms -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:59 INFO CodeGenerator: Code generated in 2.212 ms -26/04/01 05:56:59 INFO CodeGenerator: Code generated in 3.180792 ms -26/04/01 05:56:59 INFO CodeGenerator: Code generated in 2.126 ms -26/04/01 05:56:59 INFO Executor: Finished task 2.0 in stage 24.0 (TID 313). 10893 bytes result sent to driver -26/04/01 05:56:59 INFO TaskSetManager: Starting task 4.0 in stage 24.0 (TID 315) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9270 bytes) -26/04/01 05:56:59 INFO Executor: Running task 4.0 in stage 24.0 (TID 315) -26/04/01 05:56:59 INFO TaskSetManager: Finished task 2.0 in stage 24.0 (TID 313) in 569 ms on 10.0.0.133 (executor driver) (1/200) -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:59 INFO Executor: Finished task 3.0 in stage 24.0 (TID 314). 10893 bytes result sent to driver -26/04/01 05:56:59 INFO TaskSetManager: Starting task 5.0 in stage 24.0 (TID 316) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9270 bytes) -26/04/01 05:56:59 INFO TaskSetManager: Finished task 3.0 in stage 24.0 (TID 314) in 583 ms on 10.0.0.133 (executor driver) (2/200) -26/04/01 05:56:59 INFO Executor: Running task 5.0 in stage 24.0 (TID 316) -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:59 INFO Executor: Finished task 1.0 in stage 24.0 (TID 312). 10893 bytes result sent to driver -26/04/01 05:56:59 INFO TaskSetManager: Starting task 6.0 in stage 24.0 (TID 317) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9270 bytes) -26/04/01 05:56:59 INFO TaskSetManager: Finished task 1.0 in stage 24.0 (TID 312) in 599 ms on 10.0.0.133 (executor driver) (3/200) -26/04/01 05:56:59 INFO Executor: Running task 6.0 in stage 24.0 (TID 317) -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:59 INFO Executor: Finished task 0.0 in stage 24.0 (TID 311). 10893 bytes result sent to driver -26/04/01 05:56:59 INFO TaskSetManager: Starting task 7.0 in stage 24.0 (TID 318) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9270 bytes) -26/04/01 05:56:59 INFO TaskSetManager: Finished task 0.0 in stage 24.0 (TID 311) in 604 ms on 10.0.0.133 (executor driver) (4/200) -26/04/01 05:56:59 INFO Executor: Running task 7.0 in stage 24.0 (TID 318) -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:56:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO Executor: Finished task 4.0 in stage 24.0 (TID 315). 10893 bytes result sent to driver -26/04/01 05:57:00 INFO TaskSetManager: Starting task 8.0 in stage 24.0 (TID 319) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:00 INFO Executor: Running task 8.0 in stage 24.0 (TID 319) -26/04/01 05:57:00 INFO TaskSetManager: Finished task 4.0 in stage 24.0 (TID 315) in 522 ms on 10.0.0.133 (executor driver) (5/200) -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO Executor: Finished task 5.0 in stage 24.0 (TID 316). 10850 bytes result sent to driver -26/04/01 05:57:00 INFO TaskSetManager: Starting task 9.0 in stage 24.0 (TID 320) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:00 INFO TaskSetManager: Finished task 5.0 in stage 24.0 (TID 316) in 519 ms on 10.0.0.133 (executor driver) (6/200) -26/04/01 05:57:00 INFO Executor: Running task 9.0 in stage 24.0 (TID 320) -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO Executor: Finished task 6.0 in stage 24.0 (TID 317). 10850 bytes result sent to driver -26/04/01 05:57:00 INFO TaskSetManager: Starting task 10.0 in stage 24.0 (TID 321) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:00 INFO Executor: Running task 10.0 in stage 24.0 (TID 321) -26/04/01 05:57:00 INFO TaskSetManager: Finished task 6.0 in stage 24.0 (TID 317) in 519 ms on 10.0.0.133 (executor driver) (7/200) -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO Executor: Finished task 7.0 in stage 24.0 (TID 318). 10850 bytes result sent to driver -26/04/01 05:57:00 INFO TaskSetManager: Starting task 11.0 in stage 24.0 (TID 322) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:00 INFO Executor: Running task 11.0 in stage 24.0 (TID 322) -26/04/01 05:57:00 INFO TaskSetManager: Finished task 7.0 in stage 24.0 (TID 318) in 517 ms on 10.0.0.133 (executor driver) (8/200) -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO Executor: Finished task 8.0 in stage 24.0 (TID 319). 10850 bytes result sent to driver -26/04/01 05:57:00 INFO TaskSetManager: Starting task 12.0 in stage 24.0 (TID 323) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:00 INFO Executor: Running task 12.0 in stage 24.0 (TID 323) -26/04/01 05:57:00 INFO TaskSetManager: Finished task 8.0 in stage 24.0 (TID 319) in 511 ms on 10.0.0.133 (executor driver) (9/200) -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO Executor: Finished task 9.0 in stage 24.0 (TID 320). 10850 bytes result sent to driver -26/04/01 05:57:00 INFO TaskSetManager: Starting task 13.0 in stage 24.0 (TID 324) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:00 INFO Executor: Running task 13.0 in stage 24.0 (TID 324) -26/04/01 05:57:00 INFO TaskSetManager: Finished task 9.0 in stage 24.0 (TID 320) in 505 ms on 10.0.0.133 (executor driver) (10/200) -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO Executor: Finished task 10.0 in stage 24.0 (TID 321). 10807 bytes result sent to driver -26/04/01 05:57:00 INFO TaskSetManager: Starting task 14.0 in stage 24.0 (TID 325) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:00 INFO Executor: Running task 14.0 in stage 24.0 (TID 325) -26/04/01 05:57:00 INFO TaskSetManager: Finished task 10.0 in stage 24.0 (TID 321) in 501 ms on 10.0.0.133 (executor driver) (11/200) -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO Executor: Finished task 11.0 in stage 24.0 (TID 322). 10807 bytes result sent to driver -26/04/01 05:57:00 INFO TaskSetManager: Starting task 15.0 in stage 24.0 (TID 326) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:00 INFO TaskSetManager: Finished task 11.0 in stage 24.0 (TID 322) in 500 ms on 10.0.0.133 (executor driver) (12/200) -26/04/01 05:57:00 INFO Executor: Running task 15.0 in stage 24.0 (TID 326) -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO Executor: Finished task 12.0 in stage 24.0 (TID 323). 10893 bytes result sent to driver -26/04/01 05:57:01 INFO TaskSetManager: Starting task 16.0 in stage 24.0 (TID 327) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:01 INFO Executor: Running task 16.0 in stage 24.0 (TID 327) -26/04/01 05:57:01 INFO TaskSetManager: Finished task 12.0 in stage 24.0 (TID 323) in 508 ms on 10.0.0.133 (executor driver) (13/200) -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO Executor: Finished task 13.0 in stage 24.0 (TID 324). 10850 bytes result sent to driver -26/04/01 05:57:01 INFO TaskSetManager: Starting task 17.0 in stage 24.0 (TID 328) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:01 INFO TaskSetManager: Finished task 13.0 in stage 24.0 (TID 324) in 512 ms on 10.0.0.133 (executor driver) (14/200) -26/04/01 05:57:01 INFO Executor: Running task 17.0 in stage 24.0 (TID 328) -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO Executor: Finished task 14.0 in stage 24.0 (TID 325). 10893 bytes result sent to driver -26/04/01 05:57:01 INFO TaskSetManager: Starting task 18.0 in stage 24.0 (TID 329) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:01 INFO Executor: Running task 18.0 in stage 24.0 (TID 329) -26/04/01 05:57:01 INFO TaskSetManager: Finished task 14.0 in stage 24.0 (TID 325) in 509 ms on 10.0.0.133 (executor driver) (15/200) -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO Executor: Finished task 15.0 in stage 24.0 (TID 326). 10893 bytes result sent to driver -26/04/01 05:57:01 INFO TaskSetManager: Starting task 19.0 in stage 24.0 (TID 330) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:01 INFO TaskSetManager: Finished task 15.0 in stage 24.0 (TID 326) in 509 ms on 10.0.0.133 (executor driver) (16/200) -26/04/01 05:57:01 INFO Executor: Running task 19.0 in stage 24.0 (TID 330) -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO Executor: Finished task 16.0 in stage 24.0 (TID 327). 10893 bytes result sent to driver -26/04/01 05:57:01 INFO TaskSetManager: Starting task 20.0 in stage 24.0 (TID 331) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:01 INFO Executor: Running task 20.0 in stage 24.0 (TID 331) -26/04/01 05:57:01 INFO TaskSetManager: Finished task 16.0 in stage 24.0 (TID 327) in 498 ms on 10.0.0.133 (executor driver) (17/200) -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO Executor: Finished task 17.0 in stage 24.0 (TID 328). 10850 bytes result sent to driver -26/04/01 05:57:01 INFO TaskSetManager: Starting task 21.0 in stage 24.0 (TID 332) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:01 INFO TaskSetManager: Finished task 17.0 in stage 24.0 (TID 328) in 512 ms on 10.0.0.133 (executor driver) (18/200) -26/04/01 05:57:01 INFO Executor: Running task 21.0 in stage 24.0 (TID 332) -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1229.1 KiB) non-empty blocks including 7 (1229.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO Executor: Finished task 18.0 in stage 24.0 (TID 329). 10893 bytes result sent to driver -26/04/01 05:57:01 INFO TaskSetManager: Starting task 22.0 in stage 24.0 (TID 333) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:01 INFO TaskSetManager: Finished task 18.0 in stage 24.0 (TID 329) in 518 ms on 10.0.0.133 (executor driver) (19/200) -26/04/01 05:57:01 INFO Executor: Running task 22.0 in stage 24.0 (TID 333) -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO Executor: Finished task 19.0 in stage 24.0 (TID 330). 10850 bytes result sent to driver -26/04/01 05:57:01 INFO TaskSetManager: Starting task 23.0 in stage 24.0 (TID 334) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:01 INFO TaskSetManager: Finished task 19.0 in stage 24.0 (TID 330) in 519 ms on 10.0.0.133 (executor driver) (20/200) -26/04/01 05:57:01 INFO Executor: Running task 23.0 in stage 24.0 (TID 334) -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO Executor: Finished task 20.0 in stage 24.0 (TID 331). 10850 bytes result sent to driver -26/04/01 05:57:02 INFO TaskSetManager: Starting task 24.0 in stage 24.0 (TID 335) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:02 INFO Executor: Running task 24.0 in stage 24.0 (TID 335) -26/04/01 05:57:02 INFO TaskSetManager: Finished task 20.0 in stage 24.0 (TID 331) in 505 ms on 10.0.0.133 (executor driver) (21/200) -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO Executor: Finished task 21.0 in stage 24.0 (TID 332). 10850 bytes result sent to driver -26/04/01 05:57:02 INFO TaskSetManager: Starting task 25.0 in stage 24.0 (TID 336) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:02 INFO TaskSetManager: Finished task 21.0 in stage 24.0 (TID 332) in 512 ms on 10.0.0.133 (executor driver) (22/200) -26/04/01 05:57:02 INFO Executor: Running task 25.0 in stage 24.0 (TID 336) -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO Executor: Finished task 22.0 in stage 24.0 (TID 333). 10807 bytes result sent to driver -26/04/01 05:57:02 INFO TaskSetManager: Starting task 26.0 in stage 24.0 (TID 337) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:02 INFO TaskSetManager: Finished task 22.0 in stage 24.0 (TID 333) in 506 ms on 10.0.0.133 (executor driver) (23/200) -26/04/01 05:57:02 INFO Executor: Running task 26.0 in stage 24.0 (TID 337) -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO Executor: Finished task 23.0 in stage 24.0 (TID 334). 10850 bytes result sent to driver -26/04/01 05:57:02 INFO TaskSetManager: Starting task 27.0 in stage 24.0 (TID 338) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:02 INFO Executor: Running task 27.0 in stage 24.0 (TID 338) -26/04/01 05:57:02 INFO TaskSetManager: Finished task 23.0 in stage 24.0 (TID 334) in 506 ms on 10.0.0.133 (executor driver) (24/200) -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO Executor: Finished task 24.0 in stage 24.0 (TID 335). 10893 bytes result sent to driver -26/04/01 05:57:02 INFO TaskSetManager: Starting task 28.0 in stage 24.0 (TID 339) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:02 INFO Executor: Running task 28.0 in stage 24.0 (TID 339) -26/04/01 05:57:02 INFO TaskSetManager: Finished task 24.0 in stage 24.0 (TID 335) in 497 ms on 10.0.0.133 (executor driver) (25/200) -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO Executor: Finished task 25.0 in stage 24.0 (TID 336). 10893 bytes result sent to driver -26/04/01 05:57:02 INFO TaskSetManager: Starting task 29.0 in stage 24.0 (TID 340) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:02 INFO Executor: Running task 29.0 in stage 24.0 (TID 340) -26/04/01 05:57:02 INFO TaskSetManager: Finished task 25.0 in stage 24.0 (TID 336) in 517 ms on 10.0.0.133 (executor driver) (26/200) -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO Executor: Finished task 27.0 in stage 24.0 (TID 338). 10893 bytes result sent to driver -26/04/01 05:57:02 INFO TaskSetManager: Starting task 30.0 in stage 24.0 (TID 341) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:02 INFO Executor: Running task 30.0 in stage 24.0 (TID 341) -26/04/01 05:57:02 INFO TaskSetManager: Finished task 27.0 in stage 24.0 (TID 338) in 513 ms on 10.0.0.133 (executor driver) (27/200) -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO Executor: Finished task 26.0 in stage 24.0 (TID 337). 10850 bytes result sent to driver -26/04/01 05:57:02 INFO TaskSetManager: Starting task 31.0 in stage 24.0 (TID 342) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:02 INFO Executor: Running task 31.0 in stage 24.0 (TID 342) -26/04/01 05:57:02 INFO TaskSetManager: Finished task 26.0 in stage 24.0 (TID 337) in 523 ms on 10.0.0.133 (executor driver) (28/200) -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO Executor: Finished task 28.0 in stage 24.0 (TID 339). 10893 bytes result sent to driver -26/04/01 05:57:03 INFO TaskSetManager: Starting task 32.0 in stage 24.0 (TID 343) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:03 INFO Executor: Running task 32.0 in stage 24.0 (TID 343) -26/04/01 05:57:03 INFO TaskSetManager: Finished task 28.0 in stage 24.0 (TID 339) in 490 ms on 10.0.0.133 (executor driver) (29/200) -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO Executor: Finished task 29.0 in stage 24.0 (TID 340). 10893 bytes result sent to driver -26/04/01 05:57:03 INFO TaskSetManager: Starting task 33.0 in stage 24.0 (TID 344) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:03 INFO TaskSetManager: Finished task 29.0 in stage 24.0 (TID 340) in 507 ms on 10.0.0.133 (executor driver) (30/200) -26/04/01 05:57:03 INFO Executor: Running task 33.0 in stage 24.0 (TID 344) -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO Executor: Finished task 30.0 in stage 24.0 (TID 341). 10850 bytes result sent to driver -26/04/01 05:57:03 INFO TaskSetManager: Starting task 34.0 in stage 24.0 (TID 345) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:03 INFO Executor: Running task 34.0 in stage 24.0 (TID 345) -26/04/01 05:57:03 INFO TaskSetManager: Finished task 30.0 in stage 24.0 (TID 341) in 506 ms on 10.0.0.133 (executor driver) (31/200) -26/04/01 05:57:03 INFO Executor: Finished task 31.0 in stage 24.0 (TID 342). 10850 bytes result sent to driver -26/04/01 05:57:03 INFO TaskSetManager: Starting task 35.0 in stage 24.0 (TID 346) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:03 INFO Executor: Running task 35.0 in stage 24.0 (TID 346) -26/04/01 05:57:03 INFO TaskSetManager: Finished task 31.0 in stage 24.0 (TID 342) in 498 ms on 10.0.0.133 (executor driver) (32/200) -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO Executor: Finished task 32.0 in stage 24.0 (TID 343). 10850 bytes result sent to driver -26/04/01 05:57:03 INFO TaskSetManager: Starting task 36.0 in stage 24.0 (TID 347) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:03 INFO TaskSetManager: Finished task 32.0 in stage 24.0 (TID 343) in 484 ms on 10.0.0.133 (executor driver) (33/200) -26/04/01 05:57:03 INFO Executor: Running task 36.0 in stage 24.0 (TID 347) -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO Executor: Finished task 33.0 in stage 24.0 (TID 344). 10850 bytes result sent to driver -26/04/01 05:57:03 INFO Executor: Finished task 34.0 in stage 24.0 (TID 345). 10850 bytes result sent to driver -26/04/01 05:57:03 INFO TaskSetManager: Starting task 37.0 in stage 24.0 (TID 348) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:03 INFO Executor: Running task 37.0 in stage 24.0 (TID 348) -26/04/01 05:57:03 INFO TaskSetManager: Starting task 38.0 in stage 24.0 (TID 349) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:03 INFO Executor: Running task 38.0 in stage 24.0 (TID 349) -26/04/01 05:57:03 INFO TaskSetManager: Finished task 33.0 in stage 24.0 (TID 344) in 507 ms on 10.0.0.133 (executor driver) (34/200) -26/04/01 05:57:03 INFO TaskSetManager: Finished task 34.0 in stage 24.0 (TID 345) in 503 ms on 10.0.0.133 (executor driver) (35/200) -26/04/01 05:57:03 INFO Executor: Finished task 35.0 in stage 24.0 (TID 346). 10850 bytes result sent to driver -26/04/01 05:57:03 INFO TaskSetManager: Starting task 39.0 in stage 24.0 (TID 350) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:03 INFO TaskSetManager: Finished task 35.0 in stage 24.0 (TID 346) in 503 ms on 10.0.0.133 (executor driver) (36/200) -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO Executor: Running task 39.0 in stage 24.0 (TID 350) -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO Executor: Finished task 36.0 in stage 24.0 (TID 347). 10893 bytes result sent to driver -26/04/01 05:57:04 INFO TaskSetManager: Starting task 40.0 in stage 24.0 (TID 351) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:04 INFO Executor: Running task 40.0 in stage 24.0 (TID 351) -26/04/01 05:57:04 INFO TaskSetManager: Finished task 36.0 in stage 24.0 (TID 347) in 519 ms on 10.0.0.133 (executor driver) (37/200) -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO Executor: Finished task 39.0 in stage 24.0 (TID 350). 10850 bytes result sent to driver -26/04/01 05:57:04 INFO TaskSetManager: Starting task 41.0 in stage 24.0 (TID 352) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:04 INFO Executor: Running task 41.0 in stage 24.0 (TID 352) -26/04/01 05:57:04 INFO TaskSetManager: Finished task 39.0 in stage 24.0 (TID 350) in 509 ms on 10.0.0.133 (executor driver) (38/200) -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO Executor: Finished task 37.0 in stage 24.0 (TID 348). 10893 bytes result sent to driver -26/04/01 05:57:04 INFO TaskSetManager: Starting task 42.0 in stage 24.0 (TID 353) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:04 INFO Executor: Running task 42.0 in stage 24.0 (TID 353) -26/04/01 05:57:04 INFO TaskSetManager: Finished task 37.0 in stage 24.0 (TID 348) in 513 ms on 10.0.0.133 (executor driver) (39/200) -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO Executor: Finished task 38.0 in stage 24.0 (TID 349). 10893 bytes result sent to driver -26/04/01 05:57:04 INFO TaskSetManager: Starting task 43.0 in stage 24.0 (TID 354) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:04 INFO TaskSetManager: Finished task 38.0 in stage 24.0 (TID 349) in 515 ms on 10.0.0.133 (executor driver) (40/200) -26/04/01 05:57:04 INFO Executor: Running task 43.0 in stage 24.0 (TID 354) -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO Executor: Finished task 40.0 in stage 24.0 (TID 351). 10893 bytes result sent to driver -26/04/01 05:57:04 INFO TaskSetManager: Starting task 44.0 in stage 24.0 (TID 355) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:04 INFO TaskSetManager: Finished task 40.0 in stage 24.0 (TID 351) in 482 ms on 10.0.0.133 (executor driver) (41/200) -26/04/01 05:57:04 INFO Executor: Running task 44.0 in stage 24.0 (TID 355) -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO Executor: Finished task 43.0 in stage 24.0 (TID 354). 10850 bytes result sent to driver -26/04/01 05:57:04 INFO TaskSetManager: Starting task 45.0 in stage 24.0 (TID 356) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:04 INFO Executor: Running task 45.0 in stage 24.0 (TID 356) -26/04/01 05:57:04 INFO TaskSetManager: Finished task 43.0 in stage 24.0 (TID 354) in 504 ms on 10.0.0.133 (executor driver) (42/200) -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO Executor: Finished task 41.0 in stage 24.0 (TID 352). 10893 bytes result sent to driver -26/04/01 05:57:04 INFO TaskSetManager: Starting task 46.0 in stage 24.0 (TID 357) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:04 INFO Executor: Running task 46.0 in stage 24.0 (TID 357) -26/04/01 05:57:04 INFO TaskSetManager: Finished task 41.0 in stage 24.0 (TID 352) in 519 ms on 10.0.0.133 (executor driver) (43/200) -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO Executor: Finished task 42.0 in stage 24.0 (TID 353). 10850 bytes result sent to driver -26/04/01 05:57:04 INFO TaskSetManager: Starting task 47.0 in stage 24.0 (TID 358) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:04 INFO TaskSetManager: Finished task 42.0 in stage 24.0 (TID 353) in 518 ms on 10.0.0.133 (executor driver) (44/200) -26/04/01 05:57:04 INFO Executor: Running task 47.0 in stage 24.0 (TID 358) -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO Executor: Finished task 44.0 in stage 24.0 (TID 355). 10850 bytes result sent to driver -26/04/01 05:57:05 INFO TaskSetManager: Starting task 48.0 in stage 24.0 (TID 359) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:05 INFO TaskSetManager: Finished task 44.0 in stage 24.0 (TID 355) in 480 ms on 10.0.0.133 (executor driver) (45/200) -26/04/01 05:57:05 INFO Executor: Running task 48.0 in stage 24.0 (TID 359) -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO Executor: Finished task 45.0 in stage 24.0 (TID 356). 10850 bytes result sent to driver -26/04/01 05:57:05 INFO TaskSetManager: Starting task 49.0 in stage 24.0 (TID 360) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:05 INFO TaskSetManager: Finished task 45.0 in stage 24.0 (TID 356) in 493 ms on 10.0.0.133 (executor driver) (46/200) -26/04/01 05:57:05 INFO Executor: Running task 49.0 in stage 24.0 (TID 360) -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO Executor: Finished task 46.0 in stage 24.0 (TID 357). 10807 bytes result sent to driver -26/04/01 05:57:05 INFO TaskSetManager: Starting task 50.0 in stage 24.0 (TID 361) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:05 INFO Executor: Running task 50.0 in stage 24.0 (TID 361) -26/04/01 05:57:05 INFO TaskSetManager: Finished task 46.0 in stage 24.0 (TID 357) in 497 ms on 10.0.0.133 (executor driver) (47/200) -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO Executor: Finished task 47.0 in stage 24.0 (TID 358). 10807 bytes result sent to driver -26/04/01 05:57:05 INFO TaskSetManager: Starting task 51.0 in stage 24.0 (TID 362) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:05 INFO Executor: Running task 51.0 in stage 24.0 (TID 362) -26/04/01 05:57:05 INFO TaskSetManager: Finished task 47.0 in stage 24.0 (TID 358) in 507 ms on 10.0.0.133 (executor driver) (48/200) -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO Executor: Finished task 48.0 in stage 24.0 (TID 359). 10893 bytes result sent to driver -26/04/01 05:57:05 INFO TaskSetManager: Starting task 52.0 in stage 24.0 (TID 363) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:05 INFO TaskSetManager: Finished task 48.0 in stage 24.0 (TID 359) in 482 ms on 10.0.0.133 (executor driver) (49/200) -26/04/01 05:57:05 INFO Executor: Running task 52.0 in stage 24.0 (TID 363) -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO Executor: Finished task 49.0 in stage 24.0 (TID 360). 10893 bytes result sent to driver -26/04/01 05:57:05 INFO TaskSetManager: Starting task 53.0 in stage 24.0 (TID 364) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:05 INFO Executor: Running task 53.0 in stage 24.0 (TID 364) -26/04/01 05:57:05 INFO TaskSetManager: Finished task 49.0 in stage 24.0 (TID 360) in 493 ms on 10.0.0.133 (executor driver) (50/200) -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO Executor: Finished task 50.0 in stage 24.0 (TID 361). 10850 bytes result sent to driver -26/04/01 05:57:05 INFO TaskSetManager: Starting task 54.0 in stage 24.0 (TID 365) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:05 INFO Executor: Running task 54.0 in stage 24.0 (TID 365) -26/04/01 05:57:05 INFO TaskSetManager: Finished task 50.0 in stage 24.0 (TID 361) in 496 ms on 10.0.0.133 (executor driver) (51/200) -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO Executor: Finished task 51.0 in stage 24.0 (TID 362). 10850 bytes result sent to driver -26/04/01 05:57:05 INFO TaskSetManager: Starting task 55.0 in stage 24.0 (TID 366) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:05 INFO TaskSetManager: Finished task 51.0 in stage 24.0 (TID 362) in 494 ms on 10.0.0.133 (executor driver) (52/200) -26/04/01 05:57:05 INFO Executor: Running task 55.0 in stage 24.0 (TID 366) -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO Executor: Finished task 52.0 in stage 24.0 (TID 363). 10850 bytes result sent to driver -26/04/01 05:57:06 INFO TaskSetManager: Starting task 56.0 in stage 24.0 (TID 367) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:06 INFO TaskSetManager: Finished task 52.0 in stage 24.0 (TID 363) in 478 ms on 10.0.0.133 (executor driver) (53/200) -26/04/01 05:57:06 INFO Executor: Running task 56.0 in stage 24.0 (TID 367) -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO Executor: Finished task 53.0 in stage 24.0 (TID 364). 10893 bytes result sent to driver -26/04/01 05:57:06 INFO TaskSetManager: Starting task 57.0 in stage 24.0 (TID 368) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:06 INFO Executor: Running task 57.0 in stage 24.0 (TID 368) -26/04/01 05:57:06 INFO TaskSetManager: Finished task 53.0 in stage 24.0 (TID 364) in 497 ms on 10.0.0.133 (executor driver) (54/200) -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO Executor: Finished task 54.0 in stage 24.0 (TID 365). 10850 bytes result sent to driver -26/04/01 05:57:06 INFO TaskSetManager: Starting task 58.0 in stage 24.0 (TID 369) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:06 INFO Executor: Running task 58.0 in stage 24.0 (TID 369) -26/04/01 05:57:06 INFO TaskSetManager: Finished task 54.0 in stage 24.0 (TID 365) in 496 ms on 10.0.0.133 (executor driver) (55/200) -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO Executor: Finished task 55.0 in stage 24.0 (TID 366). 10850 bytes result sent to driver -26/04/01 05:57:06 INFO TaskSetManager: Starting task 59.0 in stage 24.0 (TID 370) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:06 INFO TaskSetManager: Finished task 55.0 in stage 24.0 (TID 366) in 496 ms on 10.0.0.133 (executor driver) (56/200) -26/04/01 05:57:06 INFO Executor: Running task 59.0 in stage 24.0 (TID 370) -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO Executor: Finished task 56.0 in stage 24.0 (TID 367). 10850 bytes result sent to driver -26/04/01 05:57:06 INFO TaskSetManager: Starting task 60.0 in stage 24.0 (TID 371) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:06 INFO TaskSetManager: Finished task 56.0 in stage 24.0 (TID 367) in 486 ms on 10.0.0.133 (executor driver) (57/200) -26/04/01 05:57:06 INFO Executor: Running task 60.0 in stage 24.0 (TID 371) -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO Executor: Finished task 57.0 in stage 24.0 (TID 368). 10850 bytes result sent to driver -26/04/01 05:57:06 INFO TaskSetManager: Starting task 61.0 in stage 24.0 (TID 372) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:06 INFO TaskSetManager: Finished task 57.0 in stage 24.0 (TID 368) in 496 ms on 10.0.0.133 (executor driver) (58/200) -26/04/01 05:57:06 INFO Executor: Running task 61.0 in stage 24.0 (TID 372) -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO Executor: Finished task 58.0 in stage 24.0 (TID 369). 10807 bytes result sent to driver -26/04/01 05:57:06 INFO TaskSetManager: Starting task 62.0 in stage 24.0 (TID 373) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:06 INFO TaskSetManager: Finished task 58.0 in stage 24.0 (TID 369) in 496 ms on 10.0.0.133 (executor driver) (59/200) -26/04/01 05:57:06 INFO Executor: Running task 62.0 in stage 24.0 (TID 373) -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO Executor: Finished task 59.0 in stage 24.0 (TID 370). 10807 bytes result sent to driver -26/04/01 05:57:06 INFO TaskSetManager: Starting task 63.0 in stage 24.0 (TID 374) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:06 INFO Executor: Running task 63.0 in stage 24.0 (TID 374) -26/04/01 05:57:06 INFO TaskSetManager: Finished task 59.0 in stage 24.0 (TID 370) in 495 ms on 10.0.0.133 (executor driver) (60/200) -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO Executor: Finished task 60.0 in stage 24.0 (TID 371). 10893 bytes result sent to driver -26/04/01 05:57:07 INFO TaskSetManager: Starting task 64.0 in stage 24.0 (TID 375) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:07 INFO Executor: Running task 64.0 in stage 24.0 (TID 375) -26/04/01 05:57:07 INFO TaskSetManager: Finished task 60.0 in stage 24.0 (TID 371) in 497 ms on 10.0.0.133 (executor driver) (61/200) -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO Executor: Finished task 61.0 in stage 24.0 (TID 372). 10893 bytes result sent to driver -26/04/01 05:57:07 INFO TaskSetManager: Starting task 65.0 in stage 24.0 (TID 376) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:07 INFO Executor: Running task 65.0 in stage 24.0 (TID 376) -26/04/01 05:57:07 INFO TaskSetManager: Finished task 61.0 in stage 24.0 (TID 372) in 504 ms on 10.0.0.133 (executor driver) (62/200) -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO Executor: Finished task 63.0 in stage 24.0 (TID 374). 10850 bytes result sent to driver -26/04/01 05:57:07 INFO TaskSetManager: Starting task 66.0 in stage 24.0 (TID 377) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:07 INFO Executor: Running task 66.0 in stage 24.0 (TID 377) -26/04/01 05:57:07 INFO Executor: Finished task 62.0 in stage 24.0 (TID 373). 10893 bytes result sent to driver -26/04/01 05:57:07 INFO TaskSetManager: Finished task 63.0 in stage 24.0 (TID 374) in 498 ms on 10.0.0.133 (executor driver) (63/200) -26/04/01 05:57:07 INFO TaskSetManager: Starting task 67.0 in stage 24.0 (TID 378) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:07 INFO Executor: Running task 67.0 in stage 24.0 (TID 378) -26/04/01 05:57:07 INFO TaskSetManager: Finished task 62.0 in stage 24.0 (TID 373) in 506 ms on 10.0.0.133 (executor driver) (64/200) -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO Executor: Finished task 64.0 in stage 24.0 (TID 375). 10850 bytes result sent to driver -26/04/01 05:57:07 INFO TaskSetManager: Starting task 68.0 in stage 24.0 (TID 379) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:07 INFO TaskSetManager: Finished task 64.0 in stage 24.0 (TID 375) in 488 ms on 10.0.0.133 (executor driver) (65/200) -26/04/01 05:57:07 INFO Executor: Running task 68.0 in stage 24.0 (TID 379) -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO Executor: Finished task 65.0 in stage 24.0 (TID 376). 10893 bytes result sent to driver -26/04/01 05:57:07 INFO TaskSetManager: Starting task 69.0 in stage 24.0 (TID 380) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:07 INFO TaskSetManager: Finished task 65.0 in stage 24.0 (TID 376) in 495 ms on 10.0.0.133 (executor driver) (66/200) -26/04/01 05:57:07 INFO Executor: Running task 69.0 in stage 24.0 (TID 380) -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO Executor: Finished task 67.0 in stage 24.0 (TID 378). 10850 bytes result sent to driver -26/04/01 05:57:07 INFO Executor: Finished task 66.0 in stage 24.0 (TID 377). 10850 bytes result sent to driver -26/04/01 05:57:07 INFO TaskSetManager: Starting task 70.0 in stage 24.0 (TID 381) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:07 INFO Executor: Running task 70.0 in stage 24.0 (TID 381) -26/04/01 05:57:07 INFO TaskSetManager: Starting task 71.0 in stage 24.0 (TID 382) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:07 INFO Executor: Running task 71.0 in stage 24.0 (TID 382) -26/04/01 05:57:07 INFO TaskSetManager: Finished task 67.0 in stage 24.0 (TID 378) in 505 ms on 10.0.0.133 (executor driver) (67/200) -26/04/01 05:57:07 INFO TaskSetManager: Finished task 66.0 in stage 24.0 (TID 377) in 505 ms on 10.0.0.133 (executor driver) (68/200) -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 7 (1229.1 KiB) non-empty blocks including 7 (1229.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO Executor: Finished task 68.0 in stage 24.0 (TID 379). 10893 bytes result sent to driver -26/04/01 05:57:08 INFO TaskSetManager: Starting task 72.0 in stage 24.0 (TID 383) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:08 INFO TaskSetManager: Finished task 68.0 in stage 24.0 (TID 379) in 488 ms on 10.0.0.133 (executor driver) (69/200) -26/04/01 05:57:08 INFO Executor: Running task 72.0 in stage 24.0 (TID 383) -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO Executor: Finished task 69.0 in stage 24.0 (TID 380). 10850 bytes result sent to driver -26/04/01 05:57:08 INFO TaskSetManager: Starting task 73.0 in stage 24.0 (TID 384) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:08 INFO TaskSetManager: Finished task 69.0 in stage 24.0 (TID 380) in 480 ms on 10.0.0.133 (executor driver) (70/200) -26/04/01 05:57:08 INFO Executor: Running task 73.0 in stage 24.0 (TID 384) -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO Executor: Finished task 70.0 in stage 24.0 (TID 381). 10850 bytes result sent to driver -26/04/01 05:57:08 INFO TaskSetManager: Starting task 74.0 in stage 24.0 (TID 385) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:08 INFO TaskSetManager: Finished task 70.0 in stage 24.0 (TID 381) in 492 ms on 10.0.0.133 (executor driver) (71/200) -26/04/01 05:57:08 INFO Executor: Running task 74.0 in stage 24.0 (TID 385) -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO Executor: Finished task 71.0 in stage 24.0 (TID 382). 10807 bytes result sent to driver -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO TaskSetManager: Starting task 75.0 in stage 24.0 (TID 386) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:08 INFO Executor: Running task 75.0 in stage 24.0 (TID 386) -26/04/01 05:57:08 INFO TaskSetManager: Finished task 71.0 in stage 24.0 (TID 382) in 495 ms on 10.0.0.133 (executor driver) (72/200) -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO Executor: Finished task 72.0 in stage 24.0 (TID 383). 10850 bytes result sent to driver -26/04/01 05:57:08 INFO TaskSetManager: Starting task 76.0 in stage 24.0 (TID 387) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:08 INFO Executor: Running task 76.0 in stage 24.0 (TID 387) -26/04/01 05:57:08 INFO TaskSetManager: Finished task 72.0 in stage 24.0 (TID 383) in 492 ms on 10.0.0.133 (executor driver) (73/200) -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO Executor: Finished task 73.0 in stage 24.0 (TID 384). 10893 bytes result sent to driver -26/04/01 05:57:08 INFO TaskSetManager: Starting task 77.0 in stage 24.0 (TID 388) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:08 INFO Executor: Running task 77.0 in stage 24.0 (TID 388) -26/04/01 05:57:08 INFO TaskSetManager: Finished task 73.0 in stage 24.0 (TID 384) in 492 ms on 10.0.0.133 (executor driver) (74/200) -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO Executor: Finished task 74.0 in stage 24.0 (TID 385). 10893 bytes result sent to driver -26/04/01 05:57:08 INFO TaskSetManager: Starting task 78.0 in stage 24.0 (TID 389) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:08 INFO TaskSetManager: Finished task 74.0 in stage 24.0 (TID 385) in 501 ms on 10.0.0.133 (executor driver) (75/200) -26/04/01 05:57:08 INFO Executor: Running task 78.0 in stage 24.0 (TID 389) -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO Executor: Finished task 75.0 in stage 24.0 (TID 386). 10850 bytes result sent to driver -26/04/01 05:57:08 INFO TaskSetManager: Starting task 79.0 in stage 24.0 (TID 390) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:08 INFO Executor: Running task 79.0 in stage 24.0 (TID 390) -26/04/01 05:57:08 INFO TaskSetManager: Finished task 75.0 in stage 24.0 (TID 386) in 505 ms on 10.0.0.133 (executor driver) (76/200) -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 7 (1229.1 KiB) non-empty blocks including 7 (1229.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO Executor: Finished task 76.0 in stage 24.0 (TID 387). 10893 bytes result sent to driver -26/04/01 05:57:09 INFO TaskSetManager: Starting task 80.0 in stage 24.0 (TID 391) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:09 INFO Executor: Running task 80.0 in stage 24.0 (TID 391) -26/04/01 05:57:09 INFO TaskSetManager: Finished task 76.0 in stage 24.0 (TID 387) in 491 ms on 10.0.0.133 (executor driver) (77/200) -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO Executor: Finished task 77.0 in stage 24.0 (TID 388). 10893 bytes result sent to driver -26/04/01 05:57:09 INFO TaskSetManager: Starting task 81.0 in stage 24.0 (TID 392) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:09 INFO Executor: Running task 81.0 in stage 24.0 (TID 392) -26/04/01 05:57:09 INFO TaskSetManager: Finished task 77.0 in stage 24.0 (TID 388) in 501 ms on 10.0.0.133 (executor driver) (78/200) -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO Executor: Finished task 78.0 in stage 24.0 (TID 389). 10893 bytes result sent to driver -26/04/01 05:57:09 INFO TaskSetManager: Starting task 82.0 in stage 24.0 (TID 393) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:09 INFO Executor: Running task 82.0 in stage 24.0 (TID 393) -26/04/01 05:57:09 INFO TaskSetManager: Finished task 78.0 in stage 24.0 (TID 389) in 504 ms on 10.0.0.133 (executor driver) (79/200) -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO Executor: Finished task 79.0 in stage 24.0 (TID 390). 10850 bytes result sent to driver -26/04/01 05:57:09 INFO TaskSetManager: Starting task 83.0 in stage 24.0 (TID 394) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:09 INFO Executor: Running task 83.0 in stage 24.0 (TID 394) -26/04/01 05:57:09 INFO TaskSetManager: Finished task 79.0 in stage 24.0 (TID 390) in 502 ms on 10.0.0.133 (executor driver) (80/200) -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO Executor: Finished task 80.0 in stage 24.0 (TID 391). 10893 bytes result sent to driver -26/04/01 05:57:09 INFO TaskSetManager: Starting task 84.0 in stage 24.0 (TID 395) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:09 INFO Executor: Running task 84.0 in stage 24.0 (TID 395) -26/04/01 05:57:09 INFO TaskSetManager: Finished task 80.0 in stage 24.0 (TID 391) in 484 ms on 10.0.0.133 (executor driver) (81/200) -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO Executor: Finished task 81.0 in stage 24.0 (TID 392). 10850 bytes result sent to driver -26/04/01 05:57:09 INFO TaskSetManager: Starting task 85.0 in stage 24.0 (TID 396) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:09 INFO Executor: Running task 85.0 in stage 24.0 (TID 396) -26/04/01 05:57:09 INFO TaskSetManager: Finished task 81.0 in stage 24.0 (TID 392) in 492 ms on 10.0.0.133 (executor driver) (82/200) -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO Executor: Finished task 82.0 in stage 24.0 (TID 393). 10850 bytes result sent to driver -26/04/01 05:57:09 INFO TaskSetManager: Starting task 86.0 in stage 24.0 (TID 397) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:09 INFO TaskSetManager: Finished task 82.0 in stage 24.0 (TID 393) in 491 ms on 10.0.0.133 (executor driver) (83/200) -26/04/01 05:57:09 INFO Executor: Running task 86.0 in stage 24.0 (TID 397) -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO Executor: Finished task 83.0 in stage 24.0 (TID 394). 10807 bytes result sent to driver -26/04/01 05:57:09 INFO TaskSetManager: Starting task 87.0 in stage 24.0 (TID 398) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:09 INFO Executor: Running task 87.0 in stage 24.0 (TID 398) -26/04/01 05:57:09 INFO TaskSetManager: Finished task 83.0 in stage 24.0 (TID 394) in 492 ms on 10.0.0.133 (executor driver) (84/200) -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO Executor: Finished task 84.0 in stage 24.0 (TID 395). 10850 bytes result sent to driver -26/04/01 05:57:10 INFO TaskSetManager: Starting task 88.0 in stage 24.0 (TID 399) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:10 INFO Executor: Running task 88.0 in stage 24.0 (TID 399) -26/04/01 05:57:10 INFO TaskSetManager: Finished task 84.0 in stage 24.0 (TID 395) in 479 ms on 10.0.0.133 (executor driver) (85/200) -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO Executor: Finished task 85.0 in stage 24.0 (TID 396). 10893 bytes result sent to driver -26/04/01 05:57:10 INFO TaskSetManager: Starting task 89.0 in stage 24.0 (TID 400) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:10 INFO Executor: Running task 89.0 in stage 24.0 (TID 400) -26/04/01 05:57:10 INFO TaskSetManager: Finished task 85.0 in stage 24.0 (TID 396) in 490 ms on 10.0.0.133 (executor driver) (86/200) -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO Executor: Finished task 86.0 in stage 24.0 (TID 397). 10893 bytes result sent to driver -26/04/01 05:57:10 INFO TaskSetManager: Starting task 90.0 in stage 24.0 (TID 401) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:10 INFO TaskSetManager: Finished task 86.0 in stage 24.0 (TID 397) in 498 ms on 10.0.0.133 (executor driver) (87/200) -26/04/01 05:57:10 INFO Executor: Running task 90.0 in stage 24.0 (TID 401) -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO Executor: Finished task 87.0 in stage 24.0 (TID 398). 10850 bytes result sent to driver -26/04/01 05:57:10 INFO TaskSetManager: Starting task 91.0 in stage 24.0 (TID 402) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:10 INFO Executor: Running task 91.0 in stage 24.0 (TID 402) -26/04/01 05:57:10 INFO TaskSetManager: Finished task 87.0 in stage 24.0 (TID 398) in 494 ms on 10.0.0.133 (executor driver) (88/200) -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO Executor: Finished task 88.0 in stage 24.0 (TID 399). 10850 bytes result sent to driver -26/04/01 05:57:10 INFO TaskSetManager: Starting task 92.0 in stage 24.0 (TID 403) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:10 INFO Executor: Running task 92.0 in stage 24.0 (TID 403) -26/04/01 05:57:10 INFO TaskSetManager: Finished task 88.0 in stage 24.0 (TID 399) in 486 ms on 10.0.0.133 (executor driver) (89/200) -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO Executor: Finished task 89.0 in stage 24.0 (TID 400). 10850 bytes result sent to driver -26/04/01 05:57:10 INFO TaskSetManager: Starting task 93.0 in stage 24.0 (TID 404) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:10 INFO Executor: Running task 93.0 in stage 24.0 (TID 404) -26/04/01 05:57:10 INFO TaskSetManager: Finished task 89.0 in stage 24.0 (TID 400) in 486 ms on 10.0.0.133 (executor driver) (90/200) -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO Executor: Finished task 90.0 in stage 24.0 (TID 401). 10850 bytes result sent to driver -26/04/01 05:57:10 INFO TaskSetManager: Starting task 94.0 in stage 24.0 (TID 405) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:10 INFO Executor: Running task 94.0 in stage 24.0 (TID 405) -26/04/01 05:57:10 INFO TaskSetManager: Finished task 90.0 in stage 24.0 (TID 401) in 494 ms on 10.0.0.133 (executor driver) (91/200) -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO Executor: Finished task 91.0 in stage 24.0 (TID 402). 10807 bytes result sent to driver -26/04/01 05:57:10 INFO TaskSetManager: Starting task 95.0 in stage 24.0 (TID 406) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:10 INFO TaskSetManager: Finished task 91.0 in stage 24.0 (TID 402) in 497 ms on 10.0.0.133 (executor driver) (92/200) -26/04/01 05:57:10 INFO Executor: Running task 95.0 in stage 24.0 (TID 406) -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:10 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO Executor: Finished task 92.0 in stage 24.0 (TID 403). 10893 bytes result sent to driver -26/04/01 05:57:11 INFO TaskSetManager: Starting task 96.0 in stage 24.0 (TID 407) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:11 INFO Executor: Running task 96.0 in stage 24.0 (TID 407) -26/04/01 05:57:11 INFO TaskSetManager: Finished task 92.0 in stage 24.0 (TID 403) in 486 ms on 10.0.0.133 (executor driver) (93/200) -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO Executor: Finished task 93.0 in stage 24.0 (TID 404). 10893 bytes result sent to driver -26/04/01 05:57:11 INFO TaskSetManager: Starting task 97.0 in stage 24.0 (TID 408) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:11 INFO Executor: Running task 97.0 in stage 24.0 (TID 408) -26/04/01 05:57:11 INFO TaskSetManager: Finished task 93.0 in stage 24.0 (TID 404) in 520 ms on 10.0.0.133 (executor driver) (94/200) -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO Executor: Finished task 94.0 in stage 24.0 (TID 405). 10893 bytes result sent to driver -26/04/01 05:57:11 INFO TaskSetManager: Starting task 98.0 in stage 24.0 (TID 409) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:11 INFO Executor: Running task 98.0 in stage 24.0 (TID 409) -26/04/01 05:57:11 INFO TaskSetManager: Finished task 94.0 in stage 24.0 (TID 405) in 500 ms on 10.0.0.133 (executor driver) (95/200) -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO Executor: Finished task 95.0 in stage 24.0 (TID 406). 10893 bytes result sent to driver -26/04/01 05:57:11 INFO TaskSetManager: Starting task 99.0 in stage 24.0 (TID 410) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:11 INFO Executor: Running task 99.0 in stage 24.0 (TID 410) -26/04/01 05:57:11 INFO TaskSetManager: Finished task 95.0 in stage 24.0 (TID 406) in 494 ms on 10.0.0.133 (executor driver) (96/200) -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO Executor: Finished task 96.0 in stage 24.0 (TID 407). 10850 bytes result sent to driver -26/04/01 05:57:11 INFO TaskSetManager: Starting task 100.0 in stage 24.0 (TID 411) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:11 INFO TaskSetManager: Finished task 96.0 in stage 24.0 (TID 407) in 484 ms on 10.0.0.133 (executor driver) (97/200) -26/04/01 05:57:11 INFO Executor: Running task 100.0 in stage 24.0 (TID 411) -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO Executor: Finished task 97.0 in stage 24.0 (TID 408). 10893 bytes result sent to driver -26/04/01 05:57:11 INFO TaskSetManager: Starting task 101.0 in stage 24.0 (TID 412) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:11 INFO TaskSetManager: Finished task 97.0 in stage 24.0 (TID 408) in 490 ms on 10.0.0.133 (executor driver) (98/200) -26/04/01 05:57:11 INFO Executor: Running task 101.0 in stage 24.0 (TID 412) -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO Executor: Finished task 98.0 in stage 24.0 (TID 409). 10893 bytes result sent to driver -26/04/01 05:57:11 INFO TaskSetManager: Starting task 102.0 in stage 24.0 (TID 413) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:11 INFO Executor: Running task 102.0 in stage 24.0 (TID 413) -26/04/01 05:57:11 INFO TaskSetManager: Finished task 98.0 in stage 24.0 (TID 409) in 500 ms on 10.0.0.133 (executor driver) (99/200) -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO Executor: Finished task 99.0 in stage 24.0 (TID 410). 10850 bytes result sent to driver -26/04/01 05:57:11 INFO TaskSetManager: Starting task 103.0 in stage 24.0 (TID 414) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:11 INFO Executor: Running task 103.0 in stage 24.0 (TID 414) -26/04/01 05:57:11 INFO TaskSetManager: Finished task 99.0 in stage 24.0 (TID 410) in 502 ms on 10.0.0.133 (executor driver) (100/200) -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:11 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO Executor: Finished task 100.0 in stage 24.0 (TID 411). 10893 bytes result sent to driver -26/04/01 05:57:12 INFO TaskSetManager: Starting task 104.0 in stage 24.0 (TID 415) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:12 INFO Executor: Running task 104.0 in stage 24.0 (TID 415) -26/04/01 05:57:12 INFO TaskSetManager: Finished task 100.0 in stage 24.0 (TID 411) in 488 ms on 10.0.0.133 (executor driver) (101/200) -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO Executor: Finished task 101.0 in stage 24.0 (TID 412). 10850 bytes result sent to driver -26/04/01 05:57:12 INFO TaskSetManager: Starting task 105.0 in stage 24.0 (TID 416) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:12 INFO TaskSetManager: Finished task 101.0 in stage 24.0 (TID 412) in 484 ms on 10.0.0.133 (executor driver) (102/200) -26/04/01 05:57:12 INFO Executor: Running task 105.0 in stage 24.0 (TID 416) -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO Executor: Finished task 103.0 in stage 24.0 (TID 414). 10807 bytes result sent to driver -26/04/01 05:57:12 INFO TaskSetManager: Starting task 106.0 in stage 24.0 (TID 417) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:12 INFO Executor: Running task 106.0 in stage 24.0 (TID 417) -26/04/01 05:57:12 INFO TaskSetManager: Finished task 103.0 in stage 24.0 (TID 414) in 494 ms on 10.0.0.133 (executor driver) (103/200) -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO Executor: Finished task 102.0 in stage 24.0 (TID 413). 10850 bytes result sent to driver -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO TaskSetManager: Starting task 107.0 in stage 24.0 (TID 418) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:12 INFO TaskSetManager: Finished task 102.0 in stage 24.0 (TID 413) in 499 ms on 10.0.0.133 (executor driver) (104/200) -26/04/01 05:57:12 INFO Executor: Running task 107.0 in stage 24.0 (TID 418) -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO Executor: Finished task 104.0 in stage 24.0 (TID 415). 10893 bytes result sent to driver -26/04/01 05:57:12 INFO TaskSetManager: Starting task 108.0 in stage 24.0 (TID 419) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:12 INFO Executor: Running task 108.0 in stage 24.0 (TID 419) -26/04/01 05:57:12 INFO TaskSetManager: Finished task 104.0 in stage 24.0 (TID 415) in 486 ms on 10.0.0.133 (executor driver) (105/200) -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO Executor: Finished task 105.0 in stage 24.0 (TID 416). 10893 bytes result sent to driver -26/04/01 05:57:12 INFO TaskSetManager: Starting task 109.0 in stage 24.0 (TID 420) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:12 INFO TaskSetManager: Finished task 105.0 in stage 24.0 (TID 416) in 486 ms on 10.0.0.133 (executor driver) (106/200) -26/04/01 05:57:12 INFO Executor: Running task 109.0 in stage 24.0 (TID 420) -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO Executor: Finished task 107.0 in stage 24.0 (TID 418). 10850 bytes result sent to driver -26/04/01 05:57:12 INFO TaskSetManager: Starting task 110.0 in stage 24.0 (TID 421) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:12 INFO Executor: Running task 110.0 in stage 24.0 (TID 421) -26/04/01 05:57:12 INFO TaskSetManager: Finished task 107.0 in stage 24.0 (TID 418) in 497 ms on 10.0.0.133 (executor driver) (107/200) -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO Executor: Finished task 106.0 in stage 24.0 (TID 417). 10850 bytes result sent to driver -26/04/01 05:57:12 INFO TaskSetManager: Starting task 111.0 in stage 24.0 (TID 422) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:12 INFO TaskSetManager: Finished task 106.0 in stage 24.0 (TID 417) in 506 ms on 10.0.0.133 (executor driver) (108/200) -26/04/01 05:57:12 INFO Executor: Running task 111.0 in stage 24.0 (TID 422) -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO Executor: Finished task 108.0 in stage 24.0 (TID 419). 10850 bytes result sent to driver -26/04/01 05:57:13 INFO TaskSetManager: Starting task 112.0 in stage 24.0 (TID 423) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:13 INFO TaskSetManager: Finished task 108.0 in stage 24.0 (TID 419) in 484 ms on 10.0.0.133 (executor driver) (109/200) -26/04/01 05:57:13 INFO Executor: Running task 112.0 in stage 24.0 (TID 423) -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO Executor: Finished task 109.0 in stage 24.0 (TID 420). 10893 bytes result sent to driver -26/04/01 05:57:13 INFO TaskSetManager: Starting task 113.0 in stage 24.0 (TID 424) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:13 INFO TaskSetManager: Finished task 109.0 in stage 24.0 (TID 420) in 488 ms on 10.0.0.133 (executor driver) (110/200) -26/04/01 05:57:13 INFO Executor: Running task 113.0 in stage 24.0 (TID 424) -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO Executor: Finished task 110.0 in stage 24.0 (TID 421). 10893 bytes result sent to driver -26/04/01 05:57:13 INFO TaskSetManager: Starting task 114.0 in stage 24.0 (TID 425) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:13 INFO TaskSetManager: Finished task 110.0 in stage 24.0 (TID 421) in 496 ms on 10.0.0.133 (executor driver) (111/200) -26/04/01 05:57:13 INFO Executor: Running task 114.0 in stage 24.0 (TID 425) -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO Executor: Finished task 111.0 in stage 24.0 (TID 422). 10850 bytes result sent to driver -26/04/01 05:57:13 INFO TaskSetManager: Starting task 115.0 in stage 24.0 (TID 426) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:13 INFO TaskSetManager: Finished task 111.0 in stage 24.0 (TID 422) in 494 ms on 10.0.0.133 (executor driver) (112/200) -26/04/01 05:57:13 INFO Executor: Running task 115.0 in stage 24.0 (TID 426) -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO Executor: Finished task 112.0 in stage 24.0 (TID 423). 10893 bytes result sent to driver -26/04/01 05:57:13 INFO TaskSetManager: Starting task 116.0 in stage 24.0 (TID 427) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:13 INFO Executor: Running task 116.0 in stage 24.0 (TID 427) -26/04/01 05:57:13 INFO TaskSetManager: Finished task 112.0 in stage 24.0 (TID 423) in 484 ms on 10.0.0.133 (executor driver) (113/200) -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO Executor: Finished task 113.0 in stage 24.0 (TID 424). 10850 bytes result sent to driver -26/04/01 05:57:13 INFO TaskSetManager: Starting task 117.0 in stage 24.0 (TID 428) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:13 INFO Executor: Running task 117.0 in stage 24.0 (TID 428) -26/04/01 05:57:13 INFO TaskSetManager: Finished task 113.0 in stage 24.0 (TID 424) in 483 ms on 10.0.0.133 (executor driver) (114/200) -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1229.1 KiB) non-empty blocks including 7 (1229.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO Executor: Finished task 114.0 in stage 24.0 (TID 425). 10850 bytes result sent to driver -26/04/01 05:57:13 INFO TaskSetManager: Starting task 118.0 in stage 24.0 (TID 429) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:13 INFO TaskSetManager: Finished task 114.0 in stage 24.0 (TID 425) in 490 ms on 10.0.0.133 (executor driver) (115/200) -26/04/01 05:57:13 INFO Executor: Running task 118.0 in stage 24.0 (TID 429) -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO Executor: Finished task 115.0 in stage 24.0 (TID 426). 10807 bytes result sent to driver -26/04/01 05:57:13 INFO TaskSetManager: Starting task 119.0 in stage 24.0 (TID 430) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:13 INFO TaskSetManager: Finished task 115.0 in stage 24.0 (TID 426) in 492 ms on 10.0.0.133 (executor driver) (116/200) -26/04/01 05:57:13 INFO Executor: Running task 119.0 in stage 24.0 (TID 430) -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:13 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:13 INFO Executor: Finished task 116.0 in stage 24.0 (TID 427). 10807 bytes result sent to driver -26/04/01 05:57:13 INFO TaskSetManager: Starting task 120.0 in stage 24.0 (TID 431) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:13 INFO TaskSetManager: Finished task 116.0 in stage 24.0 (TID 427) in 484 ms on 10.0.0.133 (executor driver) (117/200) -26/04/01 05:57:13 INFO Executor: Running task 120.0 in stage 24.0 (TID 431) -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO Executor: Finished task 117.0 in stage 24.0 (TID 428). 10893 bytes result sent to driver -26/04/01 05:57:14 INFO TaskSetManager: Starting task 121.0 in stage 24.0 (TID 432) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:14 INFO Executor: Running task 121.0 in stage 24.0 (TID 432) -26/04/01 05:57:14 INFO TaskSetManager: Finished task 117.0 in stage 24.0 (TID 428) in 487 ms on 10.0.0.133 (executor driver) (118/200) -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO Executor: Finished task 118.0 in stage 24.0 (TID 429). 10893 bytes result sent to driver -26/04/01 05:57:14 INFO TaskSetManager: Starting task 122.0 in stage 24.0 (TID 433) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:14 INFO TaskSetManager: Finished task 118.0 in stage 24.0 (TID 429) in 497 ms on 10.0.0.133 (executor driver) (119/200) -26/04/01 05:57:14 INFO Executor: Running task 122.0 in stage 24.0 (TID 433) -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO Executor: Finished task 119.0 in stage 24.0 (TID 430). 10850 bytes result sent to driver -26/04/01 05:57:14 INFO TaskSetManager: Starting task 123.0 in stage 24.0 (TID 434) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:14 INFO Executor: Running task 123.0 in stage 24.0 (TID 434) -26/04/01 05:57:14 INFO TaskSetManager: Finished task 119.0 in stage 24.0 (TID 430) in 499 ms on 10.0.0.133 (executor driver) (120/200) -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO Executor: Finished task 120.0 in stage 24.0 (TID 431). 10893 bytes result sent to driver -26/04/01 05:57:14 INFO TaskSetManager: Starting task 124.0 in stage 24.0 (TID 435) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:14 INFO Executor: Running task 124.0 in stage 24.0 (TID 435) -26/04/01 05:57:14 INFO TaskSetManager: Finished task 120.0 in stage 24.0 (TID 431) in 486 ms on 10.0.0.133 (executor driver) (121/200) -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO Executor: Finished task 121.0 in stage 24.0 (TID 432). 10850 bytes result sent to driver -26/04/01 05:57:14 INFO TaskSetManager: Starting task 125.0 in stage 24.0 (TID 436) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:14 INFO Executor: Running task 125.0 in stage 24.0 (TID 436) -26/04/01 05:57:14 INFO TaskSetManager: Finished task 121.0 in stage 24.0 (TID 432) in 483 ms on 10.0.0.133 (executor driver) (122/200) -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO Executor: Finished task 122.0 in stage 24.0 (TID 433). 10893 bytes result sent to driver -26/04/01 05:57:14 INFO TaskSetManager: Starting task 126.0 in stage 24.0 (TID 437) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:14 INFO Executor: Finished task 123.0 in stage 24.0 (TID 434). 10850 bytes result sent to driver -26/04/01 05:57:14 INFO Executor: Running task 126.0 in stage 24.0 (TID 437) -26/04/01 05:57:14 INFO TaskSetManager: Starting task 127.0 in stage 24.0 (TID 438) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:14 INFO Executor: Running task 127.0 in stage 24.0 (TID 438) -26/04/01 05:57:14 INFO TaskSetManager: Finished task 122.0 in stage 24.0 (TID 433) in 523 ms on 10.0.0.133 (executor driver) (123/200) -26/04/01 05:57:14 INFO TaskSetManager: Finished task 123.0 in stage 24.0 (TID 434) in 513 ms on 10.0.0.133 (executor driver) (124/200) -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO Executor: Finished task 124.0 in stage 24.0 (TID 435). 10893 bytes result sent to driver -26/04/01 05:57:14 INFO TaskSetManager: Starting task 128.0 in stage 24.0 (TID 439) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:14 INFO TaskSetManager: Finished task 124.0 in stage 24.0 (TID 435) in 482 ms on 10.0.0.133 (executor driver) (125/200) -26/04/01 05:57:14 INFO Executor: Running task 128.0 in stage 24.0 (TID 439) -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO Executor: Finished task 125.0 in stage 24.0 (TID 436). 10893 bytes result sent to driver -26/04/01 05:57:15 INFO TaskSetManager: Starting task 129.0 in stage 24.0 (TID 440) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:15 INFO Executor: Running task 129.0 in stage 24.0 (TID 440) -26/04/01 05:57:15 INFO TaskSetManager: Finished task 125.0 in stage 24.0 (TID 436) in 483 ms on 10.0.0.133 (executor driver) (126/200) -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO Executor: Finished task 126.0 in stage 24.0 (TID 437). 10850 bytes result sent to driver -26/04/01 05:57:15 INFO TaskSetManager: Starting task 130.0 in stage 24.0 (TID 441) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:15 INFO TaskSetManager: Finished task 126.0 in stage 24.0 (TID 437) in 495 ms on 10.0.0.133 (executor driver) (127/200) -26/04/01 05:57:15 INFO Executor: Running task 130.0 in stage 24.0 (TID 441) -26/04/01 05:57:15 INFO Executor: Finished task 127.0 in stage 24.0 (TID 438). 10807 bytes result sent to driver -26/04/01 05:57:15 INFO TaskSetManager: Starting task 131.0 in stage 24.0 (TID 442) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:15 INFO Executor: Running task 131.0 in stage 24.0 (TID 442) -26/04/01 05:57:15 INFO TaskSetManager: Finished task 127.0 in stage 24.0 (TID 438) in 496 ms on 10.0.0.133 (executor driver) (128/200) -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO Executor: Finished task 128.0 in stage 24.0 (TID 439). 10850 bytes result sent to driver -26/04/01 05:57:15 INFO TaskSetManager: Starting task 132.0 in stage 24.0 (TID 443) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:15 INFO Executor: Running task 132.0 in stage 24.0 (TID 443) -26/04/01 05:57:15 INFO TaskSetManager: Finished task 128.0 in stage 24.0 (TID 439) in 483 ms on 10.0.0.133 (executor driver) (129/200) -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO Executor: Finished task 129.0 in stage 24.0 (TID 440). 10893 bytes result sent to driver -26/04/01 05:57:15 INFO TaskSetManager: Starting task 133.0 in stage 24.0 (TID 444) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:15 INFO Executor: Running task 133.0 in stage 24.0 (TID 444) -26/04/01 05:57:15 INFO TaskSetManager: Finished task 129.0 in stage 24.0 (TID 440) in 484 ms on 10.0.0.133 (executor driver) (130/200) -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO Executor: Finished task 131.0 in stage 24.0 (TID 442). 10850 bytes result sent to driver -26/04/01 05:57:15 INFO TaskSetManager: Starting task 134.0 in stage 24.0 (TID 445) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:15 INFO Executor: Running task 134.0 in stage 24.0 (TID 445) -26/04/01 05:57:15 INFO TaskSetManager: Finished task 131.0 in stage 24.0 (TID 442) in 494 ms on 10.0.0.133 (executor driver) (131/200) -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO Executor: Finished task 130.0 in stage 24.0 (TID 441). 10893 bytes result sent to driver -26/04/01 05:57:15 INFO TaskSetManager: Starting task 135.0 in stage 24.0 (TID 446) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:15 INFO Executor: Running task 135.0 in stage 24.0 (TID 446) -26/04/01 05:57:15 INFO TaskSetManager: Finished task 130.0 in stage 24.0 (TID 441) in 501 ms on 10.0.0.133 (executor driver) (132/200) -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO Executor: Finished task 132.0 in stage 24.0 (TID 443). 10893 bytes result sent to driver -26/04/01 05:57:15 INFO TaskSetManager: Starting task 136.0 in stage 24.0 (TID 447) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:15 INFO Executor: Running task 136.0 in stage 24.0 (TID 447) -26/04/01 05:57:15 INFO TaskSetManager: Finished task 132.0 in stage 24.0 (TID 443) in 486 ms on 10.0.0.133 (executor driver) (133/200) -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:15 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO Executor: Finished task 133.0 in stage 24.0 (TID 444). 10850 bytes result sent to driver -26/04/01 05:57:16 INFO TaskSetManager: Starting task 137.0 in stage 24.0 (TID 448) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:16 INFO Executor: Running task 137.0 in stage 24.0 (TID 448) -26/04/01 05:57:16 INFO TaskSetManager: Finished task 133.0 in stage 24.0 (TID 444) in 489 ms on 10.0.0.133 (executor driver) (134/200) -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO Executor: Finished task 134.0 in stage 24.0 (TID 445). 10850 bytes result sent to driver -26/04/01 05:57:16 INFO TaskSetManager: Starting task 138.0 in stage 24.0 (TID 449) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:16 INFO Executor: Running task 138.0 in stage 24.0 (TID 449) -26/04/01 05:57:16 INFO TaskSetManager: Finished task 134.0 in stage 24.0 (TID 445) in 496 ms on 10.0.0.133 (executor driver) (135/200) -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO Executor: Finished task 135.0 in stage 24.0 (TID 446). 10850 bytes result sent to driver -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO TaskSetManager: Starting task 139.0 in stage 24.0 (TID 450) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:16 INFO Executor: Running task 139.0 in stage 24.0 (TID 450) -26/04/01 05:57:16 INFO TaskSetManager: Finished task 135.0 in stage 24.0 (TID 446) in 495 ms on 10.0.0.133 (executor driver) (136/200) -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO Executor: Finished task 136.0 in stage 24.0 (TID 447). 10850 bytes result sent to driver -26/04/01 05:57:16 INFO TaskSetManager: Starting task 140.0 in stage 24.0 (TID 451) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:16 INFO Executor: Running task 140.0 in stage 24.0 (TID 451) -26/04/01 05:57:16 INFO TaskSetManager: Finished task 136.0 in stage 24.0 (TID 447) in 483 ms on 10.0.0.133 (executor driver) (137/200) -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO Executor: Finished task 137.0 in stage 24.0 (TID 448). 10893 bytes result sent to driver -26/04/01 05:57:16 INFO TaskSetManager: Starting task 141.0 in stage 24.0 (TID 452) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:16 INFO Executor: Running task 141.0 in stage 24.0 (TID 452) -26/04/01 05:57:16 INFO TaskSetManager: Finished task 137.0 in stage 24.0 (TID 448) in 490 ms on 10.0.0.133 (executor driver) (138/200) -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO Executor: Finished task 139.0 in stage 24.0 (TID 450). 10807 bytes result sent to driver -26/04/01 05:57:16 INFO Executor: Finished task 138.0 in stage 24.0 (TID 449). 10850 bytes result sent to driver -26/04/01 05:57:16 INFO TaskSetManager: Starting task 142.0 in stage 24.0 (TID 453) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:16 INFO TaskSetManager: Starting task 143.0 in stage 24.0 (TID 454) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:16 INFO Executor: Running task 142.0 in stage 24.0 (TID 453) -26/04/01 05:57:16 INFO TaskSetManager: Finished task 139.0 in stage 24.0 (TID 450) in 529 ms on 10.0.0.133 (executor driver) (139/200) -26/04/01 05:57:16 INFO Executor: Running task 143.0 in stage 24.0 (TID 454) -26/04/01 05:57:16 INFO TaskSetManager: Finished task 138.0 in stage 24.0 (TID 449) in 533 ms on 10.0.0.133 (executor driver) (140/200) -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO Executor: Finished task 140.0 in stage 24.0 (TID 451). 10807 bytes result sent to driver -26/04/01 05:57:16 INFO TaskSetManager: Starting task 144.0 in stage 24.0 (TID 455) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:16 INFO TaskSetManager: Finished task 140.0 in stage 24.0 (TID 451) in 501 ms on 10.0.0.133 (executor driver) (141/200) -26/04/01 05:57:16 INFO Executor: Running task 144.0 in stage 24.0 (TID 455) -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:17 INFO Executor: Finished task 141.0 in stage 24.0 (TID 452). 10850 bytes result sent to driver -26/04/01 05:57:17 INFO TaskSetManager: Starting task 145.0 in stage 24.0 (TID 456) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:17 INFO Executor: Running task 145.0 in stage 24.0 (TID 456) -26/04/01 05:57:17 INFO TaskSetManager: Finished task 141.0 in stage 24.0 (TID 452) in 488 ms on 10.0.0.133 (executor driver) (142/200) -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:17 INFO Executor: Finished task 143.0 in stage 24.0 (TID 454). 10893 bytes result sent to driver -26/04/01 05:57:17 INFO TaskSetManager: Starting task 146.0 in stage 24.0 (TID 457) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:17 INFO TaskSetManager: Finished task 143.0 in stage 24.0 (TID 454) in 521 ms on 10.0.0.133 (executor driver) (143/200) -26/04/01 05:57:17 INFO Executor: Running task 146.0 in stage 24.0 (TID 457) -26/04/01 05:57:17 INFO Executor: Finished task 142.0 in stage 24.0 (TID 453). 10893 bytes result sent to driver -26/04/01 05:57:17 INFO TaskSetManager: Starting task 147.0 in stage 24.0 (TID 458) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:17 INFO Executor: Running task 147.0 in stage 24.0 (TID 458) -26/04/01 05:57:17 INFO TaskSetManager: Finished task 142.0 in stage 24.0 (TID 453) in 522 ms on 10.0.0.133 (executor driver) (144/200) -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 8 ms -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 8 ms -26/04/01 05:57:17 INFO Executor: Finished task 144.0 in stage 24.0 (TID 455). 10893 bytes result sent to driver -26/04/01 05:57:17 INFO TaskSetManager: Starting task 148.0 in stage 24.0 (TID 459) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:17 INFO Executor: Running task 148.0 in stage 24.0 (TID 459) -26/04/01 05:57:17 INFO TaskSetManager: Finished task 144.0 in stage 24.0 (TID 455) in 542 ms on 10.0.0.133 (executor driver) (145/200) -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:17 INFO Executor: Finished task 145.0 in stage 24.0 (TID 456). 10893 bytes result sent to driver -26/04/01 05:57:17 INFO TaskSetManager: Starting task 149.0 in stage 24.0 (TID 460) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:17 INFO Executor: Running task 149.0 in stage 24.0 (TID 460) -26/04/01 05:57:17 INFO TaskSetManager: Finished task 145.0 in stage 24.0 (TID 456) in 495 ms on 10.0.0.133 (executor driver) (146/200) -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:17 INFO Executor: Finished task 147.0 in stage 24.0 (TID 458). 10807 bytes result sent to driver -26/04/01 05:57:17 INFO TaskSetManager: Starting task 150.0 in stage 24.0 (TID 461) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:17 INFO Executor: Running task 150.0 in stage 24.0 (TID 461) -26/04/01 05:57:17 INFO TaskSetManager: Finished task 147.0 in stage 24.0 (TID 458) in 504 ms on 10.0.0.133 (executor driver) (147/200) -26/04/01 05:57:17 INFO Executor: Finished task 146.0 in stage 24.0 (TID 457). 10850 bytes result sent to driver -26/04/01 05:57:17 INFO TaskSetManager: Starting task 151.0 in stage 24.0 (TID 462) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:17 INFO Executor: Running task 151.0 in stage 24.0 (TID 462) -26/04/01 05:57:17 INFO TaskSetManager: Finished task 146.0 in stage 24.0 (TID 457) in 509 ms on 10.0.0.133 (executor driver) (148/200) -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:17 INFO Executor: Finished task 148.0 in stage 24.0 (TID 459). 10807 bytes result sent to driver -26/04/01 05:57:17 INFO TaskSetManager: Starting task 152.0 in stage 24.0 (TID 463) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:17 INFO Executor: Running task 152.0 in stage 24.0 (TID 463) -26/04/01 05:57:17 INFO TaskSetManager: Finished task 148.0 in stage 24.0 (TID 459) in 494 ms on 10.0.0.133 (executor driver) (149/200) -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO Executor: Finished task 149.0 in stage 24.0 (TID 460). 10850 bytes result sent to driver -26/04/01 05:57:18 INFO TaskSetManager: Starting task 153.0 in stage 24.0 (TID 464) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:18 INFO Executor: Running task 153.0 in stage 24.0 (TID 464) -26/04/01 05:57:18 INFO TaskSetManager: Finished task 149.0 in stage 24.0 (TID 460) in 483 ms on 10.0.0.133 (executor driver) (150/200) -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO Executor: Finished task 150.0 in stage 24.0 (TID 461). 10893 bytes result sent to driver -26/04/01 05:57:18 INFO TaskSetManager: Starting task 154.0 in stage 24.0 (TID 465) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:18 INFO Executor: Running task 154.0 in stage 24.0 (TID 465) -26/04/01 05:57:18 INFO TaskSetManager: Finished task 150.0 in stage 24.0 (TID 461) in 503 ms on 10.0.0.133 (executor driver) (151/200) -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO Executor: Finished task 151.0 in stage 24.0 (TID 462). 10850 bytes result sent to driver -26/04/01 05:57:18 INFO TaskSetManager: Starting task 155.0 in stage 24.0 (TID 466) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:18 INFO Executor: Running task 155.0 in stage 24.0 (TID 466) -26/04/01 05:57:18 INFO TaskSetManager: Finished task 151.0 in stage 24.0 (TID 462) in 506 ms on 10.0.0.133 (executor driver) (152/200) -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO Executor: Finished task 152.0 in stage 24.0 (TID 463). 10850 bytes result sent to driver -26/04/01 05:57:18 INFO TaskSetManager: Starting task 156.0 in stage 24.0 (TID 467) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:18 INFO Executor: Running task 156.0 in stage 24.0 (TID 467) -26/04/01 05:57:18 INFO TaskSetManager: Finished task 152.0 in stage 24.0 (TID 463) in 491 ms on 10.0.0.133 (executor driver) (153/200) -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO Executor: Finished task 153.0 in stage 24.0 (TID 464). 10850 bytes result sent to driver -26/04/01 05:57:18 INFO TaskSetManager: Starting task 157.0 in stage 24.0 (TID 468) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:18 INFO Executor: Running task 157.0 in stage 24.0 (TID 468) -26/04/01 05:57:18 INFO TaskSetManager: Finished task 153.0 in stage 24.0 (TID 464) in 485 ms on 10.0.0.133 (executor driver) (154/200) -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO Executor: Finished task 155.0 in stage 24.0 (TID 466). 10850 bytes result sent to driver -26/04/01 05:57:18 INFO TaskSetManager: Starting task 158.0 in stage 24.0 (TID 469) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:18 INFO Executor: Running task 158.0 in stage 24.0 (TID 469) -26/04/01 05:57:18 INFO TaskSetManager: Finished task 155.0 in stage 24.0 (TID 466) in 510 ms on 10.0.0.133 (executor driver) (155/200) -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO Executor: Finished task 154.0 in stage 24.0 (TID 465). 10893 bytes result sent to driver -26/04/01 05:57:18 INFO TaskSetManager: Starting task 159.0 in stage 24.0 (TID 470) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:18 INFO Executor: Running task 159.0 in stage 24.0 (TID 470) -26/04/01 05:57:18 INFO TaskSetManager: Finished task 154.0 in stage 24.0 (TID 465) in 528 ms on 10.0.0.133 (executor driver) (156/200) -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO Executor: Finished task 156.0 in stage 24.0 (TID 467). 10850 bytes result sent to driver -26/04/01 05:57:18 INFO TaskSetManager: Starting task 160.0 in stage 24.0 (TID 471) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:18 INFO Executor: Running task 160.0 in stage 24.0 (TID 471) -26/04/01 05:57:18 INFO TaskSetManager: Finished task 156.0 in stage 24.0 (TID 467) in 505 ms on 10.0.0.133 (executor driver) (157/200) -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO Executor: Finished task 157.0 in stage 24.0 (TID 468). 10893 bytes result sent to driver -26/04/01 05:57:19 INFO TaskSetManager: Starting task 161.0 in stage 24.0 (TID 472) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:19 INFO Executor: Running task 161.0 in stage 24.0 (TID 472) -26/04/01 05:57:19 INFO TaskSetManager: Finished task 157.0 in stage 24.0 (TID 468) in 490 ms on 10.0.0.133 (executor driver) (158/200) -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO Executor: Finished task 158.0 in stage 24.0 (TID 469). 10850 bytes result sent to driver -26/04/01 05:57:19 INFO TaskSetManager: Starting task 162.0 in stage 24.0 (TID 473) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:19 INFO Executor: Running task 162.0 in stage 24.0 (TID 473) -26/04/01 05:57:19 INFO TaskSetManager: Finished task 158.0 in stage 24.0 (TID 469) in 501 ms on 10.0.0.133 (executor driver) (159/200) -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO Executor: Finished task 160.0 in stage 24.0 (TID 471). 10807 bytes result sent to driver -26/04/01 05:57:19 INFO TaskSetManager: Starting task 163.0 in stage 24.0 (TID 474) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:19 INFO Executor: Running task 163.0 in stage 24.0 (TID 474) -26/04/01 05:57:19 INFO TaskSetManager: Finished task 160.0 in stage 24.0 (TID 471) in 498 ms on 10.0.0.133 (executor driver) (160/200) -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO Executor: Finished task 159.0 in stage 24.0 (TID 470). 10807 bytes result sent to driver -26/04/01 05:57:19 INFO TaskSetManager: Starting task 164.0 in stage 24.0 (TID 475) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:19 INFO TaskSetManager: Finished task 159.0 in stage 24.0 (TID 470) in 504 ms on 10.0.0.133 (executor driver) (161/200) -26/04/01 05:57:19 INFO Executor: Running task 164.0 in stage 24.0 (TID 475) -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO Executor: Finished task 161.0 in stage 24.0 (TID 472). 10807 bytes result sent to driver -26/04/01 05:57:19 INFO TaskSetManager: Starting task 165.0 in stage 24.0 (TID 476) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:19 INFO Executor: Running task 165.0 in stage 24.0 (TID 476) -26/04/01 05:57:19 INFO TaskSetManager: Finished task 161.0 in stage 24.0 (TID 472) in 481 ms on 10.0.0.133 (executor driver) (162/200) -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO Executor: Finished task 162.0 in stage 24.0 (TID 473). 10850 bytes result sent to driver -26/04/01 05:57:19 INFO TaskSetManager: Starting task 166.0 in stage 24.0 (TID 477) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:19 INFO Executor: Running task 166.0 in stage 24.0 (TID 477) -26/04/01 05:57:19 INFO TaskSetManager: Finished task 162.0 in stage 24.0 (TID 473) in 498 ms on 10.0.0.133 (executor driver) (163/200) -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO Executor: Finished task 163.0 in stage 24.0 (TID 474). 10850 bytes result sent to driver -26/04/01 05:57:19 INFO TaskSetManager: Starting task 167.0 in stage 24.0 (TID 478) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:19 INFO Executor: Running task 167.0 in stage 24.0 (TID 478) -26/04/01 05:57:19 INFO TaskSetManager: Finished task 163.0 in stage 24.0 (TID 474) in 497 ms on 10.0.0.133 (executor driver) (164/200) -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO Executor: Finished task 164.0 in stage 24.0 (TID 475). 10850 bytes result sent to driver -26/04/01 05:57:19 INFO TaskSetManager: Starting task 168.0 in stage 24.0 (TID 479) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:19 INFO Executor: Running task 168.0 in stage 24.0 (TID 479) -26/04/01 05:57:19 INFO TaskSetManager: Finished task 164.0 in stage 24.0 (TID 475) in 498 ms on 10.0.0.133 (executor driver) (165/200) -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Getting 208 (55.8 MiB) non-empty blocks including 208 (55.8 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO Executor: Finished task 165.0 in stage 24.0 (TID 476). 10850 bytes result sent to driver -26/04/01 05:57:20 INFO TaskSetManager: Starting task 169.0 in stage 24.0 (TID 480) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:20 INFO Executor: Running task 169.0 in stage 24.0 (TID 480) -26/04/01 05:57:20 INFO TaskSetManager: Finished task 165.0 in stage 24.0 (TID 476) in 488 ms on 10.0.0.133 (executor driver) (166/200) -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO Executor: Finished task 166.0 in stage 24.0 (TID 477). 10893 bytes result sent to driver -26/04/01 05:57:20 INFO TaskSetManager: Starting task 170.0 in stage 24.0 (TID 481) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:20 INFO TaskSetManager: Finished task 166.0 in stage 24.0 (TID 477) in 502 ms on 10.0.0.133 (executor driver) (167/200) -26/04/01 05:57:20 INFO Executor: Running task 170.0 in stage 24.0 (TID 481) -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO Executor: Finished task 167.0 in stage 24.0 (TID 478). 10850 bytes result sent to driver -26/04/01 05:57:20 INFO TaskSetManager: Starting task 171.0 in stage 24.0 (TID 482) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:20 INFO Executor: Finished task 168.0 in stage 24.0 (TID 479). 10850 bytes result sent to driver -26/04/01 05:57:20 INFO Executor: Running task 171.0 in stage 24.0 (TID 482) -26/04/01 05:57:20 INFO TaskSetManager: Finished task 167.0 in stage 24.0 (TID 478) in 510 ms on 10.0.0.133 (executor driver) (168/200) -26/04/01 05:57:20 INFO TaskSetManager: Starting task 172.0 in stage 24.0 (TID 483) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:20 INFO Executor: Running task 172.0 in stage 24.0 (TID 483) -26/04/01 05:57:20 INFO TaskSetManager: Finished task 168.0 in stage 24.0 (TID 479) in 505 ms on 10.0.0.133 (executor driver) (169/200) -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO Executor: Finished task 169.0 in stage 24.0 (TID 480). 10850 bytes result sent to driver -26/04/01 05:57:20 INFO TaskSetManager: Starting task 173.0 in stage 24.0 (TID 484) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:20 INFO Executor: Running task 173.0 in stage 24.0 (TID 484) -26/04/01 05:57:20 INFO TaskSetManager: Finished task 169.0 in stage 24.0 (TID 480) in 481 ms on 10.0.0.133 (executor driver) (170/200) -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO Executor: Finished task 170.0 in stage 24.0 (TID 481). 10850 bytes result sent to driver -26/04/01 05:57:20 INFO TaskSetManager: Starting task 174.0 in stage 24.0 (TID 485) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:20 INFO Executor: Running task 174.0 in stage 24.0 (TID 485) -26/04/01 05:57:20 INFO TaskSetManager: Finished task 170.0 in stage 24.0 (TID 481) in 497 ms on 10.0.0.133 (executor driver) (171/200) -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO Executor: Finished task 172.0 in stage 24.0 (TID 483). 10807 bytes result sent to driver -26/04/01 05:57:20 INFO TaskSetManager: Starting task 175.0 in stage 24.0 (TID 486) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:20 INFO TaskSetManager: Finished task 172.0 in stage 24.0 (TID 483) in 500 ms on 10.0.0.133 (executor driver) (172/200) -26/04/01 05:57:20 INFO Executor: Running task 175.0 in stage 24.0 (TID 486) -26/04/01 05:57:20 INFO Executor: Finished task 171.0 in stage 24.0 (TID 482). 10807 bytes result sent to driver -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1140.9 KiB) non-empty blocks including 7 (1140.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO TaskSetManager: Starting task 176.0 in stage 24.0 (TID 487) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO TaskSetManager: Finished task 171.0 in stage 24.0 (TID 482) in 501 ms on 10.0.0.133 (executor driver) (173/200) -26/04/01 05:57:20 INFO Executor: Running task 176.0 in stage 24.0 (TID 487) -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO Executor: Finished task 173.0 in stage 24.0 (TID 484). 10807 bytes result sent to driver -26/04/01 05:57:21 INFO TaskSetManager: Starting task 177.0 in stage 24.0 (TID 488) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:21 INFO TaskSetManager: Finished task 173.0 in stage 24.0 (TID 484) in 480 ms on 10.0.0.133 (executor driver) (174/200) -26/04/01 05:57:21 INFO Executor: Running task 177.0 in stage 24.0 (TID 488) -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO Executor: Finished task 174.0 in stage 24.0 (TID 485). 10893 bytes result sent to driver -26/04/01 05:57:21 INFO TaskSetManager: Starting task 178.0 in stage 24.0 (TID 489) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:21 INFO Executor: Running task 178.0 in stage 24.0 (TID 489) -26/04/01 05:57:21 INFO TaskSetManager: Finished task 174.0 in stage 24.0 (TID 485) in 496 ms on 10.0.0.133 (executor driver) (175/200) -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO Executor: Finished task 175.0 in stage 24.0 (TID 486). 10850 bytes result sent to driver -26/04/01 05:57:21 INFO TaskSetManager: Starting task 179.0 in stage 24.0 (TID 490) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:21 INFO TaskSetManager: Finished task 175.0 in stage 24.0 (TID 486) in 508 ms on 10.0.0.133 (executor driver) (176/200) -26/04/01 05:57:21 INFO Executor: Running task 179.0 in stage 24.0 (TID 490) -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO Executor: Finished task 176.0 in stage 24.0 (TID 487). 10850 bytes result sent to driver -26/04/01 05:57:21 INFO TaskSetManager: Starting task 180.0 in stage 24.0 (TID 491) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:21 INFO Executor: Running task 180.0 in stage 24.0 (TID 491) -26/04/01 05:57:21 INFO TaskSetManager: Finished task 176.0 in stage 24.0 (TID 487) in 512 ms on 10.0.0.133 (executor driver) (177/200) -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO Executor: Finished task 177.0 in stage 24.0 (TID 488). 10893 bytes result sent to driver -26/04/01 05:57:21 INFO TaskSetManager: Starting task 181.0 in stage 24.0 (TID 492) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:21 INFO Executor: Running task 181.0 in stage 24.0 (TID 492) -26/04/01 05:57:21 INFO TaskSetManager: Finished task 177.0 in stage 24.0 (TID 488) in 485 ms on 10.0.0.133 (executor driver) (178/200) -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO Executor: Finished task 178.0 in stage 24.0 (TID 489). 10893 bytes result sent to driver -26/04/01 05:57:21 INFO TaskSetManager: Starting task 182.0 in stage 24.0 (TID 493) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:21 INFO Executor: Running task 182.0 in stage 24.0 (TID 493) -26/04/01 05:57:21 INFO TaskSetManager: Finished task 178.0 in stage 24.0 (TID 489) in 494 ms on 10.0.0.133 (executor driver) (179/200) -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO Executor: Finished task 180.0 in stage 24.0 (TID 491). 10850 bytes result sent to driver -26/04/01 05:57:21 INFO TaskSetManager: Starting task 183.0 in stage 24.0 (TID 494) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:21 INFO TaskSetManager: Finished task 180.0 in stage 24.0 (TID 491) in 491 ms on 10.0.0.133 (executor driver) (180/200) -26/04/01 05:57:21 INFO Executor: Running task 183.0 in stage 24.0 (TID 494) -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO Executor: Finished task 179.0 in stage 24.0 (TID 490). 10850 bytes result sent to driver -26/04/01 05:57:21 INFO TaskSetManager: Starting task 184.0 in stage 24.0 (TID 495) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:21 INFO Executor: Running task 184.0 in stage 24.0 (TID 495) -26/04/01 05:57:21 INFO TaskSetManager: Finished task 179.0 in stage 24.0 (TID 490) in 499 ms on 10.0.0.133 (executor driver) (181/200) -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO Executor: Finished task 181.0 in stage 24.0 (TID 492). 10850 bytes result sent to driver -26/04/01 05:57:22 INFO TaskSetManager: Starting task 185.0 in stage 24.0 (TID 496) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:22 INFO Executor: Running task 185.0 in stage 24.0 (TID 496) -26/04/01 05:57:22 INFO TaskSetManager: Finished task 181.0 in stage 24.0 (TID 492) in 484 ms on 10.0.0.133 (executor driver) (182/200) -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO Executor: Finished task 182.0 in stage 24.0 (TID 493). 10850 bytes result sent to driver -26/04/01 05:57:22 INFO TaskSetManager: Starting task 186.0 in stage 24.0 (TID 497) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:22 INFO Executor: Running task 186.0 in stage 24.0 (TID 497) -26/04/01 05:57:22 INFO TaskSetManager: Finished task 182.0 in stage 24.0 (TID 493) in 491 ms on 10.0.0.133 (executor driver) (183/200) -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO Executor: Finished task 183.0 in stage 24.0 (TID 494). 10807 bytes result sent to driver -26/04/01 05:57:22 INFO TaskSetManager: Starting task 187.0 in stage 24.0 (TID 498) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:22 INFO TaskSetManager: Finished task 183.0 in stage 24.0 (TID 494) in 514 ms on 10.0.0.133 (executor driver) (184/200) -26/04/01 05:57:22 INFO Executor: Running task 187.0 in stage 24.0 (TID 498) -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO Executor: Finished task 184.0 in stage 24.0 (TID 495). 10807 bytes result sent to driver -26/04/01 05:57:22 INFO TaskSetManager: Starting task 188.0 in stage 24.0 (TID 499) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:22 INFO Executor: Running task 188.0 in stage 24.0 (TID 499) -26/04/01 05:57:22 INFO TaskSetManager: Finished task 184.0 in stage 24.0 (TID 495) in 515 ms on 10.0.0.133 (executor driver) (185/200) -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO Executor: Finished task 185.0 in stage 24.0 (TID 496). 10807 bytes result sent to driver -26/04/01 05:57:22 INFO TaskSetManager: Starting task 189.0 in stage 24.0 (TID 500) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:22 INFO Executor: Running task 189.0 in stage 24.0 (TID 500) -26/04/01 05:57:22 INFO TaskSetManager: Finished task 185.0 in stage 24.0 (TID 496) in 498 ms on 10.0.0.133 (executor driver) (186/200) -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO Executor: Finished task 186.0 in stage 24.0 (TID 497). 10893 bytes result sent to driver -26/04/01 05:57:22 INFO TaskSetManager: Starting task 190.0 in stage 24.0 (TID 501) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:22 INFO Executor: Running task 190.0 in stage 24.0 (TID 501) -26/04/01 05:57:22 INFO TaskSetManager: Finished task 186.0 in stage 24.0 (TID 497) in 489 ms on 10.0.0.133 (executor driver) (187/200) -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO Executor: Finished task 187.0 in stage 24.0 (TID 498). 10850 bytes result sent to driver -26/04/01 05:57:22 INFO TaskSetManager: Starting task 191.0 in stage 24.0 (TID 502) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:22 INFO Executor: Running task 191.0 in stage 24.0 (TID 502) -26/04/01 05:57:22 INFO TaskSetManager: Finished task 187.0 in stage 24.0 (TID 498) in 495 ms on 10.0.0.133 (executor driver) (188/200) -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1176.2 KiB) non-empty blocks including 7 (1176.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO Executor: Finished task 188.0 in stage 24.0 (TID 499). 10850 bytes result sent to driver -26/04/01 05:57:22 INFO TaskSetManager: Starting task 192.0 in stage 24.0 (TID 503) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:22 INFO Executor: Running task 192.0 in stage 24.0 (TID 503) -26/04/01 05:57:22 INFO TaskSetManager: Finished task 188.0 in stage 24.0 (TID 499) in 498 ms on 10.0.0.133 (executor driver) (189/200) -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1211.5 KiB) non-empty blocks including 7 (1211.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO Executor: Finished task 189.0 in stage 24.0 (TID 500). 10850 bytes result sent to driver -26/04/01 05:57:22 INFO TaskSetManager: Starting task 193.0 in stage 24.0 (TID 504) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:22 INFO TaskSetManager: Finished task 189.0 in stage 24.0 (TID 500) in 486 ms on 10.0.0.133 (executor driver) (190/200) -26/04/01 05:57:22 INFO Executor: Running task 193.0 in stage 24.0 (TID 504) -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:23 INFO Executor: Finished task 190.0 in stage 24.0 (TID 501). 10893 bytes result sent to driver -26/04/01 05:57:23 INFO TaskSetManager: Starting task 194.0 in stage 24.0 (TID 505) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:23 INFO Executor: Running task 194.0 in stage 24.0 (TID 505) -26/04/01 05:57:23 INFO TaskSetManager: Finished task 190.0 in stage 24.0 (TID 501) in 483 ms on 10.0.0.133 (executor driver) (191/200) -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:23 INFO Executor: Finished task 191.0 in stage 24.0 (TID 502). 10850 bytes result sent to driver -26/04/01 05:57:23 INFO TaskSetManager: Starting task 195.0 in stage 24.0 (TID 506) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:23 INFO TaskSetManager: Finished task 191.0 in stage 24.0 (TID 502) in 496 ms on 10.0.0.133 (executor driver) (192/200) -26/04/01 05:57:23 INFO Executor: Running task 195.0 in stage 24.0 (TID 506) -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:23 INFO Executor: Finished task 192.0 in stage 24.0 (TID 503). 10850 bytes result sent to driver -26/04/01 05:57:23 INFO TaskSetManager: Starting task 196.0 in stage 24.0 (TID 507) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:23 INFO Executor: Running task 196.0 in stage 24.0 (TID 507) -26/04/01 05:57:23 INFO TaskSetManager: Finished task 192.0 in stage 24.0 (TID 503) in 501 ms on 10.0.0.133 (executor driver) (193/200) -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:23 INFO Executor: Finished task 193.0 in stage 24.0 (TID 504). 10850 bytes result sent to driver -26/04/01 05:57:23 INFO TaskSetManager: Starting task 197.0 in stage 24.0 (TID 508) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:23 INFO TaskSetManager: Finished task 193.0 in stage 24.0 (TID 504) in 491 ms on 10.0.0.133 (executor driver) (194/200) -26/04/01 05:57:23 INFO Executor: Running task 197.0 in stage 24.0 (TID 508) -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 208 (56.0 MiB) non-empty blocks including 208 (56.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:23 INFO Executor: Finished task 194.0 in stage 24.0 (TID 505). 10850 bytes result sent to driver -26/04/01 05:57:23 INFO TaskSetManager: Starting task 198.0 in stage 24.0 (TID 509) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:23 INFO Executor: Running task 198.0 in stage 24.0 (TID 509) -26/04/01 05:57:23 INFO TaskSetManager: Finished task 194.0 in stage 24.0 (TID 505) in 482 ms on 10.0.0.133 (executor driver) (195/200) -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 7 (1158.5 KiB) non-empty blocks including 7 (1158.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:23 INFO Executor: Finished task 195.0 in stage 24.0 (TID 506). 10850 bytes result sent to driver -26/04/01 05:57:23 INFO TaskSetManager: Starting task 199.0 in stage 24.0 (TID 510) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:23 INFO Executor: Running task 199.0 in stage 24.0 (TID 510) -26/04/01 05:57:23 INFO TaskSetManager: Finished task 195.0 in stage 24.0 (TID 506) in 496 ms on 10.0.0.133 (executor driver) (196/200) -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 7 (1193.8 KiB) non-empty blocks including 7 (1193.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Getting 208 (55.9 MiB) non-empty blocks including 208 (55.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:23 INFO Executor: Finished task 196.0 in stage 24.0 (TID 507). 10807 bytes result sent to driver -26/04/01 05:57:23 INFO TaskSetManager: Finished task 196.0 in stage 24.0 (TID 507) in 496 ms on 10.0.0.133 (executor driver) (197/200) -26/04/01 05:57:23 INFO Executor: Finished task 197.0 in stage 24.0 (TID 508). 10807 bytes result sent to driver -26/04/01 05:57:23 INFO TaskSetManager: Finished task 197.0 in stage 24.0 (TID 508) in 494 ms on 10.0.0.133 (executor driver) (198/200) -26/04/01 05:57:24 INFO Executor: Finished task 198.0 in stage 24.0 (TID 509). 10893 bytes result sent to driver -26/04/01 05:57:24 INFO TaskSetManager: Finished task 198.0 in stage 24.0 (TID 509) in 455 ms on 10.0.0.133 (executor driver) (199/200) -26/04/01 05:57:24 INFO Executor: Finished task 199.0 in stage 24.0 (TID 510). 10850 bytes result sent to driver -26/04/01 05:57:24 INFO TaskSetManager: Finished task 199.0 in stage 24.0 (TID 510) in 453 ms on 10.0.0.133 (executor driver) (200/200) -26/04/01 05:57:24 INFO TaskSchedulerImpl: Removed TaskSet 24.0, whose tasks have all completed, from pool -26/04/01 05:57:24 INFO DAGScheduler: ShuffleMapStage 24 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 25.219 s -26/04/01 05:57:24 INFO DAGScheduler: looking for newly runnable stages -26/04/01 05:57:24 INFO DAGScheduler: running: Set() -26/04/01 05:57:24 INFO DAGScheduler: waiting: Set() -26/04/01 05:57:24 INFO DAGScheduler: failed: Set() -26/04/01 05:57:24 INFO ShufflePartitionsUtil: For shuffle(6, 4), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 05:57:24 INFO HashAggregateExec: spark.sql.codegen.aggregate.map.twolevel.enabled is set to true, but current version of codegened fast hashmap does not support this aggregate. -26/04/01 05:57:24 INFO CodeGenerator: Code generated in 9.013417 ms -26/04/01 05:57:24 INFO CodeGenerator: Code generated in 2.152792 ms -26/04/01 05:57:24 INFO CodeGenerator: Code generated in 2.298209 ms -26/04/01 05:57:24 INFO DAGScheduler: Registering RDD 68 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 7 -26/04/01 05:57:24 INFO DAGScheduler: Got map stage job 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 29 output partitions -26/04/01 05:57:24 INFO DAGScheduler: Final stage: ShuffleMapStage 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:57:24 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 30, ShuffleMapStage 29) -26/04/01 05:57:24 INFO DAGScheduler: Missing parents: List() -26/04/01 05:57:24 INFO DAGScheduler: Submitting ShuffleMapStage 31 (MapPartitionsRDD[68] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:57:24 WARN DAGScheduler: Broadcasting large task binary with size 1137.7 KiB -26/04/01 05:57:24 INFO MemoryStore: Block broadcast_27 stored as values in memory (estimated size 1137.7 KiB, free 8.6 GiB) -26/04/01 05:57:24 INFO MemoryStore: Block broadcast_27_piece0 stored as bytes in memory (estimated size 55.3 KiB, free 8.6 GiB) -26/04/01 05:57:24 INFO BlockManagerInfo: Added broadcast_27_piece0 in memory on 10.0.0.133:58206 (size: 55.3 KiB, free: 8.6 GiB) -26/04/01 05:57:24 INFO SparkContext: Created broadcast 27 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:57:24 INFO DAGScheduler: Submitting 29 missing tasks from ShuffleMapStage 31 (MapPartitionsRDD[68] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 05:57:24 INFO TaskSchedulerImpl: Adding task set 31.0 with 29 tasks resource profile 0 -26/04/01 05:57:24 INFO TaskSetManager: Starting task 0.0 in stage 31.0 (TID 511) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:24 INFO TaskSetManager: Starting task 1.0 in stage 31.0 (TID 512) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:24 INFO TaskSetManager: Starting task 2.0 in stage 31.0 (TID 513) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:24 INFO TaskSetManager: Starting task 3.0 in stage 31.0 (TID 514) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:24 INFO Executor: Running task 1.0 in stage 31.0 (TID 512) -26/04/01 05:57:24 INFO Executor: Running task 2.0 in stage 31.0 (TID 513) -26/04/01 05:57:24 INFO Executor: Running task 3.0 in stage 31.0 (TID 514) -26/04/01 05:57:24 INFO Executor: Running task 0.0 in stage 31.0 (TID 511) -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 05:57:24 INFO CodeGenerator: Code generated in 3.116625 ms -26/04/01 05:57:24 INFO CodeGenerator: Code generated in 1.721291 ms -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 4 (323.6 KiB) non-empty blocks including 4 (323.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 4 (327.0 KiB) non-empty blocks including 4 (327.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 4 (325.9 KiB) non-empty blocks including 4 (325.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Getting 4 (323.6 KiB) non-empty blocks including 4 (323.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:24 INFO CodeGenerator: Code generated in 2.260084 ms -26/04/01 05:57:24 INFO CodeGenerator: Code generated in 1.543 ms -26/04/01 05:57:24 INFO CodeGenerator: Code generated in 5.617625 ms -26/04/01 05:57:24 INFO CodeGenerator: Code generated in 1.698208 ms -26/04/01 05:57:24 INFO CodeGenerator: Code generated in 0.855083 ms -26/04/01 05:57:24 INFO CodeGenerator: Code generated in 1.547417 ms -26/04/01 05:57:24 INFO CodeGenerator: Code generated in 1.028583 ms -26/04/01 05:57:26 INFO Executor: Finished task 1.0 in stage 31.0 (TID 512). 19177 bytes result sent to driver -26/04/01 05:57:26 INFO Executor: Finished task 0.0 in stage 31.0 (TID 511). 19177 bytes result sent to driver -26/04/01 05:57:26 INFO TaskSetManager: Starting task 4.0 in stage 31.0 (TID 515) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:26 INFO TaskSetManager: Finished task 1.0 in stage 31.0 (TID 512) in 1818 ms on 10.0.0.133 (executor driver) (1/29) -26/04/01 05:57:26 INFO TaskSetManager: Starting task 5.0 in stage 31.0 (TID 516) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:26 INFO Executor: Running task 4.0 in stage 31.0 (TID 515) -26/04/01 05:57:26 INFO Executor: Running task 5.0 in stage 31.0 (TID 516) -26/04/01 05:57:26 INFO TaskSetManager: Finished task 0.0 in stage 31.0 (TID 511) in 1819 ms on 10.0.0.133 (executor driver) (2/29) -26/04/01 05:57:26 INFO Executor: Finished task 3.0 in stage 31.0 (TID 514). 19177 bytes result sent to driver -26/04/01 05:57:26 INFO TaskSetManager: Starting task 6.0 in stage 31.0 (TID 517) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:26 INFO Executor: Running task 6.0 in stage 31.0 (TID 517) -26/04/01 05:57:26 INFO Executor: Finished task 2.0 in stage 31.0 (TID 513). 19177 bytes result sent to driver -26/04/01 05:57:26 INFO TaskSetManager: Finished task 3.0 in stage 31.0 (TID 514) in 1822 ms on 10.0.0.133 (executor driver) (3/29) -26/04/01 05:57:26 INFO TaskSetManager: Starting task 7.0 in stage 31.0 (TID 518) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:26 INFO Executor: Running task 7.0 in stage 31.0 (TID 518) -26/04/01 05:57:26 INFO TaskSetManager: Finished task 2.0 in stage 31.0 (TID 513) in 1822 ms on 10.0.0.133 (executor driver) (4/29) -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 200 (63.1 MiB) non-empty blocks including 200 (63.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 200 (63.3 MiB) non-empty blocks including 200 (63.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 4 (323.6 KiB) non-empty blocks including 4 (323.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 4 (329.2 KiB) non-empty blocks including 4 (329.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 4 (325.9 KiB) non-empty blocks including 4 (325.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Getting 4 (323.6 KiB) non-empty blocks including 4 (323.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:27 INFO Executor: Finished task 5.0 in stage 31.0 (TID 516). 19177 bytes result sent to driver -26/04/01 05:57:27 INFO TaskSetManager: Starting task 8.0 in stage 31.0 (TID 519) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:27 INFO Executor: Running task 8.0 in stage 31.0 (TID 519) -26/04/01 05:57:27 INFO TaskSetManager: Finished task 5.0 in stage 31.0 (TID 516) in 1661 ms on 10.0.0.133 (executor driver) (5/29) -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 4 (324.7 KiB) non-empty blocks including 4 (324.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:27 INFO Executor: Finished task 7.0 in stage 31.0 (TID 518). 19177 bytes result sent to driver -26/04/01 05:57:27 INFO TaskSetManager: Starting task 9.0 in stage 31.0 (TID 520) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:27 INFO Executor: Running task 9.0 in stage 31.0 (TID 520) -26/04/01 05:57:27 INFO TaskSetManager: Finished task 7.0 in stage 31.0 (TID 518) in 1670 ms on 10.0.0.133 (executor driver) (6/29) -26/04/01 05:57:27 INFO Executor: Finished task 6.0 in stage 31.0 (TID 517). 19177 bytes result sent to driver -26/04/01 05:57:27 INFO TaskSetManager: Starting task 10.0 in stage 31.0 (TID 521) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:27 INFO Executor: Running task 10.0 in stage 31.0 (TID 521) -26/04/01 05:57:27 INFO TaskSetManager: Finished task 6.0 in stage 31.0 (TID 517) in 1670 ms on 10.0.0.133 (executor driver) (7/29) -26/04/01 05:57:27 INFO Executor: Finished task 4.0 in stage 31.0 (TID 515). 19177 bytes result sent to driver -26/04/01 05:57:27 INFO TaskSetManager: Starting task 11.0 in stage 31.0 (TID 522) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:27 INFO Executor: Running task 11.0 in stage 31.0 (TID 522) -26/04/01 05:57:27 INFO TaskSetManager: Finished task 4.0 in stage 31.0 (TID 515) in 1676 ms on 10.0.0.133 (executor driver) (8/29) -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 200 (63.1 MiB) non-empty blocks including 200 (63.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 4 (328.1 KiB) non-empty blocks including 4 (328.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 4 (323.6 KiB) non-empty blocks including 4 (323.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Getting 4 (323.6 KiB) non-empty blocks including 4 (323.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:27 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:29 INFO Executor: Finished task 8.0 in stage 31.0 (TID 519). 19220 bytes result sent to driver -26/04/01 05:57:29 INFO TaskSetManager: Starting task 12.0 in stage 31.0 (TID 523) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:29 INFO TaskSetManager: Finished task 8.0 in stage 31.0 (TID 519) in 1623 ms on 10.0.0.133 (executor driver) (9/29) -26/04/01 05:57:29 INFO Executor: Running task 12.0 in stage 31.0 (TID 523) -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 4 (324.7 KiB) non-empty blocks including 4 (324.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:29 INFO Executor: Finished task 9.0 in stage 31.0 (TID 520). 19220 bytes result sent to driver -26/04/01 05:57:29 INFO TaskSetManager: Starting task 13.0 in stage 31.0 (TID 524) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:29 INFO TaskSetManager: Finished task 9.0 in stage 31.0 (TID 520) in 1621 ms on 10.0.0.133 (executor driver) (10/29) -26/04/01 05:57:29 INFO Executor: Running task 13.0 in stage 31.0 (TID 524) -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:29 INFO Executor: Finished task 11.0 in stage 31.0 (TID 522). 19220 bytes result sent to driver -26/04/01 05:57:29 INFO TaskSetManager: Starting task 14.0 in stage 31.0 (TID 525) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:29 INFO Executor: Running task 14.0 in stage 31.0 (TID 525) -26/04/01 05:57:29 INFO TaskSetManager: Finished task 11.0 in stage 31.0 (TID 522) in 1623 ms on 10.0.0.133 (executor driver) (11/29) -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 200 (63.3 MiB) non-empty blocks including 200 (63.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 4 (325.9 KiB) non-empty blocks including 4 (325.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 4 (325.9 KiB) non-empty blocks including 4 (325.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:29 INFO Executor: Finished task 10.0 in stage 31.0 (TID 521). 19220 bytes result sent to driver -26/04/01 05:57:29 INFO TaskSetManager: Starting task 15.0 in stage 31.0 (TID 526) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:29 INFO TaskSetManager: Finished task 10.0 in stage 31.0 (TID 521) in 1651 ms on 10.0.0.133 (executor driver) (12/29) -26/04/01 05:57:29 INFO Executor: Running task 15.0 in stage 31.0 (TID 526) -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 200 (63.1 MiB) non-empty blocks including 200 (63.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Getting 4 (329.2 KiB) non-empty blocks including 4 (329.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:29 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:31 INFO Executor: Finished task 12.0 in stage 31.0 (TID 523). 19177 bytes result sent to driver -26/04/01 05:57:31 INFO TaskSetManager: Starting task 16.0 in stage 31.0 (TID 527) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:31 INFO Executor: Running task 16.0 in stage 31.0 (TID 527) -26/04/01 05:57:31 INFO TaskSetManager: Finished task 12.0 in stage 31.0 (TID 523) in 1618 ms on 10.0.0.133 (executor driver) (13/29) -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 200 (63.3 MiB) non-empty blocks including 200 (63.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 4 (323.6 KiB) non-empty blocks including 4 (323.6 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:31 INFO Executor: Finished task 13.0 in stage 31.0 (TID 524). 19177 bytes result sent to driver -26/04/01 05:57:31 INFO TaskSetManager: Starting task 17.0 in stage 31.0 (TID 528) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:31 INFO Executor: Running task 17.0 in stage 31.0 (TID 528) -26/04/01 05:57:31 INFO TaskSetManager: Finished task 13.0 in stage 31.0 (TID 524) in 1635 ms on 10.0.0.133 (executor driver) (14/29) -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 4 (330.3 KiB) non-empty blocks including 4 (330.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:31 INFO Executor: Finished task 14.0 in stage 31.0 (TID 525). 19177 bytes result sent to driver -26/04/01 05:57:31 INFO TaskSetManager: Starting task 18.0 in stage 31.0 (TID 529) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:31 INFO Executor: Running task 18.0 in stage 31.0 (TID 529) -26/04/01 05:57:31 INFO TaskSetManager: Finished task 14.0 in stage 31.0 (TID 525) in 1651 ms on 10.0.0.133 (executor driver) (15/29) -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 4 (327.0 KiB) non-empty blocks including 4 (327.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:31 INFO Executor: Finished task 15.0 in stage 31.0 (TID 526). 19177 bytes result sent to driver -26/04/01 05:57:31 INFO TaskSetManager: Starting task 19.0 in stage 31.0 (TID 530) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:31 INFO TaskSetManager: Finished task 15.0 in stage 31.0 (TID 526) in 1636 ms on 10.0.0.133 (executor driver) (16/29) -26/04/01 05:57:31 INFO Executor: Running task 19.0 in stage 31.0 (TID 530) -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 200 (63.1 MiB) non-empty blocks including 200 (63.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Getting 4 (322.5 KiB) non-empty blocks including 4 (322.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:31 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:32 INFO Executor: Finished task 16.0 in stage 31.0 (TID 527). 19220 bytes result sent to driver -26/04/01 05:57:32 INFO TaskSetManager: Starting task 20.0 in stage 31.0 (TID 531) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:32 INFO Executor: Running task 20.0 in stage 31.0 (TID 531) -26/04/01 05:57:32 INFO TaskSetManager: Finished task 16.0 in stage 31.0 (TID 527) in 1670 ms on 10.0.0.133 (executor driver) (17/29) -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:32 INFO Executor: Finished task 17.0 in stage 31.0 (TID 528). 19220 bytes result sent to driver -26/04/01 05:57:32 INFO TaskSetManager: Starting task 21.0 in stage 31.0 (TID 532) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:32 INFO Executor: Running task 21.0 in stage 31.0 (TID 532) -26/04/01 05:57:32 INFO TaskSetManager: Finished task 17.0 in stage 31.0 (TID 528) in 1647 ms on 10.0.0.133 (executor driver) (18/29) -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 4 (321.4 KiB) non-empty blocks including 4 (321.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 4 (320.3 KiB) non-empty blocks including 4 (320.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:32 INFO Executor: Finished task 19.0 in stage 31.0 (TID 530). 19220 bytes result sent to driver -26/04/01 05:57:32 INFO TaskSetManager: Starting task 22.0 in stage 31.0 (TID 533) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:32 INFO Executor: Running task 22.0 in stage 31.0 (TID 533) -26/04/01 05:57:32 INFO TaskSetManager: Finished task 19.0 in stage 31.0 (TID 530) in 1645 ms on 10.0.0.133 (executor driver) (19/29) -26/04/01 05:57:32 INFO Executor: Finished task 18.0 in stage 31.0 (TID 529). 19220 bytes result sent to driver -26/04/01 05:57:32 INFO TaskSetManager: Starting task 23.0 in stage 31.0 (TID 534) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:32 INFO Executor: Running task 23.0 in stage 31.0 (TID 534) -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:32 INFO TaskSetManager: Finished task 18.0 in stage 31.0 (TID 529) in 1658 ms on 10.0.0.133 (executor driver) (20/29) -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 4 (328.1 KiB) non-empty blocks including 4 (328.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Getting 4 (325.9 KiB) non-empty blocks including 4 (325.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:32 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:34 INFO Executor: Finished task 21.0 in stage 31.0 (TID 532). 19177 bytes result sent to driver -26/04/01 05:57:34 INFO TaskSetManager: Starting task 24.0 in stage 31.0 (TID 535) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:34 INFO Executor: Running task 24.0 in stage 31.0 (TID 535) -26/04/01 05:57:34 INFO TaskSetManager: Finished task 21.0 in stage 31.0 (TID 532) in 1538 ms on 10.0.0.133 (executor driver) (21/29) -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 200 (63.1 MiB) non-empty blocks including 200 (63.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:34 INFO Executor: Finished task 20.0 in stage 31.0 (TID 531). 19177 bytes result sent to driver -26/04/01 05:57:34 INFO TaskSetManager: Starting task 25.0 in stage 31.0 (TID 536) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:34 INFO Executor: Running task 25.0 in stage 31.0 (TID 536) -26/04/01 05:57:34 INFO TaskSetManager: Finished task 20.0 in stage 31.0 (TID 531) in 1545 ms on 10.0.0.133 (executor driver) (22/29) -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 4 (321.4 KiB) non-empty blocks including 4 (321.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 4 (327.0 KiB) non-empty blocks including 4 (327.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:34 INFO Executor: Finished task 22.0 in stage 31.0 (TID 533). 19177 bytes result sent to driver -26/04/01 05:57:34 INFO TaskSetManager: Starting task 26.0 in stage 31.0 (TID 537) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:34 INFO TaskSetManager: Finished task 22.0 in stage 31.0 (TID 533) in 1531 ms on 10.0.0.133 (executor driver) (23/29) -26/04/01 05:57:34 INFO Executor: Running task 26.0 in stage 31.0 (TID 537) -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 4 (327.0 KiB) non-empty blocks including 4 (327.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:34 INFO Executor: Finished task 23.0 in stage 31.0 (TID 534). 19177 bytes result sent to driver -26/04/01 05:57:34 INFO TaskSetManager: Starting task 27.0 in stage 31.0 (TID 538) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:34 INFO TaskSetManager: Finished task 23.0 in stage 31.0 (TID 534) in 1538 ms on 10.0.0.133 (executor driver) (24/29) -26/04/01 05:57:34 INFO Executor: Running task 27.0 in stage 31.0 (TID 538) -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 200 (63.3 MiB) non-empty blocks including 200 (63.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Getting 4 (330.3 KiB) non-empty blocks including 4 (330.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:34 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:35 INFO Executor: Finished task 24.0 in stage 31.0 (TID 535). 19177 bytes result sent to driver -26/04/01 05:57:35 INFO TaskSetManager: Starting task 28.0 in stage 31.0 (TID 539) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9270 bytes) -26/04/01 05:57:35 INFO Executor: Running task 28.0 in stage 31.0 (TID 539) -26/04/01 05:57:35 INFO TaskSetManager: Finished task 24.0 in stage 31.0 (TID 535) in 1493 ms on 10.0.0.133 (executor driver) (25/29) -26/04/01 05:57:35 INFO ShuffleBlockFetcherIterator: Getting 200 (36.1 MiB) non-empty blocks including 200 (36.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:35 INFO ShuffleBlockFetcherIterator: Getting 4 (187.9 KiB) non-empty blocks including 4 (187.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:35 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:35 INFO Executor: Finished task 25.0 in stage 31.0 (TID 536). 19177 bytes result sent to driver -26/04/01 05:57:35 INFO TaskSetManager: Finished task 25.0 in stage 31.0 (TID 536) in 1514 ms on 10.0.0.133 (executor driver) (26/29) -26/04/01 05:57:35 INFO Executor: Finished task 26.0 in stage 31.0 (TID 537). 19177 bytes result sent to driver -26/04/01 05:57:35 INFO TaskSetManager: Finished task 26.0 in stage 31.0 (TID 537) in 1512 ms on 10.0.0.133 (executor driver) (27/29) -26/04/01 05:57:35 INFO Executor: Finished task 27.0 in stage 31.0 (TID 538). 19177 bytes result sent to driver -26/04/01 05:57:35 INFO TaskSetManager: Finished task 27.0 in stage 31.0 (TID 538) in 1509 ms on 10.0.0.133 (executor driver) (28/29) -26/04/01 05:57:36 INFO Executor: Finished task 28.0 in stage 31.0 (TID 539). 19177 bytes result sent to driver -26/04/01 05:57:36 INFO TaskSetManager: Finished task 28.0 in stage 31.0 (TID 539) in 638 ms on 10.0.0.133 (executor driver) (29/29) -26/04/01 05:57:36 INFO TaskSchedulerImpl: Removed TaskSet 31.0, whose tasks have all completed, from pool -26/04/01 05:57:36 INFO DAGScheduler: ShuffleMapStage 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 12.067 s -26/04/01 05:57:36 INFO DAGScheduler: looking for newly runnable stages -26/04/01 05:57:36 INFO DAGScheduler: running: Set() -26/04/01 05:57:36 INFO DAGScheduler: waiting: Set() -26/04/01 05:57:36 INFO DAGScheduler: failed: Set() -26/04/01 05:57:36 INFO ShufflePartitionsUtil: For shuffle(7), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 05:57:36 INFO HashAggregateExec: spark.sql.codegen.aggregate.map.twolevel.enabled is set to true, but current version of codegened fast hashmap does not support this aggregate. -26/04/01 05:57:36 INFO CodeGenerator: Code generated in 3.514166 ms -26/04/01 05:57:36 INFO CodeGenerator: Code generated in 1.259 ms -26/04/01 05:57:36 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:57:36 INFO DAGScheduler: Got job 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 05:57:36 INFO DAGScheduler: Final stage: ResultStage 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:57:36 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 38) -26/04/01 05:57:36 INFO DAGScheduler: Missing parents: List() -26/04/01 05:57:36 INFO DAGScheduler: Submitting ResultStage 39 (MapPartitionsRDD[73] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:57:36 WARN DAGScheduler: Broadcasting large task binary with size 1133.8 KiB -26/04/01 05:57:36 INFO MemoryStore: Block broadcast_28 stored as values in memory (estimated size 1133.8 KiB, free 8.6 GiB) -26/04/01 05:57:36 INFO MemoryStore: Block broadcast_28_piece0 stored as bytes in memory (estimated size 51.1 KiB, free 8.6 GiB) -26/04/01 05:57:36 INFO BlockManagerInfo: Added broadcast_28_piece0 in memory on 10.0.0.133:58206 (size: 51.1 KiB, free: 8.6 GiB) -26/04/01 05:57:36 INFO SparkContext: Created broadcast 28 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:57:36 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 39 (MapPartitionsRDD[73] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:57:36 INFO TaskSchedulerImpl: Adding task set 39.0 with 1 tasks resource profile 0 -26/04/01 05:57:36 INFO TaskSetManager: Starting task 0.0 in stage 39.0 (TID 540) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8999 bytes) -26/04/01 05:57:36 INFO Executor: Running task 0.0 in stage 39.0 (TID 540) -26/04/01 05:57:36 INFO ShuffleBlockFetcherIterator: Getting 29 (14.0 KiB) non-empty blocks including 29 (14.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:36 INFO CodeGenerator: Code generated in 3.10925 ms -26/04/01 05:57:36 INFO CodeGenerator: Code generated in 1.063625 ms -26/04/01 05:57:36 INFO Executor: Finished task 0.0 in stage 39.0 (TID 540). 20812 bytes result sent to driver -26/04/01 05:57:36 INFO TaskSetManager: Finished task 0.0 in stage 39.0 (TID 540) in 12 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:57:36 INFO TaskSchedulerImpl: Removed TaskSet 39.0, whose tasks have all completed, from pool -26/04/01 05:57:36 INFO DAGScheduler: ResultStage 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.015 s -26/04/01 05:57:36 INFO DAGScheduler: Job 19 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:57:36 INFO TaskSchedulerImpl: Killing all running tasks in stage 39: Stage finished -26/04/01 05:57:36 INFO DAGScheduler: Job 19 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.016231 s -26/04/01 05:57:36 INFO DAGScheduler: Registering RDD 74 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 8 -26/04/01 05:57:36 INFO DAGScheduler: Got map stage job 20 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 05:57:36 INFO DAGScheduler: Final stage: ShuffleMapStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:57:36 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 46) -26/04/01 05:57:36 INFO DAGScheduler: Missing parents: List() -26/04/01 05:57:36 INFO DAGScheduler: Submitting ShuffleMapStage 47 (MapPartitionsRDD[74] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:57:36 WARN DAGScheduler: Broadcasting large task binary with size 1134.5 KiB -26/04/01 05:57:36 INFO MemoryStore: Block broadcast_29 stored as values in memory (estimated size 1134.5 KiB, free 8.6 GiB) -26/04/01 05:57:36 INFO MemoryStore: Block broadcast_29_piece0 stored as bytes in memory (estimated size 51.7 KiB, free 8.6 GiB) -26/04/01 05:57:36 INFO BlockManagerInfo: Added broadcast_29_piece0 in memory on 10.0.0.133:58206 (size: 51.7 KiB, free: 8.6 GiB) -26/04/01 05:57:36 INFO SparkContext: Created broadcast 29 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:57:36 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 47 (MapPartitionsRDD[74] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:57:36 INFO TaskSchedulerImpl: Adding task set 47.0 with 1 tasks resource profile 0 -26/04/01 05:57:36 INFO TaskSetManager: Starting task 0.0 in stage 47.0 (TID 541) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8988 bytes) -26/04/01 05:57:36 INFO Executor: Running task 0.0 in stage 47.0 (TID 541) -26/04/01 05:57:36 INFO CodeGenerator: Code generated in 2.003417 ms -26/04/01 05:57:36 INFO ShuffleBlockFetcherIterator: Getting 29 (14.0 KiB) non-empty blocks including 29 (14.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:36 INFO Executor: Finished task 0.0 in stage 47.0 (TID 541). 20603 bytes result sent to driver -26/04/01 05:57:36 INFO TaskSetManager: Finished task 0.0 in stage 47.0 (TID 541) in 9 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:57:36 INFO TaskSchedulerImpl: Removed TaskSet 47.0, whose tasks have all completed, from pool -26/04/01 05:57:36 INFO DAGScheduler: ShuffleMapStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.013 s -26/04/01 05:57:36 INFO DAGScheduler: looking for newly runnable stages -26/04/01 05:57:36 INFO DAGScheduler: running: Set() -26/04/01 05:57:36 INFO DAGScheduler: waiting: Set() -26/04/01 05:57:36 INFO DAGScheduler: failed: Set() -26/04/01 05:57:36 INFO ShufflePartitionsUtil: For shuffle(8), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 05:57:36 INFO CodeGenerator: Code generated in 1.812209 ms -26/04/01 05:57:36 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 05:57:36 INFO DAGScheduler: Got job 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 05:57:36 INFO DAGScheduler: Final stage: ResultStage 56 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 05:57:36 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 55) -26/04/01 05:57:36 INFO DAGScheduler: Missing parents: List() -26/04/01 05:57:36 INFO DAGScheduler: Submitting ResultStage 56 (MapPartitionsRDD[77] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 05:57:36 WARN DAGScheduler: Broadcasting large task binary with size 1129.3 KiB -26/04/01 05:57:36 INFO MemoryStore: Block broadcast_30 stored as values in memory (estimated size 1129.3 KiB, free 8.6 GiB) -26/04/01 05:57:36 INFO MemoryStore: Block broadcast_30_piece0 stored as bytes in memory (estimated size 49.8 KiB, free 8.6 GiB) -26/04/01 05:57:36 INFO BlockManagerInfo: Added broadcast_30_piece0 in memory on 10.0.0.133:58206 (size: 49.8 KiB, free: 8.6 GiB) -26/04/01 05:57:36 INFO SparkContext: Created broadcast 30 from broadcast at DAGScheduler.scala:1611 -26/04/01 05:57:36 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 56 (MapPartitionsRDD[77] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 05:57:36 INFO TaskSchedulerImpl: Adding task set 56.0 with 1 tasks resource profile 0 -26/04/01 05:57:36 INFO TaskSetManager: Starting task 0.0 in stage 56.0 (TID 542) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8999 bytes) -26/04/01 05:57:36 INFO Executor: Running task 0.0 in stage 56.0 (TID 542) -26/04/01 05:57:36 INFO ShuffleBlockFetcherIterator: Getting 1 (494.0 B) non-empty blocks including 1 (494.0 B) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 05:57:36 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 05:57:36 INFO CodeGenerator: Code generated in 1.718833 ms -26/04/01 05:57:36 INFO CodeGenerator: Code generated in 1.094666 ms -26/04/01 05:57:36 INFO CodeGenerator: Code generated in 1.436 ms -26/04/01 05:57:36 INFO Executor: Finished task 0.0 in stage 56.0 (TID 542). 22060 bytes result sent to driver -26/04/01 05:57:36 INFO TaskSetManager: Finished task 0.0 in stage 56.0 (TID 542) in 8 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 05:57:36 INFO TaskSchedulerImpl: Removed TaskSet 56.0, whose tasks have all completed, from pool -26/04/01 05:57:36 INFO DAGScheduler: ResultStage 56 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.011 s -26/04/01 05:57:36 INFO DAGScheduler: Job 21 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 05:57:36 INFO TaskSchedulerImpl: Killing all running tasks in stage 56: Stage finished -26/04/01 05:57:36 INFO DAGScheduler: Job 21 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.012059 s -26/04/01 05:57:36 INFO SparkContext: SparkContext is stopping with exitCode 0. -26/04/01 05:57:36 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! -26/04/01 05:57:37 INFO MemoryStore: MemoryStore cleared -26/04/01 05:57:37 INFO BlockManager: BlockManager stopped -26/04/01 05:57:37 INFO BlockManagerMaster: BlockManagerMaster stopped -26/04/01 05:57:37 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! -26/04/01 05:57:37 INFO SparkContext: Successfully stopped SparkContext -26/04/01 05:57:37 INFO ShutdownHookManager: Shutdown hook called -26/04/01 05:57:37 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-88f9396d-516c-4d9d-9b8e-7e76a0cb2fd6 -26/04/01 05:57:37 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-dc11a204-3745-4760-9e95-7bbc7e262fc3 -26/04/01 05:57:37 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-dc11a204-3745-4760-9e95-7bbc7e262fc3/pyspark-1df01b8d-5da9-49b5-bce4-fc6a2c0407d4 - 87.48 real 294.71 user 39.61 sys - 5418713088 maximum resident set size - 0 average shared memory size - 0 average unshared data size - 0 average unshared stack size - 1246378 page reclaims - 37 page faults - 0 swaps - 0 block input operations - 0 block output operations - 1014 messages sent - 1031 messages received - 137 signals received - 25107 voluntary context switches - 312151 involuntary context switches - 4230771037201 instructions retired - 1213690734800 cycles elapsed - 5398664160 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.log b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.log deleted file mode 100644 index c3d9e20f8c..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.log +++ /dev/null @@ -1,290 +0,0 @@ -Registering table customer from /opt/tpch/sf100/customer -Registering table lineitem from /opt/tpch/sf100/lineitem -Registering table nation from /opt/tpch/sf100/nation -Registering table orders from /opt/tpch/sf100/orders -Registering table part from /opt/tpch/sf100/part -Registering table partsupp from /opt/tpch/sf100/partsupp -Registering table region from /opt/tpch/sf100/region -Registering table supplier from /opt/tpch/sf100/supplier - -============================================================ -Starting iteration 1 of 1 -============================================================ - -Running query 9 from /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/queries/tpch/q9.sql -Executing: -- CometBench-H query 9 derived from TPC-H query 9 under the terms of the TPC Fair Use Policy. --- TP... -== Physical Plan == -AdaptiveSparkPlan (46) -+- Sort (45) - +- Exchange (44) - +- HashAggregate (43) - +- Exchange (42) - +- HashAggregate (41) - +- Project (40) - +- BroadcastHashJoin Inner BuildRight (39) - :- Project (35) - : +- SortMergeJoin Inner (34) - : :- Sort (29) - : : +- Exchange (28) - : : +- Project (27) - : : +- SortMergeJoin Inner (26) - : : :- Sort (21) - : : : +- Exchange (20) - : : : +- Project (19) - : : : +- SortMergeJoin Inner (18) - : : : :- Sort (13) - : : : : +- Exchange (12) - : : : : +- Project (11) - : : : : +- SortMergeJoin Inner (10) - : : : : :- Sort (5) - : : : : : +- Exchange (4) - : : : : : +- Project (3) - : : : : : +- Filter (2) - : : : : : +- Scan parquet (1) - : : : : +- Sort (9) - : : : : +- Exchange (8) - : : : : +- Filter (7) - : : : : +- Scan parquet (6) - : : : +- Sort (17) - : : : +- Exchange (16) - : : : +- Filter (15) - : : : +- Scan parquet (14) - : : +- Sort (25) - : : +- Exchange (24) - : : +- Filter (23) - : : +- Scan parquet (22) - : +- Sort (33) - : +- Exchange (32) - : +- Filter (31) - : +- Scan parquet (30) - +- BroadcastExchange (38) - +- Filter (37) - +- Scan parquet (36) - - -(1) Scan parquet -Output [2]: [p_partkey#74L, p_name#75] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/part] -PushedFilters: [IsNotNull(p_name), StringContains(p_name,moccasin), IsNotNull(p_partkey)] -ReadSchema: struct - -(2) Filter -Input [2]: [p_partkey#74L, p_name#75] -Condition : ((isnotnull(p_name#75) AND Contains(p_name#75, moccasin)) AND isnotnull(p_partkey#74L)) - -(3) Project -Output [1]: [p_partkey#74L] -Input [2]: [p_partkey#74L, p_name#75] - -(4) Exchange -Input [1]: [p_partkey#74L] -Arguments: hashpartitioning(p_partkey#74L, 200), ENSURE_REQUIREMENTS, [plan_id=137] - -(5) Sort -Input [1]: [p_partkey#74L] -Arguments: [p_partkey#74L ASC NULLS FIRST], false, 0 - -(6) Scan parquet -Output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/lineitem] -PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_suppkey), IsNotNull(l_orderkey)] -ReadSchema: struct - -(7) Filter -Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Condition : ((isnotnull(l_partkey#17L) AND isnotnull(l_suppkey#18L)) AND isnotnull(l_orderkey#16L)) - -(8) Exchange -Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: hashpartitioning(l_partkey#17L, 200), ENSURE_REQUIREMENTS, [plan_id=138] - -(9) Sort -Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: [l_partkey#17L ASC NULLS FIRST], false, 0 - -(10) SortMergeJoin -Left keys [1]: [p_partkey#74L] -Right keys [1]: [l_partkey#17L] -Join type: Inner -Join condition: None - -(11) Project -Output [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Input [7]: [p_partkey#74L, l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] - -(12) Exchange -Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: hashpartitioning(l_suppkey#18L, 200), ENSURE_REQUIREMENTS, [plan_id=145] - -(13) Sort -Input [6]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22] -Arguments: [l_suppkey#18L ASC NULLS FIRST], false, 0 - -(14) Scan parquet -Output [2]: [s_suppkey#108L, s_nationkey#111L] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/supplier] -PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] -ReadSchema: struct - -(15) Filter -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Condition : (isnotnull(s_suppkey#108L) AND isnotnull(s_nationkey#111L)) - -(16) Exchange -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: hashpartitioning(s_suppkey#108L, 200), ENSURE_REQUIREMENTS, [plan_id=146] - -(17) Sort -Input [2]: [s_suppkey#108L, s_nationkey#111L] -Arguments: [s_suppkey#108L ASC NULLS FIRST], false, 0 - -(18) SortMergeJoin -Left keys [1]: [l_suppkey#18L] -Right keys [1]: [s_suppkey#108L] -Join type: Inner -Join condition: None - -(19) Project -Output [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] -Input [8]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_suppkey#108L, s_nationkey#111L] - -(20) Exchange -Input [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] -Arguments: hashpartitioning(l_suppkey#18L, l_partkey#17L, 200), ENSURE_REQUIREMENTS, [plan_id=154] - -(21) Sort -Input [7]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L] -Arguments: [l_suppkey#18L ASC NULLS FIRST, l_partkey#17L ASC NULLS FIRST], false, 0 - -(22) Scan parquet -Output [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/partsupp] -PushedFilters: [IsNotNull(ps_suppkey), IsNotNull(ps_partkey)] -ReadSchema: struct - -(23) Filter -Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Condition : (isnotnull(ps_suppkey#93L) AND isnotnull(ps_partkey#92L)) - -(24) Exchange -Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Arguments: hashpartitioning(ps_suppkey#93L, ps_partkey#92L, 200), ENSURE_REQUIREMENTS, [plan_id=153] - -(25) Sort -Input [3]: [ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] -Arguments: [ps_suppkey#93L ASC NULLS FIRST, ps_partkey#92L ASC NULLS FIRST], false, 0 - -(26) SortMergeJoin -Left keys [2]: [l_suppkey#18L, l_partkey#17L] -Right keys [2]: [ps_suppkey#93L, ps_partkey#92L] -Join type: Inner -Join condition: None - -(27) Project -Output [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] -Input [10]: [l_orderkey#16L, l_partkey#17L, l_suppkey#18L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_partkey#92L, ps_suppkey#93L, ps_supplycost#95] - -(28) Exchange -Input [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] -Arguments: hashpartitioning(l_orderkey#16L, 200), ENSURE_REQUIREMENTS, [plan_id=161] - -(29) Sort -Input [6]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95] -Arguments: [l_orderkey#16L ASC NULLS FIRST], false, 0 - -(30) Scan parquet -Output [2]: [o_orderkey#56L, o_orderdate#60] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/orders] -PushedFilters: [IsNotNull(o_orderkey)] -ReadSchema: struct - -(31) Filter -Input [2]: [o_orderkey#56L, o_orderdate#60] -Condition : isnotnull(o_orderkey#56L) - -(32) Exchange -Input [2]: [o_orderkey#56L, o_orderdate#60] -Arguments: hashpartitioning(o_orderkey#56L, 200), ENSURE_REQUIREMENTS, [plan_id=162] - -(33) Sort -Input [2]: [o_orderkey#56L, o_orderdate#60] -Arguments: [o_orderkey#56L ASC NULLS FIRST], false, 0 - -(34) SortMergeJoin -Left keys [1]: [l_orderkey#16L] -Right keys [1]: [o_orderkey#56L] -Join type: Inner -Join condition: None - -(35) Project -Output [6]: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60] -Input [8]: [l_orderkey#16L, l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderkey#56L, o_orderdate#60] - -(36) Scan parquet -Output [2]: [n_nationkey#48L, n_name#49] -Batched: true -Location: InMemoryFileIndex [file:/opt/tpch/sf100/nation] -PushedFilters: [IsNotNull(n_nationkey)] -ReadSchema: struct - -(37) Filter -Input [2]: [n_nationkey#48L, n_name#49] -Condition : isnotnull(n_nationkey#48L) - -(38) BroadcastExchange -Input [2]: [n_nationkey#48L, n_name#49] -Arguments: HashedRelationBroadcastMode(List(input[0, bigint, false]),false), [plan_id=168] - -(39) BroadcastHashJoin -Left keys [1]: [s_nationkey#111L] -Right keys [1]: [n_nationkey#48L] -Join type: Inner -Join condition: None - -(40) Project -Output [3]: [n_name#49 AS nation#122, year(o_orderdate#60) AS o_year#123, ((l_extendedprice#21 * (1 - l_discount#22)) - (ps_supplycost#95 * l_quantity#20)) AS amount#124] -Input [8]: [l_quantity#20, l_extendedprice#21, l_discount#22, s_nationkey#111L, ps_supplycost#95, o_orderdate#60, n_nationkey#48L, n_name#49] - -(41) HashAggregate -Input [3]: [nation#122, o_year#123, amount#124] -Keys [2]: [nation#122, o_year#123] -Functions [1]: [partial_sum(amount#124)] -Aggregate Attributes [2]: [sum#130, isEmpty#131] -Results [4]: [nation#122, o_year#123, sum#132, isEmpty#133] - -(42) Exchange -Input [4]: [nation#122, o_year#123, sum#132, isEmpty#133] -Arguments: hashpartitioning(nation#122, o_year#123, 200), ENSURE_REQUIREMENTS, [plan_id=173] - -(43) HashAggregate -Input [4]: [nation#122, o_year#123, sum#132, isEmpty#133] -Keys [2]: [nation#122, o_year#123] -Functions [1]: [sum(amount#124)] -Aggregate Attributes [1]: [sum(amount#124)#126] -Results [3]: [nation#122, o_year#123, sum(amount#124)#126 AS sum_profit#125] - -(44) Exchange -Input [3]: [nation#122, o_year#123, sum_profit#125] -Arguments: rangepartitioning(nation#122 ASC NULLS FIRST, o_year#123 DESC NULLS LAST, 200), ENSURE_REQUIREMENTS, [plan_id=176] - -(45) Sort -Input [3]: [nation#122, o_year#123, sum_profit#125] -Arguments: [nation#122 ASC NULLS FIRST, o_year#123 DESC NULLS LAST], true, 0 - -(46) AdaptiveSparkPlan -Output [3]: [nation#122, o_year#123, sum_profit#125] -Arguments: isFinalPlan=false - - -Query 9 returned 175 rows, hash=fff4ba3023e74505f304fa3243cbeeb6 -Query 9 took 79.93 seconds - -Iteration 1 took 79.93 seconds - -Writing results to /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/memory-profile-results/spark-offheap4g-q9-tpch-1775054366701.json diff --git a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.time b/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.time deleted file mode 100644 index 85ad524fcd..0000000000 --- a/benchmarks/tpc/memory-profile-results/logs/spark-offheap4g-q9.time +++ /dev/null @@ -1,5067 +0,0 @@ -26/04/01 08:38:04 WARN Utils: Your hostname, Andys-Mac-Studio.local resolves to a loopback address: 127.0.0.1; using 10.0.0.133 instead (on interface en0) -26/04/01 08:38:04 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address -26/04/01 08:38:04 INFO SparkContext: Running Spark version 3.5.8 -26/04/01 08:38:04 INFO SparkContext: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 08:38:04 INFO SparkContext: Java version 17.0.17 -26/04/01 08:38:04 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable -26/04/01 08:38:04 INFO ResourceUtils: ============================================================== -26/04/01 08:38:04 INFO ResourceUtils: No custom resources configured for spark.driver. -26/04/01 08:38:04 INFO ResourceUtils: ============================================================== -26/04/01 08:38:04 INFO SparkContext: Submitted application: spark-offheap4g-q9 benchmark derived from tpch -26/04/01 08:38:04 INFO ResourceProfile: Default ResourceProfile created, executor resources: Map(cores -> name: cores, amount: 1, script: , vendor: , memory -> name: memory, amount: 1024, script: , vendor: , offHeap -> name: offHeap, amount: 4096, script: , vendor: ), task resources: Map(cpus -> name: cpus, amount: 1.0) -26/04/01 08:38:04 INFO ResourceProfile: Limiting resource is cpu -26/04/01 08:38:04 INFO ResourceProfileManager: Added ResourceProfile id: 0 -26/04/01 08:38:04 INFO SecurityManager: Changing view acls to: andy -26/04/01 08:38:04 INFO SecurityManager: Changing modify acls to: andy -26/04/01 08:38:04 INFO SecurityManager: Changing view acls groups to: -26/04/01 08:38:04 INFO SecurityManager: Changing modify acls groups to: -26/04/01 08:38:04 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: andy; groups with view permissions: EMPTY; users with modify permissions: andy; groups with modify permissions: EMPTY -26/04/01 08:38:05 INFO Utils: Successfully started service 'sparkDriver' on port 59287. -26/04/01 08:38:05 INFO SparkEnv: Registering MapOutputTracker -26/04/01 08:38:05 INFO SparkEnv: Registering BlockManagerMaster -26/04/01 08:38:05 INFO BlockManagerMasterEndpoint: Using org.apache.spark.storage.DefaultTopologyMapper for getting topology information -26/04/01 08:38:05 INFO BlockManagerMasterEndpoint: BlockManagerMasterEndpoint up -26/04/01 08:38:05 INFO SparkEnv: Registering BlockManagerMasterHeartbeat -26/04/01 08:38:05 INFO DiskBlockManager: Created local directory at /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/blockmgr-f0da0939-da02-4fb4-99f2-68ea3c8dd138 -26/04/01 08:38:05 INFO MemoryStore: MemoryStore started with capacity 8.6 GiB -26/04/01 08:38:05 INFO SparkEnv: Registering OutputCommitCoordinator -26/04/01 08:38:05 INFO Executor: Starting executor ID driver on host 10.0.0.133 -26/04/01 08:38:05 INFO Executor: OS info Mac OS X, 26.3.1, aarch64 -26/04/01 08:38:05 INFO Executor: Java version 17.0.17 -26/04/01 08:38:05 INFO Executor: Starting executor with user classpath (userClassPathFirst = false): '' -26/04/01 08:38:05 INFO Executor: Created or updated repl class loader org.apache.spark.util.MutableURLClassLoader@73312b9e for default. -26/04/01 08:38:05 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 59288. -26/04/01 08:38:05 INFO NettyBlockTransferService: Server created on 10.0.0.133:59288 -26/04/01 08:38:05 INFO BlockManager: Using org.apache.spark.storage.RandomBlockReplicationPolicy for block replication policy -26/04/01 08:38:05 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(driver, 10.0.0.133, 59288, None) -26/04/01 08:38:05 INFO BlockManagerMasterEndpoint: Registering block manager 10.0.0.133:59288 with 8.6 GiB RAM, BlockManagerId(driver, 10.0.0.133, 59288, None) -26/04/01 08:38:05 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(driver, 10.0.0.133, 59288, None) -26/04/01 08:38:05 INFO BlockManager: Initialized BlockManager: BlockManagerId(driver, 10.0.0.133, 59288, None) -26/04/01 08:38:05 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir. -26/04/01 08:38:05 INFO SharedState: Warehouse path is 'file:/Users/andy/git/apache/datafusion-comet/benchmarks/tpc/spark-warehouse'. -26/04/01 08:38:05 INFO InMemoryFileIndex: It took 21 ms to list leaf files for 1 paths. -26/04/01 08:38:05 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:38:05 INFO DAGScheduler: Got job 0 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:38:05 INFO DAGScheduler: Final stage: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:38:05 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:38:05 INFO DAGScheduler: Missing parents: List() -26/04/01 08:38:05 INFO DAGScheduler: Submitting ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:38:05 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 08:38:05 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 08:38:05 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:05 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:38:05 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (MapPartitionsRDD[1] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:38:05 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks resource profile 0 -26/04/01 08:38:05 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) -26/04/01 08:38:05 INFO Executor: Running task 0.0 in stage 0.0 (TID 0) -26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 2041 bytes result sent to driver -26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 125 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool -26/04/01 08:38:06 INFO DAGScheduler: ResultStage 0 (load at NativeMethodAccessorImpl.java:0) finished in 0.412 s -26/04/01 08:38:06 INFO DAGScheduler: Job 0 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 0: Stage finished -26/04/01 08:38:06 INFO DAGScheduler: Job 0 finished: load at NativeMethodAccessorImpl.java:0, took 0.424579 s -26/04/01 08:38:06 INFO BlockManagerInfo: Removed broadcast_0_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:06 INFO InMemoryFileIndex: It took 3 ms to list leaf files for 1 paths. -26/04/01 08:38:06 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:38:06 INFO DAGScheduler: Got job 1 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:38:06 INFO DAGScheduler: Final stage: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:38:06 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:38:06 INFO DAGScheduler: Missing parents: List() -26/04/01 08:38:06 INFO DAGScheduler: Submitting ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:38:06 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 08:38:06 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 08:38:06 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:06 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:38:06 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 1 (MapPartitionsRDD[3] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:38:06 INFO TaskSchedulerImpl: Adding task set 1.0 with 1 tasks resource profile 0 -26/04/01 08:38:06 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) -26/04/01 08:38:06 INFO Executor: Running task 0.0 in stage 1.0 (TID 1) -26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 2344 bytes result sent to driver -26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 13 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool -26/04/01 08:38:06 INFO DAGScheduler: ResultStage 1 (load at NativeMethodAccessorImpl.java:0) finished in 0.021 s -26/04/01 08:38:06 INFO DAGScheduler: Job 1 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 1: Stage finished -26/04/01 08:38:06 INFO DAGScheduler: Job 1 finished: load at NativeMethodAccessorImpl.java:0, took 0.023024 s -26/04/01 08:38:06 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 08:38:06 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:38:06 INFO DAGScheduler: Got job 2 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:38:06 INFO DAGScheduler: Final stage: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:38:06 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:38:06 INFO DAGScheduler: Missing parents: List() -26/04/01 08:38:06 INFO DAGScheduler: Submitting ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:38:06 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 08:38:06 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 08:38:06 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:06 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:38:06 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 2 (MapPartitionsRDD[5] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:38:06 INFO TaskSchedulerImpl: Adding task set 2.0 with 1 tasks resource profile 0 -26/04/01 08:38:06 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 2) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) -26/04/01 08:38:06 INFO Executor: Running task 0.0 in stage 2.0 (TID 2) -26/04/01 08:38:06 INFO BlockManagerInfo: Removed broadcast_1_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 2.0 (TID 2). 1803 bytes result sent to driver -26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 2) in 9 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool -26/04/01 08:38:06 INFO DAGScheduler: ResultStage 2 (load at NativeMethodAccessorImpl.java:0) finished in 0.016 s -26/04/01 08:38:06 INFO DAGScheduler: Job 2 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 2: Stage finished -26/04/01 08:38:06 INFO DAGScheduler: Job 2 finished: load at NativeMethodAccessorImpl.java:0, took 0.018151 s -26/04/01 08:38:06 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. -26/04/01 08:38:06 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:38:06 INFO DAGScheduler: Got job 3 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:38:06 INFO DAGScheduler: Final stage: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:38:06 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:38:06 INFO DAGScheduler: Missing parents: List() -26/04/01 08:38:06 INFO DAGScheduler: Submitting ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:38:06 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 08:38:06 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 08:38:06 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:06 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:38:06 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (MapPartitionsRDD[7] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:38:06 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks resource profile 0 -26/04/01 08:38:06 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 3) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) -26/04/01 08:38:06 INFO Executor: Running task 0.0 in stage 3.0 (TID 3) -26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 3.0 (TID 3). 2105 bytes result sent to driver -26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 3) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool -26/04/01 08:38:06 INFO DAGScheduler: ResultStage 3 (load at NativeMethodAccessorImpl.java:0) finished in 0.015 s -26/04/01 08:38:06 INFO DAGScheduler: Job 3 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 3: Stage finished -26/04/01 08:38:06 INFO DAGScheduler: Job 3 finished: load at NativeMethodAccessorImpl.java:0, took 0.016603 s -26/04/01 08:38:06 INFO InMemoryFileIndex: It took 2 ms to list leaf files for 1 paths. -26/04/01 08:38:06 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:38:06 INFO DAGScheduler: Got job 4 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:38:06 INFO DAGScheduler: Final stage: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:38:06 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:38:06 INFO DAGScheduler: Missing parents: List() -26/04/01 08:38:06 INFO DAGScheduler: Submitting ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:38:06 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 08:38:06 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 08:38:06 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:06 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:38:06 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 4 (MapPartitionsRDD[9] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:38:06 INFO TaskSchedulerImpl: Adding task set 4.0 with 1 tasks resource profile 0 -26/04/01 08:38:06 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 4) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9134 bytes) -26/04/01 08:38:06 INFO Executor: Running task 0.0 in stage 4.0 (TID 4) -26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 4.0 (TID 4). 2027 bytes result sent to driver -26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 4) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool -26/04/01 08:38:06 INFO DAGScheduler: ResultStage 4 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 08:38:06 INFO DAGScheduler: Job 4 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 4: Stage finished -26/04/01 08:38:06 INFO DAGScheduler: Job 4 finished: load at NativeMethodAccessorImpl.java:0, took 0.014998 s -26/04/01 08:38:06 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 08:38:06 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:38:06 INFO DAGScheduler: Got job 5 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:38:06 INFO DAGScheduler: Final stage: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:38:06 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:38:06 INFO DAGScheduler: Missing parents: List() -26/04/01 08:38:06 INFO DAGScheduler: Submitting ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:38:06 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 08:38:06 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 08:38:06 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:06 INFO BlockManagerInfo: Removed broadcast_3_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:06 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:38:06 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 5 (MapPartitionsRDD[11] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:38:06 INFO TaskSchedulerImpl: Adding task set 5.0 with 1 tasks resource profile 0 -26/04/01 08:38:06 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 5) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) -26/04/01 08:38:06 INFO BlockManagerInfo: Removed broadcast_4_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:06 INFO Executor: Running task 0.0 in stage 5.0 (TID 5) -26/04/01 08:38:06 INFO BlockManagerInfo: Removed broadcast_2_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 5.0 (TID 5). 1928 bytes result sent to driver -26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 5) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool -26/04/01 08:38:06 INFO DAGScheduler: ResultStage 5 (load at NativeMethodAccessorImpl.java:0) finished in 0.014 s -26/04/01 08:38:06 INFO DAGScheduler: Job 5 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 5: Stage finished -26/04/01 08:38:06 INFO DAGScheduler: Job 5 finished: load at NativeMethodAccessorImpl.java:0, took 0.015529 s -26/04/01 08:38:06 INFO InMemoryFileIndex: It took 0 ms to list leaf files for 1 paths. -26/04/01 08:38:06 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:38:06 INFO DAGScheduler: Got job 6 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:38:06 INFO DAGScheduler: Final stage: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:38:06 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:38:06 INFO DAGScheduler: Missing parents: List() -26/04/01 08:38:06 INFO DAGScheduler: Submitting ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:38:06 INFO MemoryStore: Block broadcast_6 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 08:38:06 INFO MemoryStore: Block broadcast_6_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 08:38:06 INFO BlockManagerInfo: Added broadcast_6_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:06 INFO BlockManagerInfo: Removed broadcast_5_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:06 INFO SparkContext: Created broadcast 6 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:38:06 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 6 (MapPartitionsRDD[13] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:38:06 INFO TaskSchedulerImpl: Adding task set 6.0 with 1 tasks resource profile 0 -26/04/01 08:38:06 INFO TaskSetManager: Starting task 0.0 in stage 6.0 (TID 6) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9138 bytes) -26/04/01 08:38:06 INFO Executor: Running task 0.0 in stage 6.0 (TID 6) -26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 6.0 (TID 6). 1729 bytes result sent to driver -26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 6.0 (TID 6) in 5 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 6.0, whose tasks have all completed, from pool -26/04/01 08:38:06 INFO DAGScheduler: ResultStage 6 (load at NativeMethodAccessorImpl.java:0) finished in 0.013 s -26/04/01 08:38:06 INFO DAGScheduler: Job 6 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 6: Stage finished -26/04/01 08:38:06 INFO DAGScheduler: Job 6 finished: load at NativeMethodAccessorImpl.java:0, took 0.014484 s -26/04/01 08:38:06 INFO InMemoryFileIndex: It took 1 ms to list leaf files for 1 paths. -26/04/01 08:38:06 INFO SparkContext: Starting job: load at NativeMethodAccessorImpl.java:0 -26/04/01 08:38:06 INFO DAGScheduler: Got job 7 (load at NativeMethodAccessorImpl.java:0) with 1 output partitions -26/04/01 08:38:06 INFO DAGScheduler: Final stage: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) -26/04/01 08:38:06 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:38:06 INFO DAGScheduler: Missing parents: List() -26/04/01 08:38:06 INFO DAGScheduler: Submitting ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0), which has no missing parents -26/04/01 08:38:06 INFO MemoryStore: Block broadcast_7 stored as values in memory (estimated size 103.7 KiB, free 8.6 GiB) -26/04/01 08:38:06 INFO MemoryStore: Block broadcast_7_piece0 stored as bytes in memory (estimated size 37.4 KiB, free 8.6 GiB) -26/04/01 08:38:06 INFO BlockManagerInfo: Added broadcast_7_piece0 in memory on 10.0.0.133:59288 (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:06 INFO SparkContext: Created broadcast 7 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:38:06 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 7 (MapPartitionsRDD[15] at load at NativeMethodAccessorImpl.java:0) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:38:06 INFO TaskSchedulerImpl: Adding task set 7.0 with 1 tasks resource profile 0 -26/04/01 08:38:06 INFO TaskSetManager: Starting task 0.0 in stage 7.0 (TID 7) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9142 bytes) -26/04/01 08:38:06 INFO Executor: Running task 0.0 in stage 7.0 (TID 7) -26/04/01 08:38:06 INFO Executor: Finished task 0.0 in stage 7.0 (TID 7). 1923 bytes result sent to driver -26/04/01 08:38:06 INFO TaskSetManager: Finished task 0.0 in stage 7.0 (TID 7) in 6 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:38:06 INFO TaskSchedulerImpl: Removed TaskSet 7.0, whose tasks have all completed, from pool -26/04/01 08:38:06 INFO DAGScheduler: ResultStage 7 (load at NativeMethodAccessorImpl.java:0) finished in 0.012 s -26/04/01 08:38:06 INFO DAGScheduler: Job 7 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:38:06 INFO TaskSchedulerImpl: Killing all running tasks in stage 7: Stage finished -26/04/01 08:38:06 INFO DAGScheduler: Job 7 finished: load at NativeMethodAccessorImpl.java:0, took 0.013930 s -26/04/01 08:38:06 INFO FileSourceStrategy: Pushed Filters: IsNotNull(p_name),StringContains(p_name,moccasin),IsNotNull(p_partkey) -26/04/01 08:38:06 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(p_name#75),Contains(p_name#75, moccasin),isnotnull(p_partkey#74L) -26/04/01 08:38:06 INFO FileSourceStrategy: Pushed Filters: IsNotNull(l_partkey),IsNotNull(l_suppkey),IsNotNull(l_orderkey) -26/04/01 08:38:06 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(l_partkey#17L),isnotnull(l_suppkey#18L),isnotnull(l_orderkey#16L) -26/04/01 08:38:06 INFO FileSourceStrategy: Pushed Filters: IsNotNull(s_suppkey),IsNotNull(s_nationkey) -26/04/01 08:38:06 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(s_suppkey#108L),isnotnull(s_nationkey#111L) -26/04/01 08:38:06 INFO FileSourceStrategy: Pushed Filters: IsNotNull(ps_suppkey),IsNotNull(ps_partkey) -26/04/01 08:38:06 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(ps_suppkey#93L),isnotnull(ps_partkey#92L) -26/04/01 08:38:06 INFO FileSourceStrategy: Pushed Filters: IsNotNull(o_orderkey) -26/04/01 08:38:06 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(o_orderkey#56L) -26/04/01 08:38:06 INFO FileSourceStrategy: Pushed Filters: IsNotNull(n_nationkey) -26/04/01 08:38:06 INFO FileSourceStrategy: Post-Scan Filters: isnotnull(n_nationkey#48L) -26/04/01 08:38:07 INFO CodeGenerator: Code generated in 70.492667 ms -26/04/01 08:38:07 INFO CodeGenerator: Code generated in 71.013458 ms -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_9 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_8 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_9_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_8_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_9_piece0 in memory on 10.0.0.133:59288 (size: 35.0 KiB, free: 8.6 GiB) -26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_8_piece0 in memory on 10.0.0.133:59288 (size: 35.0 KiB, free: 8.6 GiB) -26/04/01 08:38:07 INFO SparkContext: Created broadcast 9 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:38:07 INFO SparkContext: Created broadcast 8 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:38:07 INFO FileSourceScanExec: Planning scan with bin packing, max size: 102898790 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:38:07 INFO FileSourceScanExec: Planning scan with bin packing, max size: 4194304 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:38:07 INFO BlockManagerInfo: Removed broadcast_6_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:07 INFO DAGScheduler: Registering RDD 23 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 0 -26/04/01 08:38:07 INFO BlockManagerInfo: Removed broadcast_7_piece0 on 10.0.0.133:59288 in memory (size: 37.4 KiB, free: 8.6 GiB) -26/04/01 08:38:07 INFO DAGScheduler: Got map stage job 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 8 output partitions -26/04/01 08:38:07 INFO DAGScheduler: Final stage: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:38:07 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:38:07 INFO DAGScheduler: Missing parents: List() -26/04/01 08:38:07 INFO DAGScheduler: Submitting ShuffleMapStage 8 (MapPartitionsRDD[23] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:38:07 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_10 stored as values in memory (estimated size 18.2 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_10_piece0 stored as bytes in memory (estimated size 8.1 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_10_piece0 in memory on 10.0.0.133:59288 (size: 8.1 KiB, free: 8.6 GiB) -26/04/01 08:38:07 INFO SparkContext: Created broadcast 10 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:38:07 INFO DAGScheduler: Submitting 8 missing tasks from ShuffleMapStage 8 (MapPartitionsRDD[23] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7)) -26/04/01 08:38:07 INFO TaskSchedulerImpl: Adding task set 8.0 with 8 tasks resource profile 0 -26/04/01 08:38:07 INFO DAGScheduler: Got job 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 08:38:07 INFO DAGScheduler: Final stage: ResultStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:38:07 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:38:07 INFO DAGScheduler: Missing parents: List() -26/04/01 08:38:07 INFO DAGScheduler: Submitting ResultStage 9 (MapPartitionsRDD[22] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:38:07 INFO CodeGenerator: Code generated in 12.31725 ms -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_11 stored as values in memory (estimated size 15.2 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO TaskSetManager: Starting task 0.0 in stage 8.0 (TID 8) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9891 bytes) -26/04/01 08:38:07 INFO TaskSetManager: Starting task 1.0 in stage 8.0 (TID 9) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9893 bytes) -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_11_piece0 stored as bytes in memory (estimated size 6.5 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO TaskSetManager: Starting task 2.0 in stage 8.0 (TID 10) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9893 bytes) -26/04/01 08:38:07 INFO TaskSetManager: Starting task 3.0 in stage 8.0 (TID 11) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9893 bytes) -26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_11_piece0 in memory on 10.0.0.133:59288 (size: 6.5 KiB, free: 8.6 GiB) -26/04/01 08:38:07 INFO TaskSetManager: Starting task 4.0 in stage 8.0 (TID 12) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9892 bytes) -26/04/01 08:38:07 INFO TaskSetManager: Starting task 5.0 in stage 8.0 (TID 13) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9894 bytes) -26/04/01 08:38:07 INFO TaskSetManager: Starting task 6.0 in stage 8.0 (TID 14) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9893 bytes) -26/04/01 08:38:07 INFO SparkContext: Created broadcast 11 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:38:07 INFO TaskSetManager: Starting task 7.0 in stage 8.0 (TID 15) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9894 bytes) -26/04/01 08:38:07 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 9 (MapPartitionsRDD[22] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:38:07 INFO TaskSchedulerImpl: Adding task set 9.0 with 1 tasks resource profile 0 -26/04/01 08:38:07 INFO Executor: Running task 0.0 in stage 8.0 (TID 8) -26/04/01 08:38:07 INFO Executor: Running task 1.0 in stage 8.0 (TID 9) -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_12 stored as values in memory (estimated size 202.3 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO Executor: Running task 2.0 in stage 8.0 (TID 10) -26/04/01 08:38:07 INFO Executor: Running task 3.0 in stage 8.0 (TID 11) -26/04/01 08:38:07 INFO Executor: Running task 4.0 in stage 8.0 (TID 12) -26/04/01 08:38:07 INFO Executor: Running task 5.0 in stage 8.0 (TID 13) -26/04/01 08:38:07 INFO Executor: Running task 6.0 in stage 8.0 (TID 14) -26/04/01 08:38:07 INFO Executor: Running task 7.0 in stage 8.0 (TID 15) -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_12_piece0 stored as bytes in memory (estimated size 35.2 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_12_piece0 in memory on 10.0.0.133:59288 (size: 35.2 KiB, free: 8.6 GiB) -26/04/01 08:38:07 INFO SparkContext: Created broadcast 12 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:38:07 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:38:07 INFO DAGScheduler: Registering RDD 27 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 1 -26/04/01 08:38:07 INFO DAGScheduler: Got map stage job 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 208 output partitions -26/04/01 08:38:07 INFO DAGScheduler: Final stage: ShuffleMapStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:38:07 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:38:07 INFO DAGScheduler: Missing parents: List() -26/04/01 08:38:07 INFO DAGScheduler: Submitting ShuffleMapStage 10 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_13 stored as values in memory (estimated size 20.4 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_13_piece0 stored as bytes in memory (estimated size 8.5 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_13_piece0 in memory on 10.0.0.133:59288 (size: 8.5 KiB, free: 8.6 GiB) -26/04/01 08:38:07 INFO SparkContext: Created broadcast 13 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:38:07 INFO DAGScheduler: Submitting 208 missing tasks from ShuffleMapStage 10 (MapPartitionsRDD[27] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:38:07 INFO TaskSchedulerImpl: Adding task set 10.0 with 208 tasks resource profile 0 -26/04/01 08:38:07 INFO CodeGenerator: Code generated in 6.977 ms -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_14 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_14_piece0 stored as bytes in memory (estimated size 35.0 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_14_piece0 in memory on 10.0.0.133:59288 (size: 35.0 KiB, free: 8.6 GiB) -26/04/01 08:38:07 INFO SparkContext: Created broadcast 14 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:38:07 INFO FileSourceScanExec: Planning scan with bin packing, max size: 27960063 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:38:07 INFO CodeGenerator: Code generated in 5.3325 ms -26/04/01 08:38:07 INFO DAGScheduler: Registering RDD 31 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 2 -26/04/01 08:38:07 INFO DAGScheduler: Got map stage job 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 8 output partitions -26/04/01 08:38:07 INFO DAGScheduler: Final stage: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:38:07 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:38:07 INFO DAGScheduler: Missing parents: List() -26/04/01 08:38:07 INFO DAGScheduler: Submitting ShuffleMapStage 11 (MapPartitionsRDD[31] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_15 stored as values in memory (estimated size 17.7 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_15_piece0 stored as bytes in memory (estimated size 7.9 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_15_piece0 in memory on 10.0.0.133:59288 (size: 7.9 KiB, free: 8.6 GiB) -26/04/01 08:38:07 INFO SparkContext: Created broadcast 15 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:38:07 INFO DAGScheduler: Submitting 8 missing tasks from ShuffleMapStage 11 (MapPartitionsRDD[31] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7)) -26/04/01 08:38:07 INFO TaskSchedulerImpl: Adding task set 11.0 with 8 tasks resource profile 0 -26/04/01 08:38:07 INFO CodeGenerator: Code generated in 5.675916 ms -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_16 stored as values in memory (estimated size 201.9 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO CodeGenerator: Code generated in 5.802083 ms -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_16_piece0 stored as bytes in memory (estimated size 35.1 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_16_piece0 in memory on 10.0.0.133:59288 (size: 35.1 KiB, free: 8.6 GiB) -26/04/01 08:38:07 INFO SparkContext: Created broadcast 16 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:38:07 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.17.parquet, range: 0-21536094, partition values: [empty row] -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.19.parquet, range: 0-21528527, partition values: [empty row] -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.1.parquet, range: 0-21530360, partition values: [empty row] -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.8.parquet, range: 0-21539554, partition values: [empty row] -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.14.parquet, range: 0-21526139, partition values: [empty row] -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.3.parquet, range: 0-21532480, partition values: [empty row] -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.30.parquet, range: 0-21523460, partition values: [empty row] -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.6.parquet, range: 0-21534095, partition values: [empty row] -26/04/01 08:38:07 INFO DAGScheduler: Registering RDD 35 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 3 -26/04/01 08:38:07 INFO DAGScheduler: Got map stage job 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 37 output partitions -26/04/01 08:38:07 INFO DAGScheduler: Final stage: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:38:07 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:38:07 INFO DAGScheduler: Missing parents: List() -26/04/01 08:38:07 INFO DAGScheduler: Submitting ShuffleMapStage 12 (MapPartitionsRDD[35] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_17 stored as values in memory (estimated size 18.6 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_17_piece0 stored as bytes in memory (estimated size 8.2 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_17_piece0 in memory on 10.0.0.133:59288 (size: 8.2 KiB, free: 8.6 GiB) -26/04/01 08:38:07 INFO SparkContext: Created broadcast 17 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:38:07 INFO DAGScheduler: Submitting 37 missing tasks from ShuffleMapStage 12 (MapPartitionsRDD[35] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:38:07 INFO TaskSchedulerImpl: Adding task set 12.0 with 37 tasks resource profile 0 -26/04/01 08:38:07 INFO CodeGenerator: Code generated in 4.214833 ms -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_18 stored as values in memory (estimated size 201.8 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_18_piece0 stored as bytes in memory (estimated size 35.1 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_18_piece0 in memory on 10.0.0.133:59288 (size: 35.1 KiB, free: 8.6 GiB) -26/04/01 08:38:07 INFO SparkContext: Created broadcast 18 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:38:07 INFO FileSourceScanExec: Planning scan with bin packing, max size: 134217728 bytes, open cost is considered as scanning 4194304 bytes. -26/04/01 08:38:07 INFO DAGScheduler: Registering RDD 39 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 4 -26/04/01 08:38:07 INFO DAGScheduler: Got map stage job 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 64 output partitions -26/04/01 08:38:07 INFO DAGScheduler: Final stage: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:38:07 INFO DAGScheduler: Parents of final stage: List() -26/04/01 08:38:07 INFO DAGScheduler: Missing parents: List() -26/04/01 08:38:07 INFO DAGScheduler: Submitting ShuffleMapStage 13 (MapPartitionsRDD[39] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_19 stored as values in memory (estimated size 17.8 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO MemoryStore: Block broadcast_19_piece0 stored as bytes in memory (estimated size 8.0 KiB, free 8.6 GiB) -26/04/01 08:38:07 INFO BlockManagerInfo: Added broadcast_19_piece0 in memory on 10.0.0.133:59288 (size: 8.0 KiB, free: 8.6 GiB) -26/04/01 08:38:07 INFO SparkContext: Created broadcast 19 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:38:07 INFO DAGScheduler: Submitting 64 missing tasks from ShuffleMapStage 13 (MapPartitionsRDD[39] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:38:07 INFO TaskSchedulerImpl: Adding task set 13.0 with 64 tasks resource profile 0 -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@d2db547)), noteq(p_partkey, null)) -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@2bc93fdf)), noteq(p_partkey, null)) -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@741d8897)), noteq(p_partkey, null)) -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@703f69d4)), noteq(p_partkey, null)) -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@74395742)), noteq(p_partkey, null)) -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@341dd6e7)), noteq(p_partkey, null)) -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@230b76c2)), noteq(p_partkey, null)) -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@e76cdec)), noteq(p_partkey, null)) -26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 08:38:07 INFO CodecPool: Got brand-new decompressor [.snappy] -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.13.parquet, range: 0-21529438, partition values: [empty row] -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.22.parquet, range: 0-21531915, partition values: [empty row] -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.5.parquet, range: 0-21524978, partition values: [empty row] -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@3cea020e)), noteq(p_partkey, null)) -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@556f4016)), noteq(p_partkey, null)) -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.16.parquet, range: 0-21533858, partition values: [empty row] -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.28.parquet, range: 0-21538351, partition values: [empty row] -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.10.parquet, range: 0-21523089, partition values: [empty row] -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.18.parquet, range: 0-21535985, partition values: [empty row] -26/04/01 08:38:07 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.20.parquet, range: 0-21528010, partition values: [empty row] -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@5a63b0e4)), noteq(p_partkey, null)) -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@5af179ca)), noteq(p_partkey, null)) -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@4dd47702)), noteq(p_partkey, null)) -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@7b24ed9e)), noteq(p_partkey, null)) -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@51a83230)), noteq(p_partkey, null)) -26/04/01 08:38:07 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@f77f37d)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.11.parquet, range: 0-21529268, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.12.parquet, range: 0-21523797, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.9.parquet, range: 0-21537495, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.25.parquet, range: 0-21531839, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.27.parquet, range: 0-21527815, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.4.parquet, range: 0-21535415, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.31.parquet, range: 0-21533701, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.24.parquet, range: 0-21520965, partition values: [empty row] -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@43866115)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@529ae805)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@56949e42)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@7985d4fa)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@3a186a9e)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@6a92709e)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@7d20947c)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@41766d96)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.23.parquet, range: 0-21531357, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.7.parquet, range: 0-21528633, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.29.parquet, range: 0-21523744, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.32.parquet, range: 0-21526986, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.15.parquet, range: 0-21520086, partition values: [empty row] -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@335cdde4)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.2.parquet, range: 0-21537447, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.26.parquet, range: 0-21533511, partition values: [empty row] -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@5945f1c7)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/part/part.21.parquet, range: 0-21534200, partition values: [empty row] -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@b6cdab2)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@a478e01)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@54ba13c4)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@50a9de9d)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@3772abc4)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(p_name, null), userdefinedbyinstance(p_name, org.apache.spark.sql.execution.datasources.parquet.ParquetFilters$$anon$3@46d046b1)), noteq(p_partkey, null)) -26/04/01 08:38:08 INFO Executor: Finished task 6.0 in stage 8.0 (TID 14). 2294 bytes result sent to driver -26/04/01 08:38:08 INFO Executor: Finished task 3.0 in stage 8.0 (TID 11). 2294 bytes result sent to driver -26/04/01 08:38:08 INFO TaskSetManager: Starting task 0.0 in stage 9.0 (TID 16) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9608 bytes) -26/04/01 08:38:08 INFO Executor: Finished task 5.0 in stage 8.0 (TID 13). 2294 bytes result sent to driver -26/04/01 08:38:08 INFO Executor: Running task 0.0 in stage 9.0 (TID 16) -26/04/01 08:38:08 INFO TaskSetManager: Starting task 0.0 in stage 10.0 (TID 17) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:08 INFO TaskSetManager: Finished task 6.0 in stage 8.0 (TID 14) in 1095 ms on 10.0.0.133 (executor driver) (1/8) -26/04/01 08:38:08 INFO Executor: Running task 0.0 in stage 10.0 (TID 17) -26/04/01 08:38:08 INFO TaskSetManager: Starting task 1.0 in stage 10.0 (TID 18) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:08 INFO TaskSetManager: Finished task 3.0 in stage 8.0 (TID 11) in 1096 ms on 10.0.0.133 (executor driver) (2/8) -26/04/01 08:38:08 INFO TaskSetManager: Finished task 5.0 in stage 8.0 (TID 13) in 1096 ms on 10.0.0.133 (executor driver) (3/8) -26/04/01 08:38:08 INFO Executor: Running task 1.0 in stage 10.0 (TID 18) -26/04/01 08:38:08 INFO Executor: Finished task 7.0 in stage 8.0 (TID 15). 2294 bytes result sent to driver -26/04/01 08:38:08 INFO TaskSetManager: Starting task 2.0 in stage 10.0 (TID 19) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:08 INFO Executor: Running task 2.0 in stage 10.0 (TID 19) -26/04/01 08:38:08 INFO TaskSetManager: Finished task 7.0 in stage 8.0 (TID 15) in 1097 ms on 10.0.0.133 (executor driver) (4/8) -26/04/01 08:38:08 INFO Executor: Finished task 0.0 in stage 8.0 (TID 8). 2294 bytes result sent to driver -26/04/01 08:38:08 INFO TaskSetManager: Starting task 3.0 in stage 10.0 (TID 20) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:08 INFO Executor: Running task 3.0 in stage 10.0 (TID 20) -26/04/01 08:38:08 INFO Executor: Finished task 1.0 in stage 8.0 (TID 9). 2294 bytes result sent to driver -26/04/01 08:38:08 INFO TaskSetManager: Finished task 0.0 in stage 8.0 (TID 8) in 1101 ms on 10.0.0.133 (executor driver) (5/8) -26/04/01 08:38:08 INFO TaskSetManager: Starting task 4.0 in stage 10.0 (TID 21) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:08 INFO TaskSetManager: Finished task 1.0 in stage 8.0 (TID 9) in 1100 ms on 10.0.0.133 (executor driver) (6/8) -26/04/01 08:38:08 INFO Executor: Running task 4.0 in stage 10.0 (TID 21) -26/04/01 08:38:08 INFO Executor: Finished task 4.0 in stage 8.0 (TID 12). 2294 bytes result sent to driver -26/04/01 08:38:08 INFO TaskSetManager: Starting task 5.0 in stage 10.0 (TID 22) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:08 INFO Executor: Running task 5.0 in stage 10.0 (TID 22) -26/04/01 08:38:08 INFO TaskSetManager: Finished task 4.0 in stage 8.0 (TID 12) in 1100 ms on 10.0.0.133 (executor driver) (7/8) -26/04/01 08:38:08 INFO Executor: Finished task 2.0 in stage 8.0 (TID 10). 2294 bytes result sent to driver -26/04/01 08:38:08 INFO TaskSetManager: Starting task 6.0 in stage 10.0 (TID 23) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:08 INFO Executor: Running task 6.0 in stage 10.0 (TID 23) -26/04/01 08:38:08 INFO TaskSetManager: Finished task 2.0 in stage 8.0 (TID 10) in 1102 ms on 10.0.0.133 (executor driver) (8/8) -26/04/01 08:38:08 INFO TaskSchedulerImpl: Removed TaskSet 8.0, whose tasks have all completed, from pool -26/04/01 08:38:08 INFO DAGScheduler: ShuffleMapStage 8 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 1.111 s -26/04/01 08:38:08 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:38:08 INFO DAGScheduler: running: Set(ShuffleMapStage 12, ResultStage 9, ShuffleMapStage 13, ShuffleMapStage 10, ShuffleMapStage 11) -26/04/01 08:38:08 INFO DAGScheduler: waiting: Set() -26/04/01 08:38:08 INFO DAGScheduler: failed: Set() -26/04/01 08:38:08 INFO CodeGenerator: Code generated in 7.670084 ms -26/04/01 08:38:08 INFO CodeGenerator: Code generated in 10.964417 ms -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/nation/nation.1.parquet, range: 0-2670, partition values: [empty row] -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: noteq(n_nationkey, null) -26/04/01 08:38:08 INFO Executor: Finished task 0.0 in stage 9.0 (TID 16). 2376 bytes result sent to driver -26/04/01 08:38:08 INFO TaskSetManager: Starting task 7.0 in stage 10.0 (TID 24) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:08 INFO Executor: Running task 7.0 in stage 10.0 (TID 24) -26/04/01 08:38:08 INFO TaskSetManager: Finished task 0.0 in stage 9.0 (TID 16) in 21 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:38:08 INFO TaskSchedulerImpl: Removed TaskSet 9.0, whose tasks have all completed, from pool -26/04/01 08:38:08 INFO DAGScheduler: ResultStage 9 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 1.116 s -26/04/01 08:38:08 INFO DAGScheduler: Job 9 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:38:08 INFO TaskSchedulerImpl: Killing all running tasks in stage 9: Stage finished -26/04/01 08:38:08 INFO DAGScheduler: Job 9 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 1.122861 s -26/04/01 08:38:08 INFO CodeGenerator: Code generated in 5.074958 ms -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:08 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:08 INFO CodeGenerator: Code generated in 3.289791 ms -26/04/01 08:38:08 INFO MemoryStore: Block broadcast_20 stored as values in memory (estimated size 1024.2 KiB, free 8.6 GiB) -26/04/01 08:38:08 INFO MemoryStore: Block broadcast_20_piece0 stored as bytes in memory (estimated size 828.0 B, free 8.6 GiB) -26/04/01 08:38:08 INFO BlockManagerInfo: Added broadcast_20_piece0 in memory on 10.0.0.133:59288 (size: 828.0 B, free: 8.6 GiB) -26/04/01 08:38:08 INFO SparkContext: Created broadcast 20 from collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:08 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:09 INFO Executor: Finished task 5.0 in stage 10.0 (TID 22). 2294 bytes result sent to driver -26/04/01 08:38:09 INFO TaskSetManager: Starting task 8.0 in stage 10.0 (TID 25) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:09 INFO Executor: Running task 8.0 in stage 10.0 (TID 25) -26/04/01 08:38:09 INFO TaskSetManager: Finished task 5.0 in stage 10.0 (TID 22) in 1260 ms on 10.0.0.133 (executor driver) (1/208) -26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:09 INFO Executor: Finished task 4.0 in stage 10.0 (TID 21). 2294 bytes result sent to driver -26/04/01 08:38:09 INFO TaskSetManager: Starting task 9.0 in stage 10.0 (TID 26) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:09 INFO Executor: Running task 9.0 in stage 10.0 (TID 26) -26/04/01 08:38:09 INFO TaskSetManager: Finished task 4.0 in stage 10.0 (TID 21) in 1452 ms on 10.0.0.133 (executor driver) (2/208) -26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:09 INFO Executor: Finished task 7.0 in stage 10.0 (TID 24). 2294 bytes result sent to driver -26/04/01 08:38:09 INFO TaskSetManager: Starting task 10.0 in stage 10.0 (TID 27) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:09 INFO TaskSetManager: Finished task 7.0 in stage 10.0 (TID 24) in 1443 ms on 10.0.0.133 (executor driver) (3/208) -26/04/01 08:38:09 INFO Executor: Running task 10.0 in stage 10.0 (TID 27) -26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:09 INFO Executor: Finished task 6.0 in stage 10.0 (TID 23). 2294 bytes result sent to driver -26/04/01 08:38:09 INFO TaskSetManager: Starting task 11.0 in stage 10.0 (TID 28) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:09 INFO TaskSetManager: Finished task 6.0 in stage 10.0 (TID 23) in 1461 ms on 10.0.0.133 (executor driver) (4/208) -26/04/01 08:38:09 INFO Executor: Running task 11.0 in stage 10.0 (TID 28) -26/04/01 08:38:09 INFO Executor: Finished task 3.0 in stage 10.0 (TID 20). 2294 bytes result sent to driver -26/04/01 08:38:09 INFO TaskSetManager: Starting task 12.0 in stage 10.0 (TID 29) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:09 INFO Executor: Running task 12.0 in stage 10.0 (TID 29) -26/04/01 08:38:09 INFO TaskSetManager: Finished task 3.0 in stage 10.0 (TID 20) in 1465 ms on 10.0.0.133 (executor driver) (5/208) -26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:09 INFO Executor: Finished task 1.0 in stage 10.0 (TID 18). 2294 bytes result sent to driver -26/04/01 08:38:09 INFO TaskSetManager: Starting task 13.0 in stage 10.0 (TID 30) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:09 INFO TaskSetManager: Finished task 1.0 in stage 10.0 (TID 18) in 1469 ms on 10.0.0.133 (executor driver) (6/208) -26/04/01 08:38:09 INFO Executor: Finished task 0.0 in stage 10.0 (TID 17). 2294 bytes result sent to driver -26/04/01 08:38:09 INFO Executor: Running task 13.0 in stage 10.0 (TID 30) -26/04/01 08:38:09 INFO TaskSetManager: Starting task 14.0 in stage 10.0 (TID 31) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:09 INFO Executor: Running task 14.0 in stage 10.0 (TID 31) -26/04/01 08:38:09 INFO TaskSetManager: Finished task 0.0 in stage 10.0 (TID 17) in 1470 ms on 10.0.0.133 (executor driver) (7/208) -26/04/01 08:38:09 INFO Executor: Finished task 2.0 in stage 10.0 (TID 19). 2294 bytes result sent to driver -26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:09 INFO TaskSetManager: Starting task 15.0 in stage 10.0 (TID 32) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:09 INFO Executor: Running task 15.0 in stage 10.0 (TID 32) -26/04/01 08:38:09 INFO TaskSetManager: Finished task 2.0 in stage 10.0 (TID 19) in 1468 ms on 10.0.0.133 (executor driver) (8/208) -26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:09 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:09 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:10 INFO Executor: Finished task 8.0 in stage 10.0 (TID 25). 2294 bytes result sent to driver -26/04/01 08:38:10 INFO TaskSetManager: Starting task 16.0 in stage 10.0 (TID 33) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:10 INFO Executor: Running task 16.0 in stage 10.0 (TID 33) -26/04/01 08:38:10 INFO TaskSetManager: Finished task 8.0 in stage 10.0 (TID 25) in 1083 ms on 10.0.0.133 (executor driver) (9/208) -26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:10 INFO Executor: Finished task 9.0 in stage 10.0 (TID 26). 2294 bytes result sent to driver -26/04/01 08:38:10 INFO TaskSetManager: Starting task 17.0 in stage 10.0 (TID 34) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:10 INFO TaskSetManager: Finished task 9.0 in stage 10.0 (TID 26) in 1044 ms on 10.0.0.133 (executor driver) (10/208) -26/04/01 08:38:10 INFO Executor: Running task 17.0 in stage 10.0 (TID 34) -26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:10 INFO Executor: Finished task 10.0 in stage 10.0 (TID 27). 2294 bytes result sent to driver -26/04/01 08:38:10 INFO TaskSetManager: Starting task 18.0 in stage 10.0 (TID 35) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:10 INFO TaskSetManager: Finished task 10.0 in stage 10.0 (TID 27) in 1100 ms on 10.0.0.133 (executor driver) (11/208) -26/04/01 08:38:10 INFO Executor: Running task 18.0 in stage 10.0 (TID 35) -26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:10 INFO Executor: Finished task 14.0 in stage 10.0 (TID 31). 2251 bytes result sent to driver -26/04/01 08:38:10 INFO TaskSetManager: Starting task 19.0 in stage 10.0 (TID 36) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:10 INFO TaskSetManager: Finished task 14.0 in stage 10.0 (TID 31) in 1186 ms on 10.0.0.133 (executor driver) (12/208) -26/04/01 08:38:10 INFO Executor: Running task 19.0 in stage 10.0 (TID 36) -26/04/01 08:38:10 INFO Executor: Finished task 12.0 in stage 10.0 (TID 29). 2251 bytes result sent to driver -26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:10 INFO TaskSetManager: Starting task 20.0 in stage 10.0 (TID 37) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:10 INFO TaskSetManager: Finished task 12.0 in stage 10.0 (TID 29) in 1190 ms on 10.0.0.133 (executor driver) (13/208) -26/04/01 08:38:10 INFO Executor: Running task 20.0 in stage 10.0 (TID 37) -26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:10 INFO Executor: Finished task 11.0 in stage 10.0 (TID 28). 2251 bytes result sent to driver -26/04/01 08:38:10 INFO TaskSetManager: Starting task 21.0 in stage 10.0 (TID 38) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:10 INFO TaskSetManager: Finished task 11.0 in stage 10.0 (TID 28) in 1194 ms on 10.0.0.133 (executor driver) (14/208) -26/04/01 08:38:10 INFO Executor: Running task 21.0 in stage 10.0 (TID 38) -26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:10 INFO Executor: Finished task 13.0 in stage 10.0 (TID 30). 2294 bytes result sent to driver -26/04/01 08:38:10 INFO TaskSetManager: Starting task 22.0 in stage 10.0 (TID 39) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:10 INFO TaskSetManager: Finished task 13.0 in stage 10.0 (TID 30) in 1195 ms on 10.0.0.133 (executor driver) (15/208) -26/04/01 08:38:10 INFO Executor: Running task 22.0 in stage 10.0 (TID 39) -26/04/01 08:38:10 INFO Executor: Finished task 15.0 in stage 10.0 (TID 32). 2251 bytes result sent to driver -26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:10 INFO TaskSetManager: Starting task 23.0 in stage 10.0 (TID 40) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:10 INFO TaskSetManager: Finished task 15.0 in stage 10.0 (TID 32) in 1196 ms on 10.0.0.133 (executor driver) (16/208) -26/04/01 08:38:10 INFO Executor: Running task 23.0 in stage 10.0 (TID 40) -26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:10 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:10 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:11 INFO Executor: Finished task 16.0 in stage 10.0 (TID 33). 2251 bytes result sent to driver -26/04/01 08:38:11 INFO TaskSetManager: Starting task 24.0 in stage 10.0 (TID 41) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:11 INFO TaskSetManager: Finished task 16.0 in stage 10.0 (TID 33) in 1056 ms on 10.0.0.133 (executor driver) (17/208) -26/04/01 08:38:11 INFO Executor: Running task 24.0 in stage 10.0 (TID 41) -26/04/01 08:38:11 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:11 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:11 INFO Executor: Finished task 17.0 in stage 10.0 (TID 34). 2251 bytes result sent to driver -26/04/01 08:38:11 INFO TaskSetManager: Starting task 25.0 in stage 10.0 (TID 42) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:11 INFO TaskSetManager: Finished task 17.0 in stage 10.0 (TID 34) in 1021 ms on 10.0.0.133 (executor driver) (18/208) -26/04/01 08:38:11 INFO Executor: Running task 25.0 in stage 10.0 (TID 42) -26/04/01 08:38:11 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:11 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:11 INFO Executor: Finished task 18.0 in stage 10.0 (TID 35). 2251 bytes result sent to driver -26/04/01 08:38:11 INFO TaskSetManager: Starting task 26.0 in stage 10.0 (TID 43) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:11 INFO TaskSetManager: Finished task 18.0 in stage 10.0 (TID 35) in 1036 ms on 10.0.0.133 (executor driver) (19/208) -26/04/01 08:38:11 INFO Executor: Running task 26.0 in stage 10.0 (TID 43) -26/04/01 08:38:11 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:11 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:12 INFO Executor: Finished task 19.0 in stage 10.0 (TID 36). 2251 bytes result sent to driver -26/04/01 08:38:12 INFO TaskSetManager: Starting task 27.0 in stage 10.0 (TID 44) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:12 INFO TaskSetManager: Finished task 19.0 in stage 10.0 (TID 36) in 1121 ms on 10.0.0.133 (executor driver) (20/208) -26/04/01 08:38:12 INFO Executor: Running task 27.0 in stage 10.0 (TID 44) -26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:12 INFO Executor: Finished task 23.0 in stage 10.0 (TID 40). 2251 bytes result sent to driver -26/04/01 08:38:12 INFO TaskSetManager: Starting task 28.0 in stage 10.0 (TID 45) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:12 INFO TaskSetManager: Finished task 23.0 in stage 10.0 (TID 40) in 1113 ms on 10.0.0.133 (executor driver) (21/208) -26/04/01 08:38:12 INFO Executor: Running task 28.0 in stage 10.0 (TID 45) -26/04/01 08:38:12 INFO Executor: Finished task 20.0 in stage 10.0 (TID 37). 2251 bytes result sent to driver -26/04/01 08:38:12 INFO TaskSetManager: Starting task 29.0 in stage 10.0 (TID 46) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:12 INFO TaskSetManager: Finished task 20.0 in stage 10.0 (TID 37) in 1123 ms on 10.0.0.133 (executor driver) (22/208) -26/04/01 08:38:12 INFO Executor: Running task 29.0 in stage 10.0 (TID 46) -26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:12 INFO Executor: Finished task 22.0 in stage 10.0 (TID 39). 2251 bytes result sent to driver -26/04/01 08:38:12 INFO TaskSetManager: Starting task 30.0 in stage 10.0 (TID 47) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:12 INFO TaskSetManager: Finished task 22.0 in stage 10.0 (TID 39) in 1120 ms on 10.0.0.133 (executor driver) (23/208) -26/04/01 08:38:12 INFO Executor: Running task 30.0 in stage 10.0 (TID 47) -26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:12 INFO Executor: Finished task 21.0 in stage 10.0 (TID 38). 2251 bytes result sent to driver -26/04/01 08:38:12 INFO TaskSetManager: Starting task 31.0 in stage 10.0 (TID 48) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:12 INFO Executor: Running task 31.0 in stage 10.0 (TID 48) -26/04/01 08:38:12 INFO TaskSetManager: Finished task 21.0 in stage 10.0 (TID 38) in 1130 ms on 10.0.0.133 (executor driver) (24/208) -26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:12 INFO Executor: Finished task 24.0 in stage 10.0 (TID 41). 2251 bytes result sent to driver -26/04/01 08:38:12 INFO TaskSetManager: Starting task 32.0 in stage 10.0 (TID 49) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:12 INFO TaskSetManager: Finished task 24.0 in stage 10.0 (TID 41) in 1017 ms on 10.0.0.133 (executor driver) (25/208) -26/04/01 08:38:12 INFO Executor: Running task 32.0 in stage 10.0 (TID 49) -26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:12 INFO Executor: Finished task 25.0 in stage 10.0 (TID 42). 2251 bytes result sent to driver -26/04/01 08:38:12 INFO TaskSetManager: Starting task 33.0 in stage 10.0 (TID 50) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:12 INFO TaskSetManager: Finished task 25.0 in stage 10.0 (TID 42) in 979 ms on 10.0.0.133 (executor driver) (26/208) -26/04/01 08:38:12 INFO Executor: Running task 33.0 in stage 10.0 (TID 50) -26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:12 INFO Executor: Finished task 26.0 in stage 10.0 (TID 43). 2251 bytes result sent to driver -26/04/01 08:38:12 INFO TaskSetManager: Starting task 34.0 in stage 10.0 (TID 51) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:12 INFO TaskSetManager: Finished task 26.0 in stage 10.0 (TID 43) in 975 ms on 10.0.0.133 (executor driver) (27/208) -26/04/01 08:38:12 INFO Executor: Running task 34.0 in stage 10.0 (TID 51) -26/04/01 08:38:12 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:12 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:13 INFO Executor: Finished task 31.0 in stage 10.0 (TID 48). 2251 bytes result sent to driver -26/04/01 08:38:13 INFO TaskSetManager: Starting task 35.0 in stage 10.0 (TID 52) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:13 INFO TaskSetManager: Finished task 31.0 in stage 10.0 (TID 48) in 1054 ms on 10.0.0.133 (executor driver) (28/208) -26/04/01 08:38:13 INFO Executor: Running task 35.0 in stage 10.0 (TID 52) -26/04/01 08:38:13 INFO Executor: Finished task 27.0 in stage 10.0 (TID 44). 2251 bytes result sent to driver -26/04/01 08:38:13 INFO TaskSetManager: Starting task 36.0 in stage 10.0 (TID 53) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:13 INFO TaskSetManager: Finished task 27.0 in stage 10.0 (TID 44) in 1068 ms on 10.0.0.133 (executor driver) (29/208) -26/04/01 08:38:13 INFO Executor: Running task 36.0 in stage 10.0 (TID 53) -26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:13 INFO Executor: Finished task 30.0 in stage 10.0 (TID 47). 2251 bytes result sent to driver -26/04/01 08:38:13 INFO TaskSetManager: Starting task 37.0 in stage 10.0 (TID 54) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:13 INFO TaskSetManager: Finished task 30.0 in stage 10.0 (TID 47) in 1066 ms on 10.0.0.133 (executor driver) (30/208) -26/04/01 08:38:13 INFO Executor: Running task 37.0 in stage 10.0 (TID 54) -26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:13 INFO Executor: Finished task 28.0 in stage 10.0 (TID 45). 2251 bytes result sent to driver -26/04/01 08:38:13 INFO TaskSetManager: Starting task 38.0 in stage 10.0 (TID 55) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:13 INFO TaskSetManager: Finished task 28.0 in stage 10.0 (TID 45) in 1080 ms on 10.0.0.133 (executor driver) (31/208) -26/04/01 08:38:13 INFO Executor: Running task 38.0 in stage 10.0 (TID 55) -26/04/01 08:38:13 INFO Executor: Finished task 29.0 in stage 10.0 (TID 46). 2251 bytes result sent to driver -26/04/01 08:38:13 INFO TaskSetManager: Starting task 39.0 in stage 10.0 (TID 56) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:13 INFO TaskSetManager: Finished task 29.0 in stage 10.0 (TID 46) in 1079 ms on 10.0.0.133 (executor driver) (32/208) -26/04/01 08:38:13 INFO Executor: Running task 39.0 in stage 10.0 (TID 56) -26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:13 INFO Executor: Finished task 32.0 in stage 10.0 (TID 49). 2251 bytes result sent to driver -26/04/01 08:38:13 INFO TaskSetManager: Starting task 40.0 in stage 10.0 (TID 57) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:13 INFO TaskSetManager: Finished task 32.0 in stage 10.0 (TID 49) in 984 ms on 10.0.0.133 (executor driver) (33/208) -26/04/01 08:38:13 INFO Executor: Running task 40.0 in stage 10.0 (TID 57) -26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:13 INFO Executor: Finished task 33.0 in stage 10.0 (TID 50). 2251 bytes result sent to driver -26/04/01 08:38:13 INFO TaskSetManager: Starting task 41.0 in stage 10.0 (TID 58) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:13 INFO TaskSetManager: Finished task 33.0 in stage 10.0 (TID 50) in 1000 ms on 10.0.0.133 (executor driver) (34/208) -26/04/01 08:38:13 INFO Executor: Running task 41.0 in stage 10.0 (TID 58) -26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:13 INFO Executor: Finished task 34.0 in stage 10.0 (TID 51). 2251 bytes result sent to driver -26/04/01 08:38:13 INFO TaskSetManager: Starting task 42.0 in stage 10.0 (TID 59) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:13 INFO TaskSetManager: Finished task 34.0 in stage 10.0 (TID 51) in 972 ms on 10.0.0.133 (executor driver) (35/208) -26/04/01 08:38:13 INFO Executor: Running task 42.0 in stage 10.0 (TID 59) -26/04/01 08:38:13 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:13 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:14 INFO Executor: Finished task 35.0 in stage 10.0 (TID 52). 2251 bytes result sent to driver -26/04/01 08:38:14 INFO TaskSetManager: Starting task 43.0 in stage 10.0 (TID 60) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:14 INFO Executor: Running task 43.0 in stage 10.0 (TID 60) -26/04/01 08:38:14 INFO TaskSetManager: Finished task 35.0 in stage 10.0 (TID 52) in 1003 ms on 10.0.0.133 (executor driver) (36/208) -26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:14 INFO Executor: Finished task 39.0 in stage 10.0 (TID 56). 2251 bytes result sent to driver -26/04/01 08:38:14 INFO TaskSetManager: Starting task 44.0 in stage 10.0 (TID 61) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:14 INFO TaskSetManager: Finished task 39.0 in stage 10.0 (TID 56) in 1054 ms on 10.0.0.133 (executor driver) (37/208) -26/04/01 08:38:14 INFO Executor: Running task 44.0 in stage 10.0 (TID 61) -26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:14 INFO Executor: Finished task 36.0 in stage 10.0 (TID 53). 2251 bytes result sent to driver -26/04/01 08:38:14 INFO TaskSetManager: Starting task 45.0 in stage 10.0 (TID 62) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:14 INFO TaskSetManager: Finished task 36.0 in stage 10.0 (TID 53) in 1072 ms on 10.0.0.133 (executor driver) (38/208) -26/04/01 08:38:14 INFO Executor: Running task 45.0 in stage 10.0 (TID 62) -26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:14 INFO Executor: Finished task 37.0 in stage 10.0 (TID 54). 2251 bytes result sent to driver -26/04/01 08:38:14 INFO TaskSetManager: Starting task 46.0 in stage 10.0 (TID 63) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:14 INFO TaskSetManager: Finished task 37.0 in stage 10.0 (TID 54) in 1069 ms on 10.0.0.133 (executor driver) (39/208) -26/04/01 08:38:14 INFO Executor: Running task 46.0 in stage 10.0 (TID 63) -26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:14 INFO Executor: Finished task 38.0 in stage 10.0 (TID 55). 2251 bytes result sent to driver -26/04/01 08:38:14 INFO TaskSetManager: Starting task 47.0 in stage 10.0 (TID 64) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:14 INFO TaskSetManager: Finished task 38.0 in stage 10.0 (TID 55) in 1062 ms on 10.0.0.133 (executor driver) (40/208) -26/04/01 08:38:14 INFO Executor: Running task 47.0 in stage 10.0 (TID 64) -26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:14 INFO Executor: Finished task 40.0 in stage 10.0 (TID 57). 2251 bytes result sent to driver -26/04/01 08:38:14 INFO TaskSetManager: Starting task 48.0 in stage 10.0 (TID 65) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:14 INFO TaskSetManager: Finished task 40.0 in stage 10.0 (TID 57) in 1022 ms on 10.0.0.133 (executor driver) (41/208) -26/04/01 08:38:14 INFO Executor: Running task 48.0 in stage 10.0 (TID 65) -26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:14 INFO Executor: Finished task 41.0 in stage 10.0 (TID 58). 2251 bytes result sent to driver -26/04/01 08:38:14 INFO TaskSetManager: Starting task 49.0 in stage 10.0 (TID 66) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:14 INFO TaskSetManager: Finished task 41.0 in stage 10.0 (TID 58) in 1041 ms on 10.0.0.133 (executor driver) (42/208) -26/04/01 08:38:14 INFO Executor: Running task 49.0 in stage 10.0 (TID 66) -26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:14 INFO Executor: Finished task 42.0 in stage 10.0 (TID 59). 2251 bytes result sent to driver -26/04/01 08:38:14 INFO TaskSetManager: Starting task 50.0 in stage 10.0 (TID 67) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:14 INFO TaskSetManager: Finished task 42.0 in stage 10.0 (TID 59) in 1028 ms on 10.0.0.133 (executor driver) (43/208) -26/04/01 08:38:14 INFO Executor: Running task 50.0 in stage 10.0 (TID 67) -26/04/01 08:38:14 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:14 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:15 INFO Executor: Finished task 43.0 in stage 10.0 (TID 60). 2251 bytes result sent to driver -26/04/01 08:38:15 INFO TaskSetManager: Starting task 51.0 in stage 10.0 (TID 68) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:15 INFO TaskSetManager: Finished task 43.0 in stage 10.0 (TID 60) in 990 ms on 10.0.0.133 (executor driver) (44/208) -26/04/01 08:38:15 INFO Executor: Running task 51.0 in stage 10.0 (TID 68) -26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:15 INFO Executor: Finished task 47.0 in stage 10.0 (TID 64). 2251 bytes result sent to driver -26/04/01 08:38:15 INFO TaskSetManager: Starting task 52.0 in stage 10.0 (TID 69) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:15 INFO TaskSetManager: Finished task 47.0 in stage 10.0 (TID 64) in 1051 ms on 10.0.0.133 (executor driver) (45/208) -26/04/01 08:38:15 INFO Executor: Running task 52.0 in stage 10.0 (TID 69) -26/04/01 08:38:15 INFO Executor: Finished task 45.0 in stage 10.0 (TID 62). 2251 bytes result sent to driver -26/04/01 08:38:15 INFO TaskSetManager: Starting task 53.0 in stage 10.0 (TID 70) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:15 INFO TaskSetManager: Finished task 45.0 in stage 10.0 (TID 62) in 1055 ms on 10.0.0.133 (executor driver) (46/208) -26/04/01 08:38:15 INFO Executor: Running task 53.0 in stage 10.0 (TID 70) -26/04/01 08:38:15 INFO Executor: Finished task 44.0 in stage 10.0 (TID 61). 2251 bytes result sent to driver -26/04/01 08:38:15 INFO TaskSetManager: Starting task 54.0 in stage 10.0 (TID 71) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:15 INFO TaskSetManager: Finished task 44.0 in stage 10.0 (TID 61) in 1059 ms on 10.0.0.133 (executor driver) (47/208) -26/04/01 08:38:15 INFO Executor: Running task 54.0 in stage 10.0 (TID 71) -26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:15 INFO Executor: Finished task 46.0 in stage 10.0 (TID 63). 2251 bytes result sent to driver -26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:15 INFO TaskSetManager: Starting task 55.0 in stage 10.0 (TID 72) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:15 INFO TaskSetManager: Finished task 46.0 in stage 10.0 (TID 63) in 1056 ms on 10.0.0.133 (executor driver) (48/208) -26/04/01 08:38:15 INFO Executor: Running task 55.0 in stage 10.0 (TID 72) -26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:15 INFO Executor: Finished task 48.0 in stage 10.0 (TID 65). 2294 bytes result sent to driver -26/04/01 08:38:15 INFO TaskSetManager: Starting task 56.0 in stage 10.0 (TID 73) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:15 INFO Executor: Running task 56.0 in stage 10.0 (TID 73) -26/04/01 08:38:15 INFO TaskSetManager: Finished task 48.0 in stage 10.0 (TID 65) in 1023 ms on 10.0.0.133 (executor driver) (49/208) -26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:15 INFO Executor: Finished task 49.0 in stage 10.0 (TID 66). 2251 bytes result sent to driver -26/04/01 08:38:15 INFO TaskSetManager: Starting task 57.0 in stage 10.0 (TID 74) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:15 INFO TaskSetManager: Finished task 49.0 in stage 10.0 (TID 66) in 1006 ms on 10.0.0.133 (executor driver) (50/208) -26/04/01 08:38:15 INFO Executor: Running task 57.0 in stage 10.0 (TID 74) -26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:15 INFO Executor: Finished task 50.0 in stage 10.0 (TID 67). 2251 bytes result sent to driver -26/04/01 08:38:15 INFO TaskSetManager: Starting task 58.0 in stage 10.0 (TID 75) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:15 INFO Executor: Running task 58.0 in stage 10.0 (TID 75) -26/04/01 08:38:15 INFO TaskSetManager: Finished task 50.0 in stage 10.0 (TID 67) in 989 ms on 10.0.0.133 (executor driver) (51/208) -26/04/01 08:38:15 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:15 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:16 INFO Executor: Finished task 51.0 in stage 10.0 (TID 68). 2251 bytes result sent to driver -26/04/01 08:38:16 INFO TaskSetManager: Starting task 59.0 in stage 10.0 (TID 76) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:16 INFO TaskSetManager: Finished task 51.0 in stage 10.0 (TID 68) in 1002 ms on 10.0.0.133 (executor driver) (52/208) -26/04/01 08:38:16 INFO Executor: Running task 59.0 in stage 10.0 (TID 76) -26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:16 INFO Executor: Finished task 53.0 in stage 10.0 (TID 70). 2251 bytes result sent to driver -26/04/01 08:38:16 INFO TaskSetManager: Starting task 60.0 in stage 10.0 (TID 77) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:16 INFO TaskSetManager: Finished task 53.0 in stage 10.0 (TID 70) in 1075 ms on 10.0.0.133 (executor driver) (53/208) -26/04/01 08:38:16 INFO Executor: Running task 60.0 in stage 10.0 (TID 77) -26/04/01 08:38:16 INFO Executor: Finished task 52.0 in stage 10.0 (TID 69). 2251 bytes result sent to driver -26/04/01 08:38:16 INFO TaskSetManager: Starting task 61.0 in stage 10.0 (TID 78) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:16 INFO Executor: Running task 61.0 in stage 10.0 (TID 78) -26/04/01 08:38:16 INFO TaskSetManager: Finished task 52.0 in stage 10.0 (TID 69) in 1077 ms on 10.0.0.133 (executor driver) (54/208) -26/04/01 08:38:16 INFO Executor: Finished task 55.0 in stage 10.0 (TID 72). 2251 bytes result sent to driver -26/04/01 08:38:16 INFO TaskSetManager: Starting task 62.0 in stage 10.0 (TID 79) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:16 INFO TaskSetManager: Finished task 55.0 in stage 10.0 (TID 72) in 1074 ms on 10.0.0.133 (executor driver) (55/208) -26/04/01 08:38:16 INFO Executor: Running task 62.0 in stage 10.0 (TID 79) -26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:16 INFO Executor: Finished task 54.0 in stage 10.0 (TID 71). 2251 bytes result sent to driver -26/04/01 08:38:16 INFO TaskSetManager: Starting task 63.0 in stage 10.0 (TID 80) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:16 INFO TaskSetManager: Finished task 54.0 in stage 10.0 (TID 71) in 1077 ms on 10.0.0.133 (executor driver) (56/208) -26/04/01 08:38:16 INFO Executor: Running task 63.0 in stage 10.0 (TID 80) -26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:16 INFO Executor: Finished task 56.0 in stage 10.0 (TID 73). 2251 bytes result sent to driver -26/04/01 08:38:16 INFO TaskSetManager: Starting task 64.0 in stage 10.0 (TID 81) (10.0.0.133, executor driver, partition 64, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:16 INFO TaskSetManager: Finished task 56.0 in stage 10.0 (TID 73) in 1032 ms on 10.0.0.133 (executor driver) (57/208) -26/04/01 08:38:16 INFO Executor: Running task 64.0 in stage 10.0 (TID 81) -26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:16 INFO Executor: Finished task 57.0 in stage 10.0 (TID 74). 2251 bytes result sent to driver -26/04/01 08:38:16 INFO TaskSetManager: Starting task 65.0 in stage 10.0 (TID 82) (10.0.0.133, executor driver, partition 65, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:16 INFO TaskSetManager: Finished task 57.0 in stage 10.0 (TID 74) in 1050 ms on 10.0.0.133 (executor driver) (58/208) -26/04/01 08:38:16 INFO Executor: Running task 65.0 in stage 10.0 (TID 82) -26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:16 INFO Executor: Finished task 58.0 in stage 10.0 (TID 75). 2251 bytes result sent to driver -26/04/01 08:38:16 INFO TaskSetManager: Starting task 66.0 in stage 10.0 (TID 83) (10.0.0.133, executor driver, partition 66, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:16 INFO TaskSetManager: Finished task 58.0 in stage 10.0 (TID 75) in 1046 ms on 10.0.0.133 (executor driver) (59/208) -26/04/01 08:38:16 INFO Executor: Running task 66.0 in stage 10.0 (TID 83) -26/04/01 08:38:16 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:16 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:17 INFO Executor: Finished task 59.0 in stage 10.0 (TID 76). 2251 bytes result sent to driver -26/04/01 08:38:17 INFO TaskSetManager: Starting task 67.0 in stage 10.0 (TID 84) (10.0.0.133, executor driver, partition 67, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:17 INFO TaskSetManager: Finished task 59.0 in stage 10.0 (TID 76) in 1071 ms on 10.0.0.133 (executor driver) (60/208) -26/04/01 08:38:17 INFO Executor: Running task 67.0 in stage 10.0 (TID 84) -26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:17 INFO Executor: Finished task 63.0 in stage 10.0 (TID 80). 2251 bytes result sent to driver -26/04/01 08:38:17 INFO TaskSetManager: Starting task 68.0 in stage 10.0 (TID 85) (10.0.0.133, executor driver, partition 68, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:17 INFO TaskSetManager: Finished task 63.0 in stage 10.0 (TID 80) in 1115 ms on 10.0.0.133 (executor driver) (61/208) -26/04/01 08:38:17 INFO Executor: Running task 68.0 in stage 10.0 (TID 85) -26/04/01 08:38:17 INFO Executor: Finished task 61.0 in stage 10.0 (TID 78). 2251 bytes result sent to driver -26/04/01 08:38:17 INFO TaskSetManager: Starting task 69.0 in stage 10.0 (TID 86) (10.0.0.133, executor driver, partition 69, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:17 INFO TaskSetManager: Finished task 61.0 in stage 10.0 (TID 78) in 1119 ms on 10.0.0.133 (executor driver) (62/208) -26/04/01 08:38:17 INFO Executor: Running task 69.0 in stage 10.0 (TID 86) -26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:17 INFO Executor: Finished task 62.0 in stage 10.0 (TID 79). 2251 bytes result sent to driver -26/04/01 08:38:17 INFO TaskSetManager: Starting task 70.0 in stage 10.0 (TID 87) (10.0.0.133, executor driver, partition 70, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:17 INFO TaskSetManager: Finished task 62.0 in stage 10.0 (TID 79) in 1121 ms on 10.0.0.133 (executor driver) (63/208) -26/04/01 08:38:17 INFO Executor: Running task 70.0 in stage 10.0 (TID 87) -26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:17 INFO Executor: Finished task 60.0 in stage 10.0 (TID 77). 2251 bytes result sent to driver -26/04/01 08:38:17 INFO TaskSetManager: Starting task 71.0 in stage 10.0 (TID 88) (10.0.0.133, executor driver, partition 71, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:17 INFO TaskSetManager: Finished task 60.0 in stage 10.0 (TID 77) in 1129 ms on 10.0.0.133 (executor driver) (64/208) -26/04/01 08:38:17 INFO Executor: Running task 71.0 in stage 10.0 (TID 88) -26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:17 INFO Executor: Finished task 64.0 in stage 10.0 (TID 81). 2251 bytes result sent to driver -26/04/01 08:38:17 INFO TaskSetManager: Starting task 72.0 in stage 10.0 (TID 89) (10.0.0.133, executor driver, partition 72, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:17 INFO TaskSetManager: Finished task 64.0 in stage 10.0 (TID 81) in 1065 ms on 10.0.0.133 (executor driver) (65/208) -26/04/01 08:38:17 INFO Executor: Running task 72.0 in stage 10.0 (TID 89) -26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:17 INFO Executor: Finished task 66.0 in stage 10.0 (TID 83). 2251 bytes result sent to driver -26/04/01 08:38:17 INFO TaskSetManager: Starting task 73.0 in stage 10.0 (TID 90) (10.0.0.133, executor driver, partition 73, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:17 INFO Executor: Running task 73.0 in stage 10.0 (TID 90) -26/04/01 08:38:17 INFO TaskSetManager: Finished task 66.0 in stage 10.0 (TID 83) in 1076 ms on 10.0.0.133 (executor driver) (66/208) -26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:17 INFO Executor: Finished task 65.0 in stage 10.0 (TID 82). 2251 bytes result sent to driver -26/04/01 08:38:17 INFO TaskSetManager: Starting task 74.0 in stage 10.0 (TID 91) (10.0.0.133, executor driver, partition 74, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:17 INFO TaskSetManager: Finished task 65.0 in stage 10.0 (TID 82) in 1096 ms on 10.0.0.133 (executor driver) (67/208) -26/04/01 08:38:17 INFO Executor: Running task 74.0 in stage 10.0 (TID 91) -26/04/01 08:38:17 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:17 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:18 INFO Executor: Finished task 67.0 in stage 10.0 (TID 84). 2251 bytes result sent to driver -26/04/01 08:38:18 INFO TaskSetManager: Starting task 75.0 in stage 10.0 (TID 92) (10.0.0.133, executor driver, partition 75, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:18 INFO TaskSetManager: Finished task 67.0 in stage 10.0 (TID 84) in 1097 ms on 10.0.0.133 (executor driver) (68/208) -26/04/01 08:38:18 INFO Executor: Running task 75.0 in stage 10.0 (TID 92) -26/04/01 08:38:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:18 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:18 INFO Executor: Finished task 69.0 in stage 10.0 (TID 86). 2251 bytes result sent to driver -26/04/01 08:38:18 INFO TaskSetManager: Starting task 76.0 in stage 10.0 (TID 93) (10.0.0.133, executor driver, partition 76, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:18 INFO TaskSetManager: Finished task 69.0 in stage 10.0 (TID 86) in 1239 ms on 10.0.0.133 (executor driver) (69/208) -26/04/01 08:38:18 INFO Executor: Running task 76.0 in stage 10.0 (TID 93) -26/04/01 08:38:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:18 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:18 INFO Executor: Finished task 71.0 in stage 10.0 (TID 88). 2251 bytes result sent to driver -26/04/01 08:38:18 INFO TaskSetManager: Starting task 77.0 in stage 10.0 (TID 94) (10.0.0.133, executor driver, partition 77, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:18 INFO Executor: Finished task 70.0 in stage 10.0 (TID 87). 2251 bytes result sent to driver -26/04/01 08:38:18 INFO TaskSetManager: Finished task 71.0 in stage 10.0 (TID 88) in 1247 ms on 10.0.0.133 (executor driver) (70/208) -26/04/01 08:38:18 INFO Executor: Running task 77.0 in stage 10.0 (TID 94) -26/04/01 08:38:18 INFO TaskSetManager: Starting task 78.0 in stage 10.0 (TID 95) (10.0.0.133, executor driver, partition 78, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:18 INFO TaskSetManager: Finished task 70.0 in stage 10.0 (TID 87) in 1255 ms on 10.0.0.133 (executor driver) (71/208) -26/04/01 08:38:18 INFO Executor: Running task 78.0 in stage 10.0 (TID 95) -26/04/01 08:38:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:18 INFO Executor: Finished task 68.0 in stage 10.0 (TID 85). 2251 bytes result sent to driver -26/04/01 08:38:18 INFO TaskSetManager: Starting task 79.0 in stage 10.0 (TID 96) (10.0.0.133, executor driver, partition 79, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:18 INFO TaskSetManager: Finished task 68.0 in stage 10.0 (TID 85) in 1265 ms on 10.0.0.133 (executor driver) (72/208) -26/04/01 08:38:18 INFO Executor: Running task 79.0 in stage 10.0 (TID 96) -26/04/01 08:38:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:18 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:18 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:18 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:18 INFO Executor: Finished task 72.0 in stage 10.0 (TID 89). 2251 bytes result sent to driver -26/04/01 08:38:18 INFO TaskSetManager: Starting task 80.0 in stage 10.0 (TID 97) (10.0.0.133, executor driver, partition 80, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:18 INFO TaskSetManager: Finished task 72.0 in stage 10.0 (TID 89) in 1122 ms on 10.0.0.133 (executor driver) (73/208) -26/04/01 08:38:18 INFO Executor: Running task 80.0 in stage 10.0 (TID 97) -26/04/01 08:38:18 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:18 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:19 INFO Executor: Finished task 73.0 in stage 10.0 (TID 90). 2251 bytes result sent to driver -26/04/01 08:38:19 INFO TaskSetManager: Starting task 81.0 in stage 10.0 (TID 98) (10.0.0.133, executor driver, partition 81, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:19 INFO TaskSetManager: Finished task 73.0 in stage 10.0 (TID 90) in 1088 ms on 10.0.0.133 (executor driver) (74/208) -26/04/01 08:38:19 INFO Executor: Running task 81.0 in stage 10.0 (TID 98) -26/04/01 08:38:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:19 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:19 INFO Executor: Finished task 74.0 in stage 10.0 (TID 91). 2251 bytes result sent to driver -26/04/01 08:38:19 INFO TaskSetManager: Starting task 82.0 in stage 10.0 (TID 99) (10.0.0.133, executor driver, partition 82, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:19 INFO TaskSetManager: Finished task 74.0 in stage 10.0 (TID 91) in 1177 ms on 10.0.0.133 (executor driver) (75/208) -26/04/01 08:38:19 INFO Executor: Running task 82.0 in stage 10.0 (TID 99) -26/04/01 08:38:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:19 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:19 INFO Executor: Finished task 75.0 in stage 10.0 (TID 92). 2251 bytes result sent to driver -26/04/01 08:38:19 INFO TaskSetManager: Starting task 83.0 in stage 10.0 (TID 100) (10.0.0.133, executor driver, partition 83, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:19 INFO TaskSetManager: Finished task 75.0 in stage 10.0 (TID 92) in 1173 ms on 10.0.0.133 (executor driver) (76/208) -26/04/01 08:38:19 INFO Executor: Running task 83.0 in stage 10.0 (TID 100) -26/04/01 08:38:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:19 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:19 INFO Executor: Finished task 76.0 in stage 10.0 (TID 93). 2251 bytes result sent to driver -26/04/01 08:38:19 INFO TaskSetManager: Starting task 84.0 in stage 10.0 (TID 101) (10.0.0.133, executor driver, partition 84, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:19 INFO Executor: Running task 84.0 in stage 10.0 (TID 101) -26/04/01 08:38:19 INFO TaskSetManager: Finished task 76.0 in stage 10.0 (TID 93) in 1106 ms on 10.0.0.133 (executor driver) (77/208) -26/04/01 08:38:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:19 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:19 INFO Executor: Finished task 77.0 in stage 10.0 (TID 94). 2251 bytes result sent to driver -26/04/01 08:38:19 INFO TaskSetManager: Starting task 85.0 in stage 10.0 (TID 102) (10.0.0.133, executor driver, partition 85, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:19 INFO TaskSetManager: Finished task 77.0 in stage 10.0 (TID 94) in 1156 ms on 10.0.0.133 (executor driver) (78/208) -26/04/01 08:38:19 INFO Executor: Running task 85.0 in stage 10.0 (TID 102) -26/04/01 08:38:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:19 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:19 INFO Executor: Finished task 79.0 in stage 10.0 (TID 96). 2251 bytes result sent to driver -26/04/01 08:38:19 INFO TaskSetManager: Starting task 86.0 in stage 10.0 (TID 103) (10.0.0.133, executor driver, partition 86, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:19 INFO TaskSetManager: Finished task 79.0 in stage 10.0 (TID 96) in 1169 ms on 10.0.0.133 (executor driver) (79/208) -26/04/01 08:38:19 INFO Executor: Running task 86.0 in stage 10.0 (TID 103) -26/04/01 08:38:19 INFO Executor: Finished task 78.0 in stage 10.0 (TID 95). 2251 bytes result sent to driver -26/04/01 08:38:19 INFO TaskSetManager: Starting task 87.0 in stage 10.0 (TID 104) (10.0.0.133, executor driver, partition 87, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:19 INFO TaskSetManager: Finished task 78.0 in stage 10.0 (TID 95) in 1177 ms on 10.0.0.133 (executor driver) (80/208) -26/04/01 08:38:19 INFO Executor: Running task 87.0 in stage 10.0 (TID 104) -26/04/01 08:38:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:19 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:19 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:19 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:20 INFO Executor: Finished task 80.0 in stage 10.0 (TID 97). 2251 bytes result sent to driver -26/04/01 08:38:20 INFO TaskSetManager: Starting task 88.0 in stage 10.0 (TID 105) (10.0.0.133, executor driver, partition 88, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:20 INFO TaskSetManager: Finished task 80.0 in stage 10.0 (TID 97) in 1066 ms on 10.0.0.133 (executor driver) (81/208) -26/04/01 08:38:20 INFO Executor: Running task 88.0 in stage 10.0 (TID 105) -26/04/01 08:38:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:20 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:20 INFO Executor: Finished task 81.0 in stage 10.0 (TID 98). 2251 bytes result sent to driver -26/04/01 08:38:20 INFO TaskSetManager: Starting task 89.0 in stage 10.0 (TID 106) (10.0.0.133, executor driver, partition 89, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:20 INFO Executor: Running task 89.0 in stage 10.0 (TID 106) -26/04/01 08:38:20 INFO TaskSetManager: Finished task 81.0 in stage 10.0 (TID 98) in 1075 ms on 10.0.0.133 (executor driver) (82/208) -26/04/01 08:38:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:20 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:20 INFO Executor: Finished task 82.0 in stage 10.0 (TID 99). 2251 bytes result sent to driver -26/04/01 08:38:20 INFO TaskSetManager: Starting task 90.0 in stage 10.0 (TID 107) (10.0.0.133, executor driver, partition 90, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:20 INFO TaskSetManager: Finished task 82.0 in stage 10.0 (TID 99) in 1093 ms on 10.0.0.133 (executor driver) (83/208) -26/04/01 08:38:20 INFO Executor: Running task 90.0 in stage 10.0 (TID 107) -26/04/01 08:38:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:20 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:20 INFO Executor: Finished task 83.0 in stage 10.0 (TID 100). 2251 bytes result sent to driver -26/04/01 08:38:20 INFO TaskSetManager: Starting task 91.0 in stage 10.0 (TID 108) (10.0.0.133, executor driver, partition 91, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:20 INFO TaskSetManager: Finished task 83.0 in stage 10.0 (TID 100) in 1079 ms on 10.0.0.133 (executor driver) (84/208) -26/04/01 08:38:20 INFO Executor: Running task 91.0 in stage 10.0 (TID 108) -26/04/01 08:38:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:20 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:20 INFO Executor: Finished task 84.0 in stage 10.0 (TID 101). 2294 bytes result sent to driver -26/04/01 08:38:20 INFO TaskSetManager: Starting task 92.0 in stage 10.0 (TID 109) (10.0.0.133, executor driver, partition 92, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:20 INFO TaskSetManager: Finished task 84.0 in stage 10.0 (TID 101) in 1087 ms on 10.0.0.133 (executor driver) (85/208) -26/04/01 08:38:20 INFO Executor: Running task 92.0 in stage 10.0 (TID 109) -26/04/01 08:38:20 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:20 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:21 INFO Executor: Finished task 85.0 in stage 10.0 (TID 102). 2251 bytes result sent to driver -26/04/01 08:38:21 INFO TaskSetManager: Starting task 93.0 in stage 10.0 (TID 110) (10.0.0.133, executor driver, partition 93, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:21 INFO Executor: Running task 93.0 in stage 10.0 (TID 110) -26/04/01 08:38:21 INFO TaskSetManager: Finished task 85.0 in stage 10.0 (TID 102) in 1154 ms on 10.0.0.133 (executor driver) (86/208) -26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:21 INFO Executor: Finished task 87.0 in stage 10.0 (TID 104). 2251 bytes result sent to driver -26/04/01 08:38:21 INFO TaskSetManager: Starting task 94.0 in stage 10.0 (TID 111) (10.0.0.133, executor driver, partition 94, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:21 INFO Executor: Running task 94.0 in stage 10.0 (TID 111) -26/04/01 08:38:21 INFO TaskSetManager: Finished task 87.0 in stage 10.0 (TID 104) in 1137 ms on 10.0.0.133 (executor driver) (87/208) -26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:21 INFO Executor: Finished task 86.0 in stage 10.0 (TID 103). 2251 bytes result sent to driver -26/04/01 08:38:21 INFO TaskSetManager: Starting task 95.0 in stage 10.0 (TID 112) (10.0.0.133, executor driver, partition 95, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:21 INFO TaskSetManager: Finished task 86.0 in stage 10.0 (TID 103) in 1145 ms on 10.0.0.133 (executor driver) (88/208) -26/04/01 08:38:21 INFO Executor: Running task 95.0 in stage 10.0 (TID 112) -26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:21 INFO Executor: Finished task 88.0 in stage 10.0 (TID 105). 2251 bytes result sent to driver -26/04/01 08:38:21 INFO TaskSetManager: Starting task 96.0 in stage 10.0 (TID 113) (10.0.0.133, executor driver, partition 96, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:21 INFO Executor: Running task 96.0 in stage 10.0 (TID 113) -26/04/01 08:38:21 INFO TaskSetManager: Finished task 88.0 in stage 10.0 (TID 105) in 1094 ms on 10.0.0.133 (executor driver) (89/208) -26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:21 INFO Executor: Finished task 89.0 in stage 10.0 (TID 106). 2251 bytes result sent to driver -26/04/01 08:38:21 INFO TaskSetManager: Starting task 97.0 in stage 10.0 (TID 114) (10.0.0.133, executor driver, partition 97, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:21 INFO TaskSetManager: Finished task 89.0 in stage 10.0 (TID 106) in 1042 ms on 10.0.0.133 (executor driver) (90/208) -26/04/01 08:38:21 INFO Executor: Running task 97.0 in stage 10.0 (TID 114) -26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:21 INFO Executor: Finished task 90.0 in stage 10.0 (TID 107). 2251 bytes result sent to driver -26/04/01 08:38:21 INFO TaskSetManager: Starting task 98.0 in stage 10.0 (TID 115) (10.0.0.133, executor driver, partition 98, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:21 INFO TaskSetManager: Finished task 90.0 in stage 10.0 (TID 107) in 1142 ms on 10.0.0.133 (executor driver) (91/208) -26/04/01 08:38:21 INFO Executor: Running task 98.0 in stage 10.0 (TID 115) -26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:21 INFO Executor: Finished task 91.0 in stage 10.0 (TID 108). 2251 bytes result sent to driver -26/04/01 08:38:21 INFO TaskSetManager: Starting task 99.0 in stage 10.0 (TID 116) (10.0.0.133, executor driver, partition 99, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:21 INFO TaskSetManager: Finished task 91.0 in stage 10.0 (TID 108) in 1131 ms on 10.0.0.133 (executor driver) (92/208) -26/04/01 08:38:21 INFO Executor: Running task 99.0 in stage 10.0 (TID 116) -26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:21 INFO Executor: Finished task 92.0 in stage 10.0 (TID 109). 2251 bytes result sent to driver -26/04/01 08:38:21 INFO TaskSetManager: Starting task 100.0 in stage 10.0 (TID 117) (10.0.0.133, executor driver, partition 100, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:21 INFO TaskSetManager: Finished task 92.0 in stage 10.0 (TID 109) in 1102 ms on 10.0.0.133 (executor driver) (93/208) -26/04/01 08:38:21 INFO Executor: Running task 100.0 in stage 10.0 (TID 117) -26/04/01 08:38:21 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:21 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:22 INFO Executor: Finished task 93.0 in stage 10.0 (TID 110). 2251 bytes result sent to driver -26/04/01 08:38:22 INFO TaskSetManager: Starting task 101.0 in stage 10.0 (TID 118) (10.0.0.133, executor driver, partition 101, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:22 INFO TaskSetManager: Finished task 93.0 in stage 10.0 (TID 110) in 1158 ms on 10.0.0.133 (executor driver) (94/208) -26/04/01 08:38:22 INFO Executor: Running task 101.0 in stage 10.0 (TID 118) -26/04/01 08:38:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:22 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:22 INFO Executor: Finished task 95.0 in stage 10.0 (TID 112). 2251 bytes result sent to driver -26/04/01 08:38:22 INFO TaskSetManager: Starting task 102.0 in stage 10.0 (TID 119) (10.0.0.133, executor driver, partition 102, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:22 INFO TaskSetManager: Finished task 95.0 in stage 10.0 (TID 112) in 1180 ms on 10.0.0.133 (executor driver) (95/208) -26/04/01 08:38:22 INFO Executor: Running task 102.0 in stage 10.0 (TID 119) -26/04/01 08:38:22 INFO Executor: Finished task 94.0 in stage 10.0 (TID 111). 2251 bytes result sent to driver -26/04/01 08:38:22 INFO TaskSetManager: Starting task 103.0 in stage 10.0 (TID 120) (10.0.0.133, executor driver, partition 103, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:22 INFO TaskSetManager: Finished task 94.0 in stage 10.0 (TID 111) in 1188 ms on 10.0.0.133 (executor driver) (96/208) -26/04/01 08:38:22 INFO Executor: Running task 103.0 in stage 10.0 (TID 120) -26/04/01 08:38:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:22 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:22 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:22 INFO Executor: Finished task 96.0 in stage 10.0 (TID 113). 2251 bytes result sent to driver -26/04/01 08:38:22 INFO TaskSetManager: Starting task 104.0 in stage 10.0 (TID 121) (10.0.0.133, executor driver, partition 104, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:22 INFO TaskSetManager: Finished task 96.0 in stage 10.0 (TID 113) in 1138 ms on 10.0.0.133 (executor driver) (97/208) -26/04/01 08:38:22 INFO Executor: Running task 104.0 in stage 10.0 (TID 121) -26/04/01 08:38:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:22 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:22 INFO Executor: Finished task 97.0 in stage 10.0 (TID 114). 2251 bytes result sent to driver -26/04/01 08:38:22 INFO TaskSetManager: Starting task 105.0 in stage 10.0 (TID 122) (10.0.0.133, executor driver, partition 105, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:22 INFO TaskSetManager: Finished task 97.0 in stage 10.0 (TID 114) in 1156 ms on 10.0.0.133 (executor driver) (98/208) -26/04/01 08:38:22 INFO Executor: Running task 105.0 in stage 10.0 (TID 122) -26/04/01 08:38:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:22 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:22 INFO Executor: Finished task 98.0 in stage 10.0 (TID 115). 2251 bytes result sent to driver -26/04/01 08:38:22 INFO TaskSetManager: Starting task 106.0 in stage 10.0 (TID 123) (10.0.0.133, executor driver, partition 106, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:22 INFO TaskSetManager: Finished task 98.0 in stage 10.0 (TID 115) in 1103 ms on 10.0.0.133 (executor driver) (99/208) -26/04/01 08:38:22 INFO Executor: Running task 106.0 in stage 10.0 (TID 123) -26/04/01 08:38:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:22 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:22 INFO Executor: Finished task 99.0 in stage 10.0 (TID 116). 2251 bytes result sent to driver -26/04/01 08:38:22 INFO TaskSetManager: Starting task 107.0 in stage 10.0 (TID 124) (10.0.0.133, executor driver, partition 107, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:22 INFO TaskSetManager: Finished task 99.0 in stage 10.0 (TID 116) in 1147 ms on 10.0.0.133 (executor driver) (100/208) -26/04/01 08:38:22 INFO Executor: Running task 107.0 in stage 10.0 (TID 124) -26/04/01 08:38:22 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:22 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:23 INFO Executor: Finished task 100.0 in stage 10.0 (TID 117). 2251 bytes result sent to driver -26/04/01 08:38:23 INFO TaskSetManager: Starting task 108.0 in stage 10.0 (TID 125) (10.0.0.133, executor driver, partition 108, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:23 INFO Executor: Running task 108.0 in stage 10.0 (TID 125) -26/04/01 08:38:23 INFO TaskSetManager: Finished task 100.0 in stage 10.0 (TID 117) in 1158 ms on 10.0.0.133 (executor driver) (101/208) -26/04/01 08:38:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:23 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:23 INFO Executor: Finished task 101.0 in stage 10.0 (TID 118). 2251 bytes result sent to driver -26/04/01 08:38:23 INFO TaskSetManager: Starting task 109.0 in stage 10.0 (TID 126) (10.0.0.133, executor driver, partition 109, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:23 INFO TaskSetManager: Finished task 101.0 in stage 10.0 (TID 118) in 1065 ms on 10.0.0.133 (executor driver) (102/208) -26/04/01 08:38:23 INFO Executor: Running task 109.0 in stage 10.0 (TID 126) -26/04/01 08:38:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:23 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:23 INFO Executor: Finished task 103.0 in stage 10.0 (TID 120). 2251 bytes result sent to driver -26/04/01 08:38:23 INFO TaskSetManager: Starting task 110.0 in stage 10.0 (TID 127) (10.0.0.133, executor driver, partition 110, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:23 INFO TaskSetManager: Finished task 103.0 in stage 10.0 (TID 120) in 1298 ms on 10.0.0.133 (executor driver) (103/208) -26/04/01 08:38:23 INFO Executor: Running task 110.0 in stage 10.0 (TID 127) -26/04/01 08:38:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:23 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:23 INFO Executor: Finished task 102.0 in stage 10.0 (TID 119). 2251 bytes result sent to driver -26/04/01 08:38:23 INFO TaskSetManager: Starting task 111.0 in stage 10.0 (TID 128) (10.0.0.133, executor driver, partition 111, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:23 INFO TaskSetManager: Finished task 102.0 in stage 10.0 (TID 119) in 1378 ms on 10.0.0.133 (executor driver) (104/208) -26/04/01 08:38:23 INFO Executor: Running task 111.0 in stage 10.0 (TID 128) -26/04/01 08:38:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:23 INFO Executor: Finished task 105.0 in stage 10.0 (TID 122). 2251 bytes result sent to driver -26/04/01 08:38:23 INFO TaskSetManager: Starting task 112.0 in stage 10.0 (TID 129) (10.0.0.133, executor driver, partition 112, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:23 INFO TaskSetManager: Finished task 105.0 in stage 10.0 (TID 122) in 1266 ms on 10.0.0.133 (executor driver) (105/208) -26/04/01 08:38:23 INFO Executor: Running task 112.0 in stage 10.0 (TID 129) -26/04/01 08:38:23 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:23 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:23 INFO Executor: Finished task 106.0 in stage 10.0 (TID 123). 2251 bytes result sent to driver -26/04/01 08:38:23 INFO TaskSetManager: Starting task 113.0 in stage 10.0 (TID 130) (10.0.0.133, executor driver, partition 113, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:23 INFO TaskSetManager: Finished task 106.0 in stage 10.0 (TID 123) in 1239 ms on 10.0.0.133 (executor driver) (106/208) -26/04/01 08:38:23 INFO Executor: Running task 113.0 in stage 10.0 (TID 130) -26/04/01 08:38:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:23 INFO Executor: Finished task 104.0 in stage 10.0 (TID 121). 2251 bytes result sent to driver -26/04/01 08:38:23 INFO TaskSetManager: Starting task 114.0 in stage 10.0 (TID 131) (10.0.0.133, executor driver, partition 114, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:23 INFO Executor: Running task 114.0 in stage 10.0 (TID 131) -26/04/01 08:38:23 INFO TaskSetManager: Finished task 104.0 in stage 10.0 (TID 121) in 1489 ms on 10.0.0.133 (executor driver) (107/208) -26/04/01 08:38:23 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:23 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:23 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:24 INFO Executor: Finished task 107.0 in stage 10.0 (TID 124). 2251 bytes result sent to driver -26/04/01 08:38:24 INFO TaskSetManager: Starting task 115.0 in stage 10.0 (TID 132) (10.0.0.133, executor driver, partition 115, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:24 INFO TaskSetManager: Finished task 107.0 in stage 10.0 (TID 124) in 1204 ms on 10.0.0.133 (executor driver) (108/208) -26/04/01 08:38:24 INFO Executor: Running task 115.0 in stage 10.0 (TID 132) -26/04/01 08:38:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:24 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:24 INFO Executor: Finished task 108.0 in stage 10.0 (TID 125). 2251 bytes result sent to driver -26/04/01 08:38:24 INFO TaskSetManager: Starting task 116.0 in stage 10.0 (TID 133) (10.0.0.133, executor driver, partition 116, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:24 INFO TaskSetManager: Finished task 108.0 in stage 10.0 (TID 125) in 1168 ms on 10.0.0.133 (executor driver) (109/208) -26/04/01 08:38:24 INFO Executor: Running task 116.0 in stage 10.0 (TID 133) -26/04/01 08:38:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:24 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:24 INFO Executor: Finished task 109.0 in stage 10.0 (TID 126). 2251 bytes result sent to driver -26/04/01 08:38:24 INFO TaskSetManager: Starting task 117.0 in stage 10.0 (TID 134) (10.0.0.133, executor driver, partition 117, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:24 INFO Executor: Running task 117.0 in stage 10.0 (TID 134) -26/04/01 08:38:24 INFO TaskSetManager: Finished task 109.0 in stage 10.0 (TID 126) in 1150 ms on 10.0.0.133 (executor driver) (110/208) -26/04/01 08:38:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:24 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:24 INFO Executor: Finished task 110.0 in stage 10.0 (TID 127). 2251 bytes result sent to driver -26/04/01 08:38:24 INFO TaskSetManager: Starting task 118.0 in stage 10.0 (TID 135) (10.0.0.133, executor driver, partition 118, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:24 INFO Executor: Running task 118.0 in stage 10.0 (TID 135) -26/04/01 08:38:24 INFO TaskSetManager: Finished task 110.0 in stage 10.0 (TID 127) in 1097 ms on 10.0.0.133 (executor driver) (111/208) -26/04/01 08:38:24 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:24 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:25 INFO Executor: Finished task 112.0 in stage 10.0 (TID 129). 2251 bytes result sent to driver -26/04/01 08:38:25 INFO TaskSetManager: Starting task 119.0 in stage 10.0 (TID 136) (10.0.0.133, executor driver, partition 119, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:25 INFO Executor: Finished task 111.0 in stage 10.0 (TID 128). 2251 bytes result sent to driver -26/04/01 08:38:25 INFO Executor: Running task 119.0 in stage 10.0 (TID 136) -26/04/01 08:38:25 INFO TaskSetManager: Starting task 120.0 in stage 10.0 (TID 137) (10.0.0.133, executor driver, partition 120, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:25 INFO Executor: Running task 120.0 in stage 10.0 (TID 137) -26/04/01 08:38:25 INFO TaskSetManager: Finished task 112.0 in stage 10.0 (TID 129) in 1607 ms on 10.0.0.133 (executor driver) (112/208) -26/04/01 08:38:25 INFO TaskSetManager: Finished task 111.0 in stage 10.0 (TID 128) in 1613 ms on 10.0.0.133 (executor driver) (113/208) -26/04/01 08:38:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:25 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:25 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:25 INFO Executor: Finished task 114.0 in stage 10.0 (TID 131). 2251 bytes result sent to driver -26/04/01 08:38:25 INFO TaskSetManager: Starting task 121.0 in stage 10.0 (TID 138) (10.0.0.133, executor driver, partition 121, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:25 INFO TaskSetManager: Finished task 114.0 in stage 10.0 (TID 131) in 1511 ms on 10.0.0.133 (executor driver) (114/208) -26/04/01 08:38:25 INFO Executor: Running task 121.0 in stage 10.0 (TID 138) -26/04/01 08:38:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:25 INFO Executor: Finished task 113.0 in stage 10.0 (TID 130). 2251 bytes result sent to driver -26/04/01 08:38:25 INFO TaskSetManager: Starting task 122.0 in stage 10.0 (TID 139) (10.0.0.133, executor driver, partition 122, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:25 INFO TaskSetManager: Finished task 113.0 in stage 10.0 (TID 130) in 1519 ms on 10.0.0.133 (executor driver) (115/208) -26/04/01 08:38:25 INFO Executor: Running task 122.0 in stage 10.0 (TID 139) -26/04/01 08:38:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:25 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:25 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:25 INFO Executor: Finished task 115.0 in stage 10.0 (TID 132). 2251 bytes result sent to driver -26/04/01 08:38:25 INFO TaskSetManager: Starting task 123.0 in stage 10.0 (TID 140) (10.0.0.133, executor driver, partition 123, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:25 INFO TaskSetManager: Finished task 115.0 in stage 10.0 (TID 132) in 1454 ms on 10.0.0.133 (executor driver) (116/208) -26/04/01 08:38:25 INFO Executor: Running task 123.0 in stage 10.0 (TID 140) -26/04/01 08:38:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:25 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:25 INFO BlockManagerInfo: Removed broadcast_10_piece0 on 10.0.0.133:59288 in memory (size: 8.1 KiB, free: 8.6 GiB) -26/04/01 08:38:25 INFO BlockManagerInfo: Removed broadcast_11_piece0 on 10.0.0.133:59288 in memory (size: 6.5 KiB, free: 8.6 GiB) -26/04/01 08:38:25 INFO Executor: Finished task 116.0 in stage 10.0 (TID 133). 2251 bytes result sent to driver -26/04/01 08:38:25 INFO TaskSetManager: Starting task 124.0 in stage 10.0 (TID 141) (10.0.0.133, executor driver, partition 124, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:25 INFO TaskSetManager: Finished task 116.0 in stage 10.0 (TID 133) in 1471 ms on 10.0.0.133 (executor driver) (117/208) -26/04/01 08:38:25 INFO Executor: Running task 124.0 in stage 10.0 (TID 141) -26/04/01 08:38:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:25 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:25 INFO Executor: Finished task 117.0 in stage 10.0 (TID 134). 2251 bytes result sent to driver -26/04/01 08:38:25 INFO TaskSetManager: Starting task 125.0 in stage 10.0 (TID 142) (10.0.0.133, executor driver, partition 125, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:25 INFO TaskSetManager: Finished task 117.0 in stage 10.0 (TID 134) in 1453 ms on 10.0.0.133 (executor driver) (118/208) -26/04/01 08:38:25 INFO Executor: Running task 125.0 in stage 10.0 (TID 142) -26/04/01 08:38:25 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:25 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:26 INFO Executor: Finished task 118.0 in stage 10.0 (TID 135). 2251 bytes result sent to driver -26/04/01 08:38:26 INFO TaskSetManager: Starting task 126.0 in stage 10.0 (TID 143) (10.0.0.133, executor driver, partition 126, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:26 INFO TaskSetManager: Finished task 118.0 in stage 10.0 (TID 135) in 1446 ms on 10.0.0.133 (executor driver) (119/208) -26/04/01 08:38:26 INFO Executor: Running task 126.0 in stage 10.0 (TID 143) -26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:26 INFO Executor: Finished task 119.0 in stage 10.0 (TID 136). 2251 bytes result sent to driver -26/04/01 08:38:26 INFO TaskSetManager: Starting task 127.0 in stage 10.0 (TID 144) (10.0.0.133, executor driver, partition 127, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:26 INFO TaskSetManager: Finished task 119.0 in stage 10.0 (TID 136) in 1063 ms on 10.0.0.133 (executor driver) (120/208) -26/04/01 08:38:26 INFO Executor: Running task 127.0 in stage 10.0 (TID 144) -26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:26 INFO Executor: Finished task 120.0 in stage 10.0 (TID 137). 2251 bytes result sent to driver -26/04/01 08:38:26 INFO TaskSetManager: Starting task 128.0 in stage 10.0 (TID 145) (10.0.0.133, executor driver, partition 128, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:26 INFO TaskSetManager: Finished task 120.0 in stage 10.0 (TID 137) in 1098 ms on 10.0.0.133 (executor driver) (121/208) -26/04/01 08:38:26 INFO Executor: Running task 128.0 in stage 10.0 (TID 145) -26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:26 INFO Executor: Finished task 122.0 in stage 10.0 (TID 139). 2251 bytes result sent to driver -26/04/01 08:38:26 INFO TaskSetManager: Starting task 129.0 in stage 10.0 (TID 146) (10.0.0.133, executor driver, partition 129, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:26 INFO Executor: Running task 129.0 in stage 10.0 (TID 146) -26/04/01 08:38:26 INFO TaskSetManager: Finished task 122.0 in stage 10.0 (TID 139) in 1110 ms on 10.0.0.133 (executor driver) (122/208) -26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:26 INFO Executor: Finished task 121.0 in stage 10.0 (TID 138). 2251 bytes result sent to driver -26/04/01 08:38:26 INFO TaskSetManager: Starting task 130.0 in stage 10.0 (TID 147) (10.0.0.133, executor driver, partition 130, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:26 INFO TaskSetManager: Finished task 121.0 in stage 10.0 (TID 138) in 1114 ms on 10.0.0.133 (executor driver) (123/208) -26/04/01 08:38:26 INFO Executor: Running task 130.0 in stage 10.0 (TID 147) -26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:26 INFO Executor: Finished task 123.0 in stage 10.0 (TID 140). 2251 bytes result sent to driver -26/04/01 08:38:26 INFO TaskSetManager: Starting task 131.0 in stage 10.0 (TID 148) (10.0.0.133, executor driver, partition 131, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:26 INFO TaskSetManager: Finished task 123.0 in stage 10.0 (TID 140) in 1075 ms on 10.0.0.133 (executor driver) (124/208) -26/04/01 08:38:26 INFO Executor: Running task 131.0 in stage 10.0 (TID 148) -26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:26 INFO Executor: Finished task 124.0 in stage 10.0 (TID 141). 2251 bytes result sent to driver -26/04/01 08:38:26 INFO TaskSetManager: Starting task 132.0 in stage 10.0 (TID 149) (10.0.0.133, executor driver, partition 132, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:26 INFO TaskSetManager: Finished task 124.0 in stage 10.0 (TID 141) in 1028 ms on 10.0.0.133 (executor driver) (125/208) -26/04/01 08:38:26 INFO Executor: Running task 132.0 in stage 10.0 (TID 149) -26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:26 INFO Executor: Finished task 125.0 in stage 10.0 (TID 142). 2294 bytes result sent to driver -26/04/01 08:38:26 INFO TaskSetManager: Starting task 133.0 in stage 10.0 (TID 150) (10.0.0.133, executor driver, partition 133, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:26 INFO TaskSetManager: Finished task 125.0 in stage 10.0 (TID 142) in 1036 ms on 10.0.0.133 (executor driver) (126/208) -26/04/01 08:38:26 INFO Executor: Running task 133.0 in stage 10.0 (TID 150) -26/04/01 08:38:26 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:26 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:27 INFO Executor: Finished task 126.0 in stage 10.0 (TID 143). 2251 bytes result sent to driver -26/04/01 08:38:27 INFO TaskSetManager: Starting task 134.0 in stage 10.0 (TID 151) (10.0.0.133, executor driver, partition 134, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:27 INFO TaskSetManager: Finished task 126.0 in stage 10.0 (TID 143) in 1035 ms on 10.0.0.133 (executor driver) (127/208) -26/04/01 08:38:27 INFO Executor: Running task 134.0 in stage 10.0 (TID 151) -26/04/01 08:38:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:27 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:27 INFO Executor: Finished task 127.0 in stage 10.0 (TID 144). 2251 bytes result sent to driver -26/04/01 08:38:27 INFO TaskSetManager: Starting task 135.0 in stage 10.0 (TID 152) (10.0.0.133, executor driver, partition 135, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:27 INFO TaskSetManager: Finished task 127.0 in stage 10.0 (TID 144) in 1010 ms on 10.0.0.133 (executor driver) (128/208) -26/04/01 08:38:27 INFO Executor: Running task 135.0 in stage 10.0 (TID 152) -26/04/01 08:38:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:27 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:27 INFO Executor: Finished task 128.0 in stage 10.0 (TID 145). 2251 bytes result sent to driver -26/04/01 08:38:27 INFO TaskSetManager: Starting task 136.0 in stage 10.0 (TID 153) (10.0.0.133, executor driver, partition 136, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:27 INFO TaskSetManager: Finished task 128.0 in stage 10.0 (TID 145) in 1130 ms on 10.0.0.133 (executor driver) (129/208) -26/04/01 08:38:27 INFO Executor: Running task 136.0 in stage 10.0 (TID 153) -26/04/01 08:38:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:27 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:27 INFO Executor: Finished task 129.0 in stage 10.0 (TID 146). 2251 bytes result sent to driver -26/04/01 08:38:27 INFO TaskSetManager: Starting task 137.0 in stage 10.0 (TID 154) (10.0.0.133, executor driver, partition 137, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:27 INFO TaskSetManager: Finished task 129.0 in stage 10.0 (TID 146) in 1110 ms on 10.0.0.133 (executor driver) (130/208) -26/04/01 08:38:27 INFO Executor: Running task 137.0 in stage 10.0 (TID 154) -26/04/01 08:38:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:27 INFO Executor: Finished task 130.0 in stage 10.0 (TID 147). 2251 bytes result sent to driver -26/04/01 08:38:27 INFO TaskSetManager: Starting task 138.0 in stage 10.0 (TID 155) (10.0.0.133, executor driver, partition 138, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:27 INFO TaskSetManager: Finished task 130.0 in stage 10.0 (TID 147) in 1110 ms on 10.0.0.133 (executor driver) (131/208) -26/04/01 08:38:27 INFO Executor: Running task 138.0 in stage 10.0 (TID 155) -26/04/01 08:38:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:27 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:27 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:27 INFO Executor: Finished task 131.0 in stage 10.0 (TID 148). 2251 bytes result sent to driver -26/04/01 08:38:27 INFO TaskSetManager: Starting task 139.0 in stage 10.0 (TID 156) (10.0.0.133, executor driver, partition 139, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:27 INFO TaskSetManager: Finished task 131.0 in stage 10.0 (TID 148) in 1096 ms on 10.0.0.133 (executor driver) (132/208) -26/04/01 08:38:27 INFO Executor: Running task 139.0 in stage 10.0 (TID 156) -26/04/01 08:38:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:27 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:27 INFO Executor: Finished task 132.0 in stage 10.0 (TID 149). 2251 bytes result sent to driver -26/04/01 08:38:27 INFO TaskSetManager: Starting task 140.0 in stage 10.0 (TID 157) (10.0.0.133, executor driver, partition 140, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:27 INFO Executor: Running task 140.0 in stage 10.0 (TID 157) -26/04/01 08:38:27 INFO TaskSetManager: Finished task 132.0 in stage 10.0 (TID 149) in 1123 ms on 10.0.0.133 (executor driver) (133/208) -26/04/01 08:38:27 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:27 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:28 INFO Executor: Finished task 133.0 in stage 10.0 (TID 150). 2251 bytes result sent to driver -26/04/01 08:38:28 INFO TaskSetManager: Starting task 141.0 in stage 10.0 (TID 158) (10.0.0.133, executor driver, partition 141, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:28 INFO TaskSetManager: Finished task 133.0 in stage 10.0 (TID 150) in 1140 ms on 10.0.0.133 (executor driver) (134/208) -26/04/01 08:38:28 INFO Executor: Running task 141.0 in stage 10.0 (TID 158) -26/04/01 08:38:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:28 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:28 INFO Executor: Finished task 134.0 in stage 10.0 (TID 151). 2251 bytes result sent to driver -26/04/01 08:38:28 INFO TaskSetManager: Starting task 142.0 in stage 10.0 (TID 159) (10.0.0.133, executor driver, partition 142, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:28 INFO Executor: Running task 142.0 in stage 10.0 (TID 159) -26/04/01 08:38:28 INFO TaskSetManager: Finished task 134.0 in stage 10.0 (TID 151) in 1139 ms on 10.0.0.133 (executor driver) (135/208) -26/04/01 08:38:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:28 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:28 INFO Executor: Finished task 135.0 in stage 10.0 (TID 152). 2251 bytes result sent to driver -26/04/01 08:38:28 INFO TaskSetManager: Starting task 143.0 in stage 10.0 (TID 160) (10.0.0.133, executor driver, partition 143, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:28 INFO TaskSetManager: Finished task 135.0 in stage 10.0 (TID 152) in 1120 ms on 10.0.0.133 (executor driver) (136/208) -26/04/01 08:38:28 INFO Executor: Running task 143.0 in stage 10.0 (TID 160) -26/04/01 08:38:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:28 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:28 INFO Executor: Finished task 138.0 in stage 10.0 (TID 155). 2251 bytes result sent to driver -26/04/01 08:38:28 INFO TaskSetManager: Starting task 144.0 in stage 10.0 (TID 161) (10.0.0.133, executor driver, partition 144, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:28 INFO TaskSetManager: Finished task 138.0 in stage 10.0 (TID 155) in 1165 ms on 10.0.0.133 (executor driver) (137/208) -26/04/01 08:38:28 INFO Executor: Running task 144.0 in stage 10.0 (TID 161) -26/04/01 08:38:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:28 INFO Executor: Finished task 137.0 in stage 10.0 (TID 154). 2251 bytes result sent to driver -26/04/01 08:38:28 INFO TaskSetManager: Starting task 145.0 in stage 10.0 (TID 162) (10.0.0.133, executor driver, partition 145, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:28 INFO TaskSetManager: Finished task 137.0 in stage 10.0 (TID 154) in 1169 ms on 10.0.0.133 (executor driver) (138/208) -26/04/01 08:38:28 INFO Executor: Running task 145.0 in stage 10.0 (TID 162) -26/04/01 08:38:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:28 INFO Executor: Finished task 136.0 in stage 10.0 (TID 153). 2251 bytes result sent to driver -26/04/01 08:38:28 INFO TaskSetManager: Starting task 146.0 in stage 10.0 (TID 163) (10.0.0.133, executor driver, partition 146, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:28 INFO TaskSetManager: Finished task 136.0 in stage 10.0 (TID 153) in 1219 ms on 10.0.0.133 (executor driver) (139/208) -26/04/01 08:38:28 INFO Executor: Running task 146.0 in stage 10.0 (TID 163) -26/04/01 08:38:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:28 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:28 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:28 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:28 INFO Executor: Finished task 139.0 in stage 10.0 (TID 156). 2251 bytes result sent to driver -26/04/01 08:38:28 INFO TaskSetManager: Starting task 147.0 in stage 10.0 (TID 164) (10.0.0.133, executor driver, partition 147, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:28 INFO TaskSetManager: Finished task 139.0 in stage 10.0 (TID 156) in 1107 ms on 10.0.0.133 (executor driver) (140/208) -26/04/01 08:38:28 INFO Executor: Running task 147.0 in stage 10.0 (TID 164) -26/04/01 08:38:28 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:28 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:29 INFO Executor: Finished task 140.0 in stage 10.0 (TID 157). 2251 bytes result sent to driver -26/04/01 08:38:29 INFO TaskSetManager: Starting task 148.0 in stage 10.0 (TID 165) (10.0.0.133, executor driver, partition 148, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:29 INFO TaskSetManager: Finished task 140.0 in stage 10.0 (TID 157) in 1133 ms on 10.0.0.133 (executor driver) (141/208) -26/04/01 08:38:29 INFO Executor: Running task 148.0 in stage 10.0 (TID 165) -26/04/01 08:38:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:29 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:29 INFO Executor: Finished task 141.0 in stage 10.0 (TID 158). 2251 bytes result sent to driver -26/04/01 08:38:29 INFO TaskSetManager: Starting task 149.0 in stage 10.0 (TID 166) (10.0.0.133, executor driver, partition 149, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:29 INFO Executor: Running task 149.0 in stage 10.0 (TID 166) -26/04/01 08:38:29 INFO TaskSetManager: Finished task 141.0 in stage 10.0 (TID 158) in 1123 ms on 10.0.0.133 (executor driver) (142/208) -26/04/01 08:38:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:29 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:29 INFO Executor: Finished task 142.0 in stage 10.0 (TID 159). 2251 bytes result sent to driver -26/04/01 08:38:29 INFO TaskSetManager: Starting task 150.0 in stage 10.0 (TID 167) (10.0.0.133, executor driver, partition 150, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:29 INFO Executor: Running task 150.0 in stage 10.0 (TID 167) -26/04/01 08:38:29 INFO TaskSetManager: Finished task 142.0 in stage 10.0 (TID 159) in 1109 ms on 10.0.0.133 (executor driver) (143/208) -26/04/01 08:38:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:29 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:29 INFO Executor: Finished task 143.0 in stage 10.0 (TID 160). 2251 bytes result sent to driver -26/04/01 08:38:29 INFO TaskSetManager: Starting task 151.0 in stage 10.0 (TID 168) (10.0.0.133, executor driver, partition 151, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:29 INFO TaskSetManager: Finished task 143.0 in stage 10.0 (TID 160) in 1099 ms on 10.0.0.133 (executor driver) (144/208) -26/04/01 08:38:29 INFO Executor: Running task 151.0 in stage 10.0 (TID 168) -26/04/01 08:38:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:29 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:29 INFO Executor: Finished task 146.0 in stage 10.0 (TID 163). 2251 bytes result sent to driver -26/04/01 08:38:29 INFO TaskSetManager: Starting task 152.0 in stage 10.0 (TID 169) (10.0.0.133, executor driver, partition 152, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:29 INFO TaskSetManager: Finished task 146.0 in stage 10.0 (TID 163) in 1159 ms on 10.0.0.133 (executor driver) (145/208) -26/04/01 08:38:29 INFO Executor: Running task 152.0 in stage 10.0 (TID 169) -26/04/01 08:38:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:29 INFO Executor: Finished task 145.0 in stage 10.0 (TID 162). 2251 bytes result sent to driver -26/04/01 08:38:29 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:29 INFO TaskSetManager: Starting task 153.0 in stage 10.0 (TID 170) (10.0.0.133, executor driver, partition 153, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:29 INFO TaskSetManager: Finished task 145.0 in stage 10.0 (TID 162) in 1179 ms on 10.0.0.133 (executor driver) (146/208) -26/04/01 08:38:29 INFO Executor: Running task 153.0 in stage 10.0 (TID 170) -26/04/01 08:38:29 INFO Executor: Finished task 144.0 in stage 10.0 (TID 161). 2251 bytes result sent to driver -26/04/01 08:38:29 INFO TaskSetManager: Starting task 154.0 in stage 10.0 (TID 171) (10.0.0.133, executor driver, partition 154, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:29 INFO TaskSetManager: Finished task 144.0 in stage 10.0 (TID 161) in 1190 ms on 10.0.0.133 (executor driver) (147/208) -26/04/01 08:38:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:29 INFO Executor: Running task 154.0 in stage 10.0 (TID 171) -26/04/01 08:38:29 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:29 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:29 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:30 INFO Executor: Finished task 147.0 in stage 10.0 (TID 164). 2251 bytes result sent to driver -26/04/01 08:38:30 INFO TaskSetManager: Starting task 155.0 in stage 10.0 (TID 172) (10.0.0.133, executor driver, partition 155, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:30 INFO TaskSetManager: Finished task 147.0 in stage 10.0 (TID 164) in 1297 ms on 10.0.0.133 (executor driver) (148/208) -26/04/01 08:38:30 INFO Executor: Running task 155.0 in stage 10.0 (TID 172) -26/04/01 08:38:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:30 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:30 INFO Executor: Finished task 148.0 in stage 10.0 (TID 165). 2251 bytes result sent to driver -26/04/01 08:38:30 INFO TaskSetManager: Starting task 156.0 in stage 10.0 (TID 173) (10.0.0.133, executor driver, partition 156, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:30 INFO TaskSetManager: Finished task 148.0 in stage 10.0 (TID 165) in 1225 ms on 10.0.0.133 (executor driver) (149/208) -26/04/01 08:38:30 INFO Executor: Running task 156.0 in stage 10.0 (TID 173) -26/04/01 08:38:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:30 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:30 INFO Executor: Finished task 149.0 in stage 10.0 (TID 166). 2251 bytes result sent to driver -26/04/01 08:38:30 INFO TaskSetManager: Starting task 157.0 in stage 10.0 (TID 174) (10.0.0.133, executor driver, partition 157, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:30 INFO TaskSetManager: Finished task 149.0 in stage 10.0 (TID 166) in 1176 ms on 10.0.0.133 (executor driver) (150/208) -26/04/01 08:38:30 INFO Executor: Running task 157.0 in stage 10.0 (TID 174) -26/04/01 08:38:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:30 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:30 INFO Executor: Finished task 150.0 in stage 10.0 (TID 167). 2251 bytes result sent to driver -26/04/01 08:38:30 INFO TaskSetManager: Starting task 158.0 in stage 10.0 (TID 175) (10.0.0.133, executor driver, partition 158, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:30 INFO Executor: Running task 158.0 in stage 10.0 (TID 175) -26/04/01 08:38:30 INFO TaskSetManager: Finished task 150.0 in stage 10.0 (TID 167) in 1184 ms on 10.0.0.133 (executor driver) (151/208) -26/04/01 08:38:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:30 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:30 INFO Executor: Finished task 151.0 in stage 10.0 (TID 168). 2251 bytes result sent to driver -26/04/01 08:38:30 INFO TaskSetManager: Starting task 159.0 in stage 10.0 (TID 176) (10.0.0.133, executor driver, partition 159, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:30 INFO TaskSetManager: Finished task 151.0 in stage 10.0 (TID 168) in 1175 ms on 10.0.0.133 (executor driver) (152/208) -26/04/01 08:38:30 INFO Executor: Running task 159.0 in stage 10.0 (TID 176) -26/04/01 08:38:30 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:30 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:31 INFO Executor: Finished task 153.0 in stage 10.0 (TID 170). 2294 bytes result sent to driver -26/04/01 08:38:31 INFO TaskSetManager: Starting task 160.0 in stage 10.0 (TID 177) (10.0.0.133, executor driver, partition 160, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:31 INFO TaskSetManager: Finished task 153.0 in stage 10.0 (TID 170) in 1364 ms on 10.0.0.133 (executor driver) (153/208) -26/04/01 08:38:31 INFO Executor: Running task 160.0 in stage 10.0 (TID 177) -26/04/01 08:38:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:31 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:31 INFO Executor: Finished task 154.0 in stage 10.0 (TID 171). 2294 bytes result sent to driver -26/04/01 08:38:31 INFO TaskSetManager: Starting task 161.0 in stage 10.0 (TID 178) (10.0.0.133, executor driver, partition 161, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:31 INFO TaskSetManager: Finished task 154.0 in stage 10.0 (TID 171) in 1385 ms on 10.0.0.133 (executor driver) (154/208) -26/04/01 08:38:31 INFO Executor: Running task 161.0 in stage 10.0 (TID 178) -26/04/01 08:38:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:31 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:31 INFO Executor: Finished task 152.0 in stage 10.0 (TID 169). 2251 bytes result sent to driver -26/04/01 08:38:31 INFO TaskSetManager: Starting task 162.0 in stage 10.0 (TID 179) (10.0.0.133, executor driver, partition 162, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:31 INFO Executor: Running task 162.0 in stage 10.0 (TID 179) -26/04/01 08:38:31 INFO TaskSetManager: Finished task 152.0 in stage 10.0 (TID 169) in 1429 ms on 10.0.0.133 (executor driver) (155/208) -26/04/01 08:38:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:31 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:31 INFO Executor: Finished task 155.0 in stage 10.0 (TID 172). 2251 bytes result sent to driver -26/04/01 08:38:31 INFO TaskSetManager: Starting task 163.0 in stage 10.0 (TID 180) (10.0.0.133, executor driver, partition 163, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:31 INFO TaskSetManager: Finished task 155.0 in stage 10.0 (TID 172) in 1394 ms on 10.0.0.133 (executor driver) (156/208) -26/04/01 08:38:31 INFO Executor: Running task 163.0 in stage 10.0 (TID 180) -26/04/01 08:38:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:31 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:31 INFO Executor: Finished task 157.0 in stage 10.0 (TID 174). 2251 bytes result sent to driver -26/04/01 08:38:31 INFO TaskSetManager: Starting task 164.0 in stage 10.0 (TID 181) (10.0.0.133, executor driver, partition 164, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:31 INFO Executor: Running task 164.0 in stage 10.0 (TID 181) -26/04/01 08:38:31 INFO TaskSetManager: Finished task 157.0 in stage 10.0 (TID 174) in 1456 ms on 10.0.0.133 (executor driver) (157/208) -26/04/01 08:38:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:31 INFO Executor: Finished task 156.0 in stage 10.0 (TID 173). 2251 bytes result sent to driver -26/04/01 08:38:31 INFO TaskSetManager: Starting task 165.0 in stage 10.0 (TID 182) (10.0.0.133, executor driver, partition 165, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:31 INFO TaskSetManager: Finished task 156.0 in stage 10.0 (TID 173) in 1486 ms on 10.0.0.133 (executor driver) (158/208) -26/04/01 08:38:31 INFO Executor: Running task 165.0 in stage 10.0 (TID 182) -26/04/01 08:38:31 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:31 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:31 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:32 INFO Executor: Finished task 158.0 in stage 10.0 (TID 175). 2251 bytes result sent to driver -26/04/01 08:38:32 INFO TaskSetManager: Starting task 166.0 in stage 10.0 (TID 183) (10.0.0.133, executor driver, partition 166, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:32 INFO TaskSetManager: Finished task 158.0 in stage 10.0 (TID 175) in 1505 ms on 10.0.0.133 (executor driver) (159/208) -26/04/01 08:38:32 INFO Executor: Running task 166.0 in stage 10.0 (TID 183) -26/04/01 08:38:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:32 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:32 INFO Executor: Finished task 159.0 in stage 10.0 (TID 176). 2251 bytes result sent to driver -26/04/01 08:38:32 INFO TaskSetManager: Starting task 167.0 in stage 10.0 (TID 184) (10.0.0.133, executor driver, partition 167, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:32 INFO Executor: Running task 167.0 in stage 10.0 (TID 184) -26/04/01 08:38:32 INFO TaskSetManager: Finished task 159.0 in stage 10.0 (TID 176) in 1562 ms on 10.0.0.133 (executor driver) (160/208) -26/04/01 08:38:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:32 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:32 INFO Executor: Finished task 160.0 in stage 10.0 (TID 177). 2251 bytes result sent to driver -26/04/01 08:38:32 INFO TaskSetManager: Starting task 168.0 in stage 10.0 (TID 185) (10.0.0.133, executor driver, partition 168, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:32 INFO TaskSetManager: Finished task 160.0 in stage 10.0 (TID 177) in 1662 ms on 10.0.0.133 (executor driver) (161/208) -26/04/01 08:38:32 INFO Executor: Running task 168.0 in stage 10.0 (TID 185) -26/04/01 08:38:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:32 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:32 INFO Executor: Finished task 161.0 in stage 10.0 (TID 178). 2251 bytes result sent to driver -26/04/01 08:38:32 INFO TaskSetManager: Starting task 169.0 in stage 10.0 (TID 186) (10.0.0.133, executor driver, partition 169, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:32 INFO TaskSetManager: Finished task 161.0 in stage 10.0 (TID 178) in 1674 ms on 10.0.0.133 (executor driver) (162/208) -26/04/01 08:38:32 INFO Executor: Running task 169.0 in stage 10.0 (TID 186) -26/04/01 08:38:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:32 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:32 INFO Executor: Finished task 162.0 in stage 10.0 (TID 179). 2251 bytes result sent to driver -26/04/01 08:38:32 INFO TaskSetManager: Starting task 170.0 in stage 10.0 (TID 187) (10.0.0.133, executor driver, partition 170, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:32 INFO TaskSetManager: Finished task 162.0 in stage 10.0 (TID 179) in 1668 ms on 10.0.0.133 (executor driver) (163/208) -26/04/01 08:38:32 INFO Executor: Running task 170.0 in stage 10.0 (TID 187) -26/04/01 08:38:32 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:32 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:33 INFO Executor: Finished task 163.0 in stage 10.0 (TID 180). 2251 bytes result sent to driver -26/04/01 08:38:33 INFO TaskSetManager: Starting task 171.0 in stage 10.0 (TID 188) (10.0.0.133, executor driver, partition 171, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:33 INFO TaskSetManager: Finished task 163.0 in stage 10.0 (TID 180) in 1664 ms on 10.0.0.133 (executor driver) (164/208) -26/04/01 08:38:33 INFO Executor: Running task 171.0 in stage 10.0 (TID 188) -26/04/01 08:38:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:33 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:33 INFO Executor: Finished task 165.0 in stage 10.0 (TID 182). 2251 bytes result sent to driver -26/04/01 08:38:33 INFO TaskSetManager: Starting task 172.0 in stage 10.0 (TID 189) (10.0.0.133, executor driver, partition 172, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:33 INFO TaskSetManager: Finished task 165.0 in stage 10.0 (TID 182) in 1737 ms on 10.0.0.133 (executor driver) (165/208) -26/04/01 08:38:33 INFO Executor: Running task 172.0 in stage 10.0 (TID 189) -26/04/01 08:38:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:33 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:33 INFO Executor: Finished task 164.0 in stage 10.0 (TID 181). 2251 bytes result sent to driver -26/04/01 08:38:33 INFO TaskSetManager: Starting task 173.0 in stage 10.0 (TID 190) (10.0.0.133, executor driver, partition 173, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:33 INFO TaskSetManager: Finished task 164.0 in stage 10.0 (TID 181) in 1748 ms on 10.0.0.133 (executor driver) (166/208) -26/04/01 08:38:33 INFO Executor: Running task 173.0 in stage 10.0 (TID 190) -26/04/01 08:38:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:33 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:33 INFO Executor: Finished task 166.0 in stage 10.0 (TID 183). 2251 bytes result sent to driver -26/04/01 08:38:33 INFO TaskSetManager: Starting task 174.0 in stage 10.0 (TID 191) (10.0.0.133, executor driver, partition 174, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:33 INFO TaskSetManager: Finished task 166.0 in stage 10.0 (TID 183) in 1673 ms on 10.0.0.133 (executor driver) (167/208) -26/04/01 08:38:33 INFO Executor: Running task 174.0 in stage 10.0 (TID 191) -26/04/01 08:38:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:33 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:33 INFO Executor: Finished task 167.0 in stage 10.0 (TID 184). 2251 bytes result sent to driver -26/04/01 08:38:33 INFO TaskSetManager: Starting task 175.0 in stage 10.0 (TID 192) (10.0.0.133, executor driver, partition 175, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:33 INFO TaskSetManager: Finished task 167.0 in stage 10.0 (TID 184) in 1726 ms on 10.0.0.133 (executor driver) (168/208) -26/04/01 08:38:33 INFO Executor: Running task 175.0 in stage 10.0 (TID 192) -26/04/01 08:38:33 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:33 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:34 INFO Executor: Finished task 168.0 in stage 10.0 (TID 185). 2251 bytes result sent to driver -26/04/01 08:38:34 INFO TaskSetManager: Starting task 176.0 in stage 10.0 (TID 193) (10.0.0.133, executor driver, partition 176, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:34 INFO TaskSetManager: Finished task 168.0 in stage 10.0 (TID 185) in 1775 ms on 10.0.0.133 (executor driver) (169/208) -26/04/01 08:38:34 INFO Executor: Running task 176.0 in stage 10.0 (TID 193) -26/04/01 08:38:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:34 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:34 INFO Executor: Finished task 170.0 in stage 10.0 (TID 187). 2251 bytes result sent to driver -26/04/01 08:38:34 INFO TaskSetManager: Starting task 177.0 in stage 10.0 (TID 194) (10.0.0.133, executor driver, partition 177, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:34 INFO TaskSetManager: Finished task 170.0 in stage 10.0 (TID 187) in 1777 ms on 10.0.0.133 (executor driver) (170/208) -26/04/01 08:38:34 INFO Executor: Running task 177.0 in stage 10.0 (TID 194) -26/04/01 08:38:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:34 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:34 INFO Executor: Finished task 169.0 in stage 10.0 (TID 186). 2251 bytes result sent to driver -26/04/01 08:38:34 INFO TaskSetManager: Starting task 178.0 in stage 10.0 (TID 195) (10.0.0.133, executor driver, partition 178, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:34 INFO Executor: Running task 178.0 in stage 10.0 (TID 195) -26/04/01 08:38:34 INFO TaskSetManager: Finished task 169.0 in stage 10.0 (TID 186) in 1826 ms on 10.0.0.133 (executor driver) (171/208) -26/04/01 08:38:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:34 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:34 INFO Executor: Finished task 171.0 in stage 10.0 (TID 188). 2251 bytes result sent to driver -26/04/01 08:38:34 INFO TaskSetManager: Starting task 179.0 in stage 10.0 (TID 196) (10.0.0.133, executor driver, partition 179, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:34 INFO Executor: Running task 179.0 in stage 10.0 (TID 196) -26/04/01 08:38:34 INFO TaskSetManager: Finished task 171.0 in stage 10.0 (TID 188) in 1691 ms on 10.0.0.133 (executor driver) (172/208) -26/04/01 08:38:34 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:34 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:35 INFO Executor: Finished task 172.0 in stage 10.0 (TID 189). 2251 bytes result sent to driver -26/04/01 08:38:35 INFO TaskSetManager: Starting task 180.0 in stage 10.0 (TID 197) (10.0.0.133, executor driver, partition 180, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:35 INFO TaskSetManager: Finished task 172.0 in stage 10.0 (TID 189) in 1634 ms on 10.0.0.133 (executor driver) (173/208) -26/04/01 08:38:35 INFO Executor: Running task 180.0 in stage 10.0 (TID 197) -26/04/01 08:38:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:35 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:35 INFO Executor: Finished task 173.0 in stage 10.0 (TID 190). 2294 bytes result sent to driver -26/04/01 08:38:35 INFO TaskSetManager: Starting task 181.0 in stage 10.0 (TID 198) (10.0.0.133, executor driver, partition 181, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:35 INFO TaskSetManager: Finished task 173.0 in stage 10.0 (TID 190) in 1633 ms on 10.0.0.133 (executor driver) (174/208) -26/04/01 08:38:35 INFO Executor: Running task 181.0 in stage 10.0 (TID 198) -26/04/01 08:38:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:35 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:35 INFO Executor: Finished task 174.0 in stage 10.0 (TID 191). 2251 bytes result sent to driver -26/04/01 08:38:35 INFO TaskSetManager: Starting task 182.0 in stage 10.0 (TID 199) (10.0.0.133, executor driver, partition 182, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:35 INFO Executor: Running task 182.0 in stage 10.0 (TID 199) -26/04/01 08:38:35 INFO TaskSetManager: Finished task 174.0 in stage 10.0 (TID 191) in 1604 ms on 10.0.0.133 (executor driver) (175/208) -26/04/01 08:38:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:35 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:35 INFO Executor: Finished task 175.0 in stage 10.0 (TID 192). 2251 bytes result sent to driver -26/04/01 08:38:35 INFO TaskSetManager: Starting task 183.0 in stage 10.0 (TID 200) (10.0.0.133, executor driver, partition 183, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:35 INFO TaskSetManager: Finished task 175.0 in stage 10.0 (TID 192) in 1552 ms on 10.0.0.133 (executor driver) (176/208) -26/04/01 08:38:35 INFO Executor: Running task 183.0 in stage 10.0 (TID 200) -26/04/01 08:38:35 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:35 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:36 INFO Executor: Finished task 176.0 in stage 10.0 (TID 193). 2251 bytes result sent to driver -26/04/01 08:38:36 INFO TaskSetManager: Starting task 184.0 in stage 10.0 (TID 201) (10.0.0.133, executor driver, partition 184, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:36 INFO TaskSetManager: Finished task 176.0 in stage 10.0 (TID 193) in 1582 ms on 10.0.0.133 (executor driver) (177/208) -26/04/01 08:38:36 INFO Executor: Running task 184.0 in stage 10.0 (TID 201) -26/04/01 08:38:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:36 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:36 INFO Executor: Finished task 177.0 in stage 10.0 (TID 194). 2251 bytes result sent to driver -26/04/01 08:38:36 INFO TaskSetManager: Starting task 185.0 in stage 10.0 (TID 202) (10.0.0.133, executor driver, partition 185, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:36 INFO TaskSetManager: Finished task 177.0 in stage 10.0 (TID 194) in 1578 ms on 10.0.0.133 (executor driver) (178/208) -26/04/01 08:38:36 INFO Executor: Running task 185.0 in stage 10.0 (TID 202) -26/04/01 08:38:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:36 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:36 INFO Executor: Finished task 178.0 in stage 10.0 (TID 195). 2251 bytes result sent to driver -26/04/01 08:38:36 INFO TaskSetManager: Starting task 186.0 in stage 10.0 (TID 203) (10.0.0.133, executor driver, partition 186, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:36 INFO TaskSetManager: Finished task 178.0 in stage 10.0 (TID 195) in 1566 ms on 10.0.0.133 (executor driver) (179/208) -26/04/01 08:38:36 INFO Executor: Running task 186.0 in stage 10.0 (TID 203) -26/04/01 08:38:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:36 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:36 INFO Executor: Finished task 179.0 in stage 10.0 (TID 196). 2251 bytes result sent to driver -26/04/01 08:38:36 INFO TaskSetManager: Starting task 187.0 in stage 10.0 (TID 204) (10.0.0.133, executor driver, partition 187, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:36 INFO Executor: Running task 187.0 in stage 10.0 (TID 204) -26/04/01 08:38:36 INFO TaskSetManager: Finished task 179.0 in stage 10.0 (TID 196) in 1557 ms on 10.0.0.133 (executor driver) (180/208) -26/04/01 08:38:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 134217728-268435456, partition values: [empty row] -26/04/01 08:38:36 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:36 INFO Executor: Finished task 181.0 in stage 10.0 (TID 198). 2251 bytes result sent to driver -26/04/01 08:38:36 INFO TaskSetManager: Starting task 188.0 in stage 10.0 (TID 205) (10.0.0.133, executor driver, partition 188, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:36 INFO TaskSetManager: Finished task 181.0 in stage 10.0 (TID 198) in 1664 ms on 10.0.0.133 (executor driver) (181/208) -26/04/01 08:38:36 INFO Executor: Running task 188.0 in stage 10.0 (TID 205) -26/04/01 08:38:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 268435456-402653184, partition values: [empty row] -26/04/01 08:38:36 INFO Executor: Finished task 180.0 in stage 10.0 (TID 197). 2251 bytes result sent to driver -26/04/01 08:38:36 INFO TaskSetManager: Starting task 189.0 in stage 10.0 (TID 206) (10.0.0.133, executor driver, partition 189, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:36 INFO Executor: Running task 189.0 in stage 10.0 (TID 206) -26/04/01 08:38:36 INFO TaskSetManager: Finished task 180.0 in stage 10.0 (TID 197) in 1676 ms on 10.0.0.133 (executor driver) (182/208) -26/04/01 08:38:36 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 402653184-536870912, partition values: [empty row] -26/04/01 08:38:36 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:36 INFO Executor: Finished task 182.0 in stage 10.0 (TID 199). 2251 bytes result sent to driver -26/04/01 08:38:36 INFO TaskSetManager: Starting task 190.0 in stage 10.0 (TID 207) (10.0.0.133, executor driver, partition 190, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:36 INFO TaskSetManager: Finished task 182.0 in stage 10.0 (TID 199) in 1599 ms on 10.0.0.133 (executor driver) (183/208) -26/04/01 08:38:36 INFO Executor: Running task 190.0 in stage 10.0 (TID 207) -26/04/01 08:38:36 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 536870912-671088640, partition values: [empty row] -26/04/01 08:38:36 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:37 INFO Executor: Finished task 183.0 in stage 10.0 (TID 200). 2251 bytes result sent to driver -26/04/01 08:38:37 INFO TaskSetManager: Starting task 191.0 in stage 10.0 (TID 208) (10.0.0.133, executor driver, partition 191, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:37 INFO Executor: Running task 191.0 in stage 10.0 (TID 208) -26/04/01 08:38:37 INFO TaskSetManager: Finished task 183.0 in stage 10.0 (TID 200) in 1645 ms on 10.0.0.133 (executor driver) (184/208) -26/04/01 08:38:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 671088640-805306368, partition values: [empty row] -26/04/01 08:38:37 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:37 INFO Executor: Finished task 184.0 in stage 10.0 (TID 201). 2251 bytes result sent to driver -26/04/01 08:38:37 INFO TaskSetManager: Starting task 192.0 in stage 10.0 (TID 209) (10.0.0.133, executor driver, partition 192, PROCESS_LOCAL, 9710 bytes) -26/04/01 08:38:37 INFO TaskSetManager: Finished task 184.0 in stage 10.0 (TID 201) in 1703 ms on 10.0.0.133 (executor driver) (185/208) -26/04/01 08:38:37 INFO Executor: Running task 192.0 in stage 10.0 (TID 209) -26/04/01 08:38:37 INFO Executor: Finished task 185.0 in stage 10.0 (TID 202). 2251 bytes result sent to driver -26/04/01 08:38:37 INFO TaskSetManager: Starting task 193.0 in stage 10.0 (TID 210) (10.0.0.133, executor driver, partition 193, PROCESS_LOCAL, 9709 bytes) -26/04/01 08:38:37 INFO TaskSetManager: Finished task 185.0 in stage 10.0 (TID 202) in 1649 ms on 10.0.0.133 (executor driver) (186/208) -26/04/01 08:38:37 INFO Executor: Running task 193.0 in stage 10.0 (TID 210) -26/04/01 08:38:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.32.parquet, range: 805306368-848649288, partition values: [empty row] -26/04/01 08:38:37 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.26.parquet, range: 805306368-848563442, partition values: [empty row] -26/04/01 08:38:37 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:37 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:38 INFO Executor: Finished task 186.0 in stage 10.0 (TID 203). 2251 bytes result sent to driver -26/04/01 08:38:38 INFO TaskSetManager: Starting task 194.0 in stage 10.0 (TID 211) (10.0.0.133, executor driver, partition 194, PROCESS_LOCAL, 9710 bytes) -26/04/01 08:38:38 INFO Executor: Running task 194.0 in stage 10.0 (TID 211) -26/04/01 08:38:38 INFO TaskSetManager: Finished task 186.0 in stage 10.0 (TID 203) in 1743 ms on 10.0.0.133 (executor driver) (187/208) -26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.20.parquet, range: 805306368-848496433, partition values: [empty row] -26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:38 INFO Executor: Finished task 187.0 in stage 10.0 (TID 204). 2251 bytes result sent to driver -26/04/01 08:38:38 INFO TaskSetManager: Starting task 195.0 in stage 10.0 (TID 212) (10.0.0.133, executor driver, partition 195, PROCESS_LOCAL, 9710 bytes) -26/04/01 08:38:38 INFO TaskSetManager: Finished task 187.0 in stage 10.0 (TID 204) in 1663 ms on 10.0.0.133 (executor driver) (188/208) -26/04/01 08:38:38 INFO Executor: Running task 195.0 in stage 10.0 (TID 212) -26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.12.parquet, range: 805306368-848463796, partition values: [empty row] -26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.5.parquet, range: 805306368-848521656, partition values: [empty row] -26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.21.parquet, range: 805306368-848594494, partition values: [empty row] -26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.31.parquet, range: 805306368-848442878, partition values: [empty row] -26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.11.parquet, range: 805306368-848489219, partition values: [empty row] -26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:38 INFO Executor: Finished task 188.0 in stage 10.0 (TID 205). 2251 bytes result sent to driver -26/04/01 08:38:38 INFO TaskSetManager: Starting task 196.0 in stage 10.0 (TID 213) (10.0.0.133, executor driver, partition 196, PROCESS_LOCAL, 9709 bytes) -26/04/01 08:38:38 INFO TaskSetManager: Finished task 188.0 in stage 10.0 (TID 205) in 1688 ms on 10.0.0.133 (executor driver) (189/208) -26/04/01 08:38:38 INFO Executor: Running task 196.0 in stage 10.0 (TID 213) -26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.19.parquet, range: 805306368-848432523, partition values: [empty row] -26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:38 INFO Executor: Finished task 189.0 in stage 10.0 (TID 206). 2294 bytes result sent to driver -26/04/01 08:38:38 INFO TaskSetManager: Starting task 197.0 in stage 10.0 (TID 214) (10.0.0.133, executor driver, partition 197, PROCESS_LOCAL, 9709 bytes) -26/04/01 08:38:38 INFO TaskSetManager: Finished task 189.0 in stage 10.0 (TID 206) in 1769 ms on 10.0.0.133 (executor driver) (190/208) -26/04/01 08:38:38 INFO Executor: Running task 197.0 in stage 10.0 (TID 214) -26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.7.parquet, range: 805306368-848422348, partition values: [empty row] -26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:38 INFO Executor: Finished task 190.0 in stage 10.0 (TID 207). 2294 bytes result sent to driver -26/04/01 08:38:38 INFO TaskSetManager: Starting task 198.0 in stage 10.0 (TID 215) (10.0.0.133, executor driver, partition 198, PROCESS_LOCAL, 9710 bytes) -26/04/01 08:38:38 INFO TaskSetManager: Finished task 190.0 in stage 10.0 (TID 207) in 1709 ms on 10.0.0.133 (executor driver) (191/208) -26/04/01 08:38:38 INFO Executor: Running task 198.0 in stage 10.0 (TID 215) -26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.17.parquet, range: 805306368-848399844, partition values: [empty row] -26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:38 INFO Executor: Finished task 191.0 in stage 10.0 (TID 208). 2251 bytes result sent to driver -26/04/01 08:38:38 INFO TaskSetManager: Starting task 199.0 in stage 10.0 (TID 216) (10.0.0.133, executor driver, partition 199, PROCESS_LOCAL, 9710 bytes) -26/04/01 08:38:38 INFO TaskSetManager: Finished task 191.0 in stage 10.0 (TID 208) in 1581 ms on 10.0.0.133 (executor driver) (192/208) -26/04/01 08:38:38 INFO Executor: Running task 199.0 in stage 10.0 (TID 216) -26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.10.parquet, range: 805306368-848378208, partition values: [empty row] -26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:38 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.9.parquet, range: 805306368-848423659, partition values: [empty row] -26/04/01 08:38:38 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.29.parquet, range: 805306368-848411809, partition values: [empty row] -26/04/01 08:38:39 INFO Executor: Finished task 192.0 in stage 10.0 (TID 209). 2251 bytes result sent to driver -26/04/01 08:38:39 INFO TaskSetManager: Starting task 200.0 in stage 10.0 (TID 217) (10.0.0.133, executor driver, partition 200, PROCESS_LOCAL, 9708 bytes) -26/04/01 08:38:39 INFO TaskSetManager: Finished task 192.0 in stage 10.0 (TID 209) in 1126 ms on 10.0.0.133 (executor driver) (193/208) -26/04/01 08:38:39 INFO Executor: Running task 200.0 in stage 10.0 (TID 217) -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.2.parquet, range: 805306368-848344114, partition values: [empty row] -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO Executor: Finished task 193.0 in stage 10.0 (TID 210). 2251 bytes result sent to driver -26/04/01 08:38:39 INFO TaskSetManager: Starting task 201.0 in stage 10.0 (TID 218) (10.0.0.133, executor driver, partition 201, PROCESS_LOCAL, 9709 bytes) -26/04/01 08:38:39 INFO TaskSetManager: Finished task 193.0 in stage 10.0 (TID 210) in 1132 ms on 10.0.0.133 (executor driver) (194/208) -26/04/01 08:38:39 INFO Executor: Running task 201.0 in stage 10.0 (TID 218) -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.30.parquet, range: 805306368-848379799, partition values: [empty row] -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.15.parquet, range: 805306368-848329366, partition values: [empty row] -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.22.parquet, range: 805306368-848348606, partition values: [empty row] -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO Executor: Finished task 194.0 in stage 10.0 (TID 211). 2251 bytes result sent to driver -26/04/01 08:38:39 INFO TaskSetManager: Starting task 202.0 in stage 10.0 (TID 219) (10.0.0.133, executor driver, partition 202, PROCESS_LOCAL, 9709 bytes) -26/04/01 08:38:39 INFO Executor: Finished task 195.0 in stage 10.0 (TID 212). 2251 bytes result sent to driver -26/04/01 08:38:39 INFO Executor: Running task 202.0 in stage 10.0 (TID 219) -26/04/01 08:38:39 INFO TaskSetManager: Starting task 203.0 in stage 10.0 (TID 220) (10.0.0.133, executor driver, partition 203, PROCESS_LOCAL, 9710 bytes) -26/04/01 08:38:39 INFO TaskSetManager: Finished task 194.0 in stage 10.0 (TID 211) in 1127 ms on 10.0.0.133 (executor driver) (195/208) -26/04/01 08:38:39 INFO Executor: Running task 203.0 in stage 10.0 (TID 220) -26/04/01 08:38:39 INFO TaskSetManager: Finished task 195.0 in stage 10.0 (TID 212) in 1120 ms on 10.0.0.133 (executor driver) (196/208) -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.23.parquet, range: 805306368-848328570, partition values: [empty row] -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.14.parquet, range: 805306368-848301737, partition values: [empty row] -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.8.parquet, range: 805306368-848338051, partition values: [empty row] -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.4.parquet, range: 805306368-848329327, partition values: [empty row] -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO Executor: Finished task 196.0 in stage 10.0 (TID 213). 2251 bytes result sent to driver -26/04/01 08:38:39 INFO TaskSetManager: Starting task 204.0 in stage 10.0 (TID 221) (10.0.0.133, executor driver, partition 204, PROCESS_LOCAL, 9710 bytes) -26/04/01 08:38:39 INFO TaskSetManager: Finished task 196.0 in stage 10.0 (TID 213) in 1032 ms on 10.0.0.133 (executor driver) (197/208) -26/04/01 08:38:39 INFO Executor: Running task 204.0 in stage 10.0 (TID 221) -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.28.parquet, range: 805306368-848263904, partition values: [empty row] -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.6.parquet, range: 805306368-848306153, partition values: [empty row] -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.24.parquet, range: 805306368-848278425, partition values: [empty row] -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO Executor: Finished task 197.0 in stage 10.0 (TID 214). 2251 bytes result sent to driver -26/04/01 08:38:39 INFO TaskSetManager: Starting task 205.0 in stage 10.0 (TID 222) (10.0.0.133, executor driver, partition 205, PROCESS_LOCAL, 9710 bytes) -26/04/01 08:38:39 INFO TaskSetManager: Finished task 197.0 in stage 10.0 (TID 214) in 1086 ms on 10.0.0.133 (executor driver) (198/208) -26/04/01 08:38:39 INFO Executor: Running task 205.0 in stage 10.0 (TID 222) -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.25.parquet, range: 805306368-848235021, partition values: [empty row] -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO Executor: Finished task 198.0 in stage 10.0 (TID 215). 2251 bytes result sent to driver -26/04/01 08:38:39 INFO TaskSetManager: Starting task 206.0 in stage 10.0 (TID 223) (10.0.0.133, executor driver, partition 206, PROCESS_LOCAL, 9709 bytes) -26/04/01 08:38:39 INFO TaskSetManager: Finished task 198.0 in stage 10.0 (TID 215) in 1096 ms on 10.0.0.133 (executor driver) (199/208) -26/04/01 08:38:39 INFO Executor: Running task 206.0 in stage 10.0 (TID 223) -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.1.parquet, range: 805306368-848107384, partition values: [empty row] -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO Executor: Finished task 199.0 in stage 10.0 (TID 216). 2251 bytes result sent to driver -26/04/01 08:38:39 INFO TaskSetManager: Starting task 207.0 in stage 10.0 (TID 224) (10.0.0.133, executor driver, partition 207, PROCESS_LOCAL, 9709 bytes) -26/04/01 08:38:39 INFO TaskSetManager: Finished task 199.0 in stage 10.0 (TID 216) in 1050 ms on 10.0.0.133 (executor driver) (200/208) -26/04/01 08:38:39 INFO Executor: Running task 207.0 in stage 10.0 (TID 224) -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.16.parquet, range: 805306368-848005421, partition values: [empty row] -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:39 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.27.parquet, range: 805306368-848247851, partition values: [empty row] -26/04/01 08:38:39 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.13.parquet, range: 805306368-848234776, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.18.parquet, range: 805306368-848050794, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:40 INFO Executor: Finished task 200.0 in stage 10.0 (TID 217). 2251 bytes result sent to driver -26/04/01 08:38:40 INFO TaskSetManager: Starting task 0.0 in stage 11.0 (TID 225) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9924 bytes) -26/04/01 08:38:40 INFO TaskSetManager: Finished task 200.0 in stage 10.0 (TID 217) in 1195 ms on 10.0.0.133 (executor driver) (201/208) -26/04/01 08:38:40 INFO Executor: Running task 0.0 in stage 11.0 (TID 225) -26/04/01 08:38:40 INFO CodeGenerator: Code generated in 3.62975 ms -26/04/01 08:38:40 INFO CodeGenerator: Code generated in 3.54975 ms -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.9.parquet, range: 0-2801305, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO Executor: Finished task 201.0 in stage 10.0 (TID 218). 2251 bytes result sent to driver -26/04/01 08:38:40 INFO TaskSetManager: Starting task 1.0 in stage 11.0 (TID 226) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9925 bytes) -26/04/01 08:38:40 INFO TaskSetManager: Finished task 201.0 in stage 10.0 (TID 218) in 1220 ms on 10.0.0.133 (executor driver) (202/208) -26/04/01 08:38:40 INFO Executor: Running task 1.0 in stage 11.0 (TID 226) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.31.parquet, range: 0-2799730, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.21.parquet, range: 0-2800748, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/lineitem/lineitem.3.parquet, range: 805306368-847699715, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(and(noteq(l_partkey, null), noteq(l_suppkey, null)), noteq(l_orderkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.30.parquet, range: 0-2799509, partition values: [empty row] -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.13.parquet, range: 0-2800604, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.25.parquet, range: 0-2798499, partition values: [empty row] -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.1.parquet, range: 0-2799999, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO Executor: Finished task 203.0 in stage 10.0 (TID 220). 2251 bytes result sent to driver -26/04/01 08:38:40 INFO TaskSetManager: Starting task 2.0 in stage 11.0 (TID 227) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9925 bytes) -26/04/01 08:38:40 INFO Executor: Running task 2.0 in stage 11.0 (TID 227) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.6.parquet, range: 0-2798414, partition values: [empty row] -26/04/01 08:38:40 INFO TaskSetManager: Finished task 203.0 in stage 10.0 (TID 220) in 1188 ms on 10.0.0.133 (executor driver) (203/208) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.11.parquet, range: 0-2798396, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO Executor: Finished task 202.0 in stage 10.0 (TID 219). 2251 bytes result sent to driver -26/04/01 08:38:40 INFO TaskSetManager: Starting task 3.0 in stage 11.0 (TID 228) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9926 bytes) -26/04/01 08:38:40 INFO TaskSetManager: Finished task 202.0 in stage 10.0 (TID 219) in 1203 ms on 10.0.0.133 (executor driver) (204/208) -26/04/01 08:38:40 INFO Executor: Running task 3.0 in stage 11.0 (TID 228) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.26.parquet, range: 0-2797291, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.18.parquet, range: 0-2797632, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.2.parquet, range: 0-2797502, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.12.parquet, range: 0-2797316, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.14.parquet, range: 0-2797103, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.22.parquet, range: 0-2797014, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.17.parquet, range: 0-2795731, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO Executor: Finished task 0.0 in stage 11.0 (TID 225). 2208 bytes result sent to driver -26/04/01 08:38:40 INFO TaskSetManager: Starting task 4.0 in stage 11.0 (TID 229) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9926 bytes) -26/04/01 08:38:40 INFO Executor: Running task 4.0 in stage 11.0 (TID 229) -26/04/01 08:38:40 INFO TaskSetManager: Finished task 0.0 in stage 11.0 (TID 225) in 184 ms on 10.0.0.133 (executor driver) (1/8) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.27.parquet, range: 0-2795177, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO Executor: Finished task 1.0 in stage 11.0 (TID 226). 2251 bytes result sent to driver -26/04/01 08:38:40 INFO TaskSetManager: Starting task 5.0 in stage 11.0 (TID 230) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9925 bytes) -26/04/01 08:38:40 INFO TaskSetManager: Finished task 1.0 in stage 11.0 (TID 226) in 166 ms on 10.0.0.133 (executor driver) (2/8) -26/04/01 08:38:40 INFO Executor: Running task 5.0 in stage 11.0 (TID 230) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.3.parquet, range: 0-2794406, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.10.parquet, range: 0-2795129, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO Executor: Finished task 2.0 in stage 11.0 (TID 227). 2208 bytes result sent to driver -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.19.parquet, range: 0-2794808, partition values: [empty row] -26/04/01 08:38:40 INFO TaskSetManager: Starting task 6.0 in stage 11.0 (TID 231) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9924 bytes) -26/04/01 08:38:40 INFO Executor: Running task 6.0 in stage 11.0 (TID 231) -26/04/01 08:38:40 INFO TaskSetManager: Finished task 2.0 in stage 11.0 (TID 227) in 125 ms on 10.0.0.133 (executor driver) (3/8) -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.32.parquet, range: 0-2792572, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.15.parquet, range: 0-2794714, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.16.parquet, range: 0-2794298, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.24.parquet, range: 0-2793929, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO Executor: Finished task 3.0 in stage 11.0 (TID 228). 2251 bytes result sent to driver -26/04/01 08:38:40 INFO TaskSetManager: Starting task 7.0 in stage 11.0 (TID 232) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9924 bytes) -26/04/01 08:38:40 INFO Executor: Running task 7.0 in stage 11.0 (TID 232) -26/04/01 08:38:40 INFO TaskSetManager: Finished task 3.0 in stage 11.0 (TID 228) in 148 ms on 10.0.0.133 (executor driver) (4/8) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.20.parquet, range: 0-2791117, partition values: [empty row] -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.23.parquet, range: 0-2792937, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.5.parquet, range: 0-2792421, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.7.parquet, range: 0-2791953, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.29.parquet, range: 0-2791555, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.28.parquet, range: 0-2790847, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.8.parquet, range: 0-2790608, partition values: [empty row] -26/04/01 08:38:40 INFO Executor: Finished task 4.0 in stage 11.0 (TID 229). 2208 bytes result sent to driver -26/04/01 08:38:40 INFO TaskSetManager: Starting task 0.0 in stage 12.0 (TID 233) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:40 INFO Executor: Running task 0.0 in stage 12.0 (TID 233) -26/04/01 08:38:40 INFO TaskSetManager: Finished task 4.0 in stage 11.0 (TID 229) in 164 ms on 10.0.0.133 (executor driver) (5/8) -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO CodeGenerator: Code generated in 3.386042 ms -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/supplier/supplier.4.parquet, range: 0-2789514, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(s_suppkey, null), noteq(s_nationkey, null)) -26/04/01 08:38:40 INFO CodeGenerator: Code generated in 3.224208 ms -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.15.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:40 INFO Executor: Finished task 5.0 in stage 11.0 (TID 230). 2208 bytes result sent to driver -26/04/01 08:38:40 INFO TaskSetManager: Starting task 1.0 in stage 12.0 (TID 234) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:40 INFO TaskSetManager: Finished task 5.0 in stage 11.0 (TID 230) in 207 ms on 10.0.0.133 (executor driver) (6/8) -26/04/01 08:38:40 INFO Executor: Running task 1.0 in stage 12.0 (TID 234) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.23.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:40 INFO Executor: Finished task 6.0 in stage 11.0 (TID 231). 2208 bytes result sent to driver -26/04/01 08:38:40 INFO TaskSetManager: Starting task 2.0 in stage 12.0 (TID 235) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:40 INFO TaskSetManager: Finished task 6.0 in stage 11.0 (TID 231) in 203 ms on 10.0.0.133 (executor driver) (7/8) -26/04/01 08:38:40 INFO Executor: Running task 2.0 in stage 12.0 (TID 235) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.9.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:40 INFO Executor: Finished task 7.0 in stage 11.0 (TID 232). 2208 bytes result sent to driver -26/04/01 08:38:40 INFO TaskSetManager: Starting task 3.0 in stage 12.0 (TID 236) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:40 INFO Executor: Running task 3.0 in stage 12.0 (TID 236) -26/04/01 08:38:40 INFO TaskSetManager: Finished task 7.0 in stage 11.0 (TID 232) in 250 ms on 10.0.0.133 (executor driver) (8/8) -26/04/01 08:38:40 INFO TaskSchedulerImpl: Removed TaskSet 11.0, whose tasks have all completed, from pool -26/04/01 08:38:40 INFO DAGScheduler: ShuffleMapStage 11 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 33.513 s -26/04/01 08:38:40 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:38:40 INFO DAGScheduler: running: Set(ShuffleMapStage 12, ShuffleMapStage 13, ShuffleMapStage 10) -26/04/01 08:38:40 INFO DAGScheduler: waiting: Set() -26/04/01 08:38:40 INFO DAGScheduler: failed: Set() -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.22.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:40 INFO Executor: Finished task 204.0 in stage 10.0 (TID 221). 2251 bytes result sent to driver -26/04/01 08:38:40 INFO TaskSetManager: Starting task 4.0 in stage 12.0 (TID 237) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:40 INFO Executor: Running task 4.0 in stage 12.0 (TID 237) -26/04/01 08:38:40 INFO TaskSetManager: Finished task 204.0 in stage 10.0 (TID 221) in 1223 ms on 10.0.0.133 (executor driver) (205/208) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.32.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:40 INFO Executor: Finished task 205.0 in stage 10.0 (TID 222). 2251 bytes result sent to driver -26/04/01 08:38:40 INFO TaskSetManager: Starting task 5.0 in stage 12.0 (TID 238) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:40 INFO TaskSetManager: Finished task 205.0 in stage 10.0 (TID 222) in 1232 ms on 10.0.0.133 (executor driver) (206/208) -26/04/01 08:38:40 INFO Executor: Running task 5.0 in stage 12.0 (TID 238) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.8.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:40 INFO Executor: Finished task 206.0 in stage 10.0 (TID 223). 2251 bytes result sent to driver -26/04/01 08:38:40 INFO TaskSetManager: Starting task 6.0 in stage 12.0 (TID 239) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:40 INFO Executor: Running task 6.0 in stage 12.0 (TID 239) -26/04/01 08:38:40 INFO TaskSetManager: Finished task 206.0 in stage 10.0 (TID 223) in 1246 ms on 10.0.0.133 (executor driver) (207/208) -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.1.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:40 INFO Executor: Finished task 207.0 in stage 10.0 (TID 224). 2251 bytes result sent to driver -26/04/01 08:38:40 INFO TaskSetManager: Starting task 7.0 in stage 12.0 (TID 240) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:40 INFO TaskSetManager: Finished task 207.0 in stage 10.0 (TID 224) in 1193 ms on 10.0.0.133 (executor driver) (208/208) -26/04/01 08:38:40 INFO Executor: Running task 7.0 in stage 12.0 (TID 240) -26/04/01 08:38:40 INFO TaskSchedulerImpl: Removed TaskSet 10.0, whose tasks have all completed, from pool -26/04/01 08:38:40 INFO DAGScheduler: ShuffleMapStage 10 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 33.763 s -26/04/01 08:38:40 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:38:40 INFO DAGScheduler: running: Set(ShuffleMapStage 12, ShuffleMapStage 13) -26/04/01 08:38:40 INFO DAGScheduler: waiting: Set() -26/04/01 08:38:40 INFO DAGScheduler: failed: Set() -26/04/01 08:38:40 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.14.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:40 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:40 INFO ShufflePartitionsUtil: For shuffle(0, 1), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 08:38:41 INFO CodeGenerator: Code generated in 10.615 ms -26/04/01 08:38:41 INFO CodeGenerator: Code generated in 4.236625 ms -26/04/01 08:38:41 INFO CodeGenerator: Code generated in 2.943458 ms -26/04/01 08:38:41 INFO DAGScheduler: Registering RDD 46 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 5 -26/04/01 08:38:41 INFO DAGScheduler: Got map stage job 14 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 200 output partitions -26/04/01 08:38:41 INFO DAGScheduler: Final stage: ShuffleMapStage 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:38:41 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 15, ShuffleMapStage 14) -26/04/01 08:38:41 INFO DAGScheduler: Missing parents: List() -26/04/01 08:38:41 INFO DAGScheduler: Submitting ShuffleMapStage 16 (MapPartitionsRDD[46] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:38:41 INFO MemoryStore: Block broadcast_21 stored as values in memory (estimated size 55.2 KiB, free 8.6 GiB) -26/04/01 08:38:41 INFO MemoryStore: Block broadcast_21_piece0 stored as bytes in memory (estimated size 26.3 KiB, free 8.6 GiB) -26/04/01 08:38:41 INFO BlockManagerInfo: Added broadcast_21_piece0 in memory on 10.0.0.133:59288 (size: 26.3 KiB, free: 8.6 GiB) -26/04/01 08:38:41 INFO SparkContext: Created broadcast 21 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:38:41 INFO DAGScheduler: Submitting 200 missing tasks from ShuffleMapStage 16 (MapPartitionsRDD[46] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:38:41 INFO TaskSchedulerImpl: Adding task set 16.0 with 200 tasks resource profile 0 -26/04/01 08:38:41 INFO Executor: Finished task 0.0 in stage 12.0 (TID 233). 2251 bytes result sent to driver -26/04/01 08:38:41 INFO TaskSetManager: Starting task 8.0 in stage 12.0 (TID 241) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:41 INFO Executor: Running task 8.0 in stage 12.0 (TID 241) -26/04/01 08:38:41 INFO TaskSetManager: Finished task 0.0 in stage 12.0 (TID 233) in 1007 ms on 10.0.0.133 (executor driver) (1/37) -26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.30.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:41 INFO Executor: Finished task 2.0 in stage 12.0 (TID 235). 2251 bytes result sent to driver -26/04/01 08:38:41 INFO TaskSetManager: Starting task 9.0 in stage 12.0 (TID 242) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:41 INFO Executor: Running task 9.0 in stage 12.0 (TID 242) -26/04/01 08:38:41 INFO TaskSetManager: Finished task 2.0 in stage 12.0 (TID 235) in 1002 ms on 10.0.0.133 (executor driver) (2/37) -26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.20.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:41 INFO Executor: Finished task 1.0 in stage 12.0 (TID 234). 2251 bytes result sent to driver -26/04/01 08:38:41 INFO TaskSetManager: Starting task 10.0 in stage 12.0 (TID 243) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:41 INFO TaskSetManager: Finished task 1.0 in stage 12.0 (TID 234) in 1041 ms on 10.0.0.133 (executor driver) (3/37) -26/04/01 08:38:41 INFO Executor: Running task 10.0 in stage 12.0 (TID 243) -26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.29.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:41 INFO Executor: Finished task 4.0 in stage 12.0 (TID 237). 2251 bytes result sent to driver -26/04/01 08:38:41 INFO TaskSetManager: Starting task 11.0 in stage 12.0 (TID 244) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:41 INFO Executor: Running task 11.0 in stage 12.0 (TID 244) -26/04/01 08:38:41 INFO TaskSetManager: Finished task 4.0 in stage 12.0 (TID 237) in 941 ms on 10.0.0.133 (executor driver) (4/37) -26/04/01 08:38:41 INFO Executor: Finished task 3.0 in stage 12.0 (TID 236). 2251 bytes result sent to driver -26/04/01 08:38:41 INFO TaskSetManager: Starting task 12.0 in stage 12.0 (TID 245) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:41 INFO TaskSetManager: Finished task 3.0 in stage 12.0 (TID 236) in 960 ms on 10.0.0.133 (executor driver) (5/37) -26/04/01 08:38:41 INFO Executor: Running task 12.0 in stage 12.0 (TID 245) -26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.3.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.16.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:41 INFO Executor: Finished task 5.0 in stage 12.0 (TID 238). 2251 bytes result sent to driver -26/04/01 08:38:41 INFO TaskSetManager: Starting task 13.0 in stage 12.0 (TID 246) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:41 INFO TaskSetManager: Finished task 5.0 in stage 12.0 (TID 238) in 985 ms on 10.0.0.133 (executor driver) (6/37) -26/04/01 08:38:41 INFO Executor: Running task 13.0 in stage 12.0 (TID 246) -26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.28.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:41 INFO Executor: Finished task 6.0 in stage 12.0 (TID 239). 2251 bytes result sent to driver -26/04/01 08:38:41 INFO TaskSetManager: Starting task 14.0 in stage 12.0 (TID 247) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:41 INFO Executor: Running task 14.0 in stage 12.0 (TID 247) -26/04/01 08:38:41 INFO TaskSetManager: Finished task 6.0 in stage 12.0 (TID 239) in 1012 ms on 10.0.0.133 (executor driver) (7/37) -26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.2.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:41 INFO Executor: Finished task 7.0 in stage 12.0 (TID 240). 2251 bytes result sent to driver -26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:41 INFO TaskSetManager: Starting task 15.0 in stage 12.0 (TID 248) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:41 INFO TaskSetManager: Finished task 7.0 in stage 12.0 (TID 240) in 989 ms on 10.0.0.133 (executor driver) (8/37) -26/04/01 08:38:41 INFO Executor: Running task 15.0 in stage 12.0 (TID 248) -26/04/01 08:38:41 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.17.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:41 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:42 INFO Executor: Finished task 8.0 in stage 12.0 (TID 241). 2251 bytes result sent to driver -26/04/01 08:38:42 INFO TaskSetManager: Starting task 16.0 in stage 12.0 (TID 249) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:42 INFO TaskSetManager: Finished task 8.0 in stage 12.0 (TID 241) in 758 ms on 10.0.0.133 (executor driver) (9/37) -26/04/01 08:38:42 INFO Executor: Running task 16.0 in stage 12.0 (TID 249) -26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.21.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:42 INFO Executor: Finished task 9.0 in stage 12.0 (TID 242). 2251 bytes result sent to driver -26/04/01 08:38:42 INFO TaskSetManager: Starting task 17.0 in stage 12.0 (TID 250) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:42 INFO TaskSetManager: Finished task 9.0 in stage 12.0 (TID 242) in 925 ms on 10.0.0.133 (executor driver) (10/37) -26/04/01 08:38:42 INFO Executor: Running task 17.0 in stage 12.0 (TID 250) -26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.31.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:42 INFO Executor: Finished task 10.0 in stage 12.0 (TID 243). 2251 bytes result sent to driver -26/04/01 08:38:42 INFO TaskSetManager: Starting task 18.0 in stage 12.0 (TID 251) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:42 INFO Executor: Running task 18.0 in stage 12.0 (TID 251) -26/04/01 08:38:42 INFO TaskSetManager: Finished task 10.0 in stage 12.0 (TID 243) in 990 ms on 10.0.0.133 (executor driver) (11/37) -26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.24.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:42 INFO Executor: Finished task 11.0 in stage 12.0 (TID 244). 2251 bytes result sent to driver -26/04/01 08:38:42 INFO TaskSetManager: Starting task 19.0 in stage 12.0 (TID 252) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:42 INFO Executor: Running task 19.0 in stage 12.0 (TID 252) -26/04/01 08:38:42 INFO TaskSetManager: Finished task 11.0 in stage 12.0 (TID 244) in 973 ms on 10.0.0.133 (executor driver) (12/37) -26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.12.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:42 INFO Executor: Finished task 12.0 in stage 12.0 (TID 245). 2251 bytes result sent to driver -26/04/01 08:38:42 INFO TaskSetManager: Starting task 20.0 in stage 12.0 (TID 253) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:42 INFO TaskSetManager: Finished task 12.0 in stage 12.0 (TID 245) in 987 ms on 10.0.0.133 (executor driver) (13/37) -26/04/01 08:38:42 INFO Executor: Running task 20.0 in stage 12.0 (TID 253) -26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.7.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:42 INFO Executor: Finished task 13.0 in stage 12.0 (TID 246). 2251 bytes result sent to driver -26/04/01 08:38:42 INFO TaskSetManager: Starting task 21.0 in stage 12.0 (TID 254) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:42 INFO TaskSetManager: Finished task 13.0 in stage 12.0 (TID 246) in 856 ms on 10.0.0.133 (executor driver) (14/37) -26/04/01 08:38:42 INFO Executor: Running task 21.0 in stage 12.0 (TID 254) -26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.13.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:42 INFO Executor: Finished task 15.0 in stage 12.0 (TID 248). 2251 bytes result sent to driver -26/04/01 08:38:42 INFO TaskSetManager: Starting task 22.0 in stage 12.0 (TID 255) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:42 INFO TaskSetManager: Finished task 15.0 in stage 12.0 (TID 248) in 868 ms on 10.0.0.133 (executor driver) (15/37) -26/04/01 08:38:42 INFO Executor: Running task 22.0 in stage 12.0 (TID 255) -26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.6.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:42 INFO Executor: Finished task 14.0 in stage 12.0 (TID 247). 2251 bytes result sent to driver -26/04/01 08:38:42 INFO TaskSetManager: Starting task 23.0 in stage 12.0 (TID 256) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:42 INFO Executor: Running task 23.0 in stage 12.0 (TID 256) -26/04/01 08:38:42 INFO TaskSetManager: Finished task 14.0 in stage 12.0 (TID 247) in 882 ms on 10.0.0.133 (executor driver) (16/37) -26/04/01 08:38:42 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.25.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:42 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:43 INFO Executor: Finished task 16.0 in stage 12.0 (TID 249). 2251 bytes result sent to driver -26/04/01 08:38:43 INFO TaskSetManager: Starting task 24.0 in stage 12.0 (TID 257) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:43 INFO TaskSetManager: Finished task 16.0 in stage 12.0 (TID 249) in 792 ms on 10.0.0.133 (executor driver) (17/37) -26/04/01 08:38:43 INFO Executor: Running task 24.0 in stage 12.0 (TID 257) -26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.11.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:43 INFO Executor: Finished task 17.0 in stage 12.0 (TID 250). 2294 bytes result sent to driver -26/04/01 08:38:43 INFO TaskSetManager: Starting task 25.0 in stage 12.0 (TID 258) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:43 INFO TaskSetManager: Finished task 17.0 in stage 12.0 (TID 250) in 856 ms on 10.0.0.133 (executor driver) (18/37) -26/04/01 08:38:43 INFO Executor: Running task 25.0 in stage 12.0 (TID 258) -26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.4.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:43 INFO Executor: Finished task 18.0 in stage 12.0 (TID 251). 2251 bytes result sent to driver -26/04/01 08:38:43 INFO TaskSetManager: Starting task 26.0 in stage 12.0 (TID 259) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:43 INFO TaskSetManager: Finished task 18.0 in stage 12.0 (TID 251) in 810 ms on 10.0.0.133 (executor driver) (19/37) -26/04/01 08:38:43 INFO Executor: Running task 26.0 in stage 12.0 (TID 259) -26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.18.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:43 INFO Executor: Finished task 19.0 in stage 12.0 (TID 252). 2251 bytes result sent to driver -26/04/01 08:38:43 INFO TaskSetManager: Starting task 27.0 in stage 12.0 (TID 260) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:43 INFO Executor: Running task 27.0 in stage 12.0 (TID 260) -26/04/01 08:38:43 INFO TaskSetManager: Finished task 19.0 in stage 12.0 (TID 252) in 827 ms on 10.0.0.133 (executor driver) (20/37) -26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.27.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:43 INFO Executor: Finished task 20.0 in stage 12.0 (TID 253). 2251 bytes result sent to driver -26/04/01 08:38:43 INFO TaskSetManager: Starting task 28.0 in stage 12.0 (TID 261) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:43 INFO Executor: Running task 28.0 in stage 12.0 (TID 261) -26/04/01 08:38:43 INFO TaskSetManager: Finished task 20.0 in stage 12.0 (TID 253) in 910 ms on 10.0.0.133 (executor driver) (21/37) -26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.19.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:43 INFO Executor: Finished task 21.0 in stage 12.0 (TID 254). 2251 bytes result sent to driver -26/04/01 08:38:43 INFO TaskSetManager: Starting task 29.0 in stage 12.0 (TID 262) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:43 INFO TaskSetManager: Finished task 21.0 in stage 12.0 (TID 254) in 854 ms on 10.0.0.133 (executor driver) (22/37) -26/04/01 08:38:43 INFO Executor: Running task 29.0 in stage 12.0 (TID 262) -26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.26.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:43 INFO Executor: Finished task 22.0 in stage 12.0 (TID 255). 2251 bytes result sent to driver -26/04/01 08:38:43 INFO TaskSetManager: Starting task 30.0 in stage 12.0 (TID 263) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9602 bytes) -26/04/01 08:38:43 INFO Executor: Running task 30.0 in stage 12.0 (TID 263) -26/04/01 08:38:43 INFO TaskSetManager: Finished task 22.0 in stage 12.0 (TID 255) in 837 ms on 10.0.0.133 (executor driver) (23/37) -26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.10.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:43 INFO Executor: Finished task 23.0 in stage 12.0 (TID 256). 2251 bytes result sent to driver -26/04/01 08:38:43 INFO TaskSetManager: Starting task 31.0 in stage 12.0 (TID 264) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9601 bytes) -26/04/01 08:38:43 INFO TaskSetManager: Finished task 23.0 in stage 12.0 (TID 256) in 830 ms on 10.0.0.133 (executor driver) (24/37) -26/04/01 08:38:43 INFO Executor: Running task 31.0 in stage 12.0 (TID 264) -26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.5.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:43 INFO Executor: Finished task 24.0 in stage 12.0 (TID 257). 2251 bytes result sent to driver -26/04/01 08:38:43 INFO TaskSetManager: Starting task 32.0 in stage 12.0 (TID 265) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 10249 bytes) -26/04/01 08:38:43 INFO TaskSetManager: Finished task 24.0 in stage 12.0 (TID 257) in 766 ms on 10.0.0.133 (executor driver) (25/37) -26/04/01 08:38:43 INFO Executor: Running task 32.0 in stage 12.0 (TID 265) -26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.30.parquet, range: 134217728-148587359, partition values: [empty row] -26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.20.parquet, range: 134217728-148586863, partition values: [empty row] -26/04/01 08:38:43 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:43 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.12.parquet, range: 134217728-148577687, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.13.parquet, range: 134217728-148575763, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.7.parquet, range: 134217728-148573022, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.21.parquet, range: 134217728-148568504, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO Executor: Finished task 25.0 in stage 12.0 (TID 258). 2294 bytes result sent to driver -26/04/01 08:38:44 INFO TaskSetManager: Starting task 33.0 in stage 12.0 (TID 266) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 10250 bytes) -26/04/01 08:38:44 INFO Executor: Running task 33.0 in stage 12.0 (TID 266) -26/04/01 08:38:44 INFO TaskSetManager: Finished task 25.0 in stage 12.0 (TID 258) in 833 ms on 10.0.0.133 (executor driver) (26/37) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.19.parquet, range: 134217728-148561478, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO Executor: Finished task 26.0 in stage 12.0 (TID 259). 2251 bytes result sent to driver -26/04/01 08:38:44 INFO TaskSetManager: Starting task 34.0 in stage 12.0 (TID 267) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 10246 bytes) -26/04/01 08:38:44 INFO TaskSetManager: Finished task 26.0 in stage 12.0 (TID 259) in 869 ms on 10.0.0.133 (executor driver) (27/37) -26/04/01 08:38:44 INFO Executor: Running task 34.0 in stage 12.0 (TID 267) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.1.parquet, range: 134217728-148552923, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.32.parquet, range: 134217728-148564089, partition values: [empty row] -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.26.parquet, range: 134217728-148560884, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.27.parquet, range: 134217728-148559546, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.2.parquet, range: 134217728-148551531, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO Executor: Finished task 27.0 in stage 12.0 (TID 260). 2251 bytes result sent to driver -26/04/01 08:38:44 INFO TaskSetManager: Starting task 35.0 in stage 12.0 (TID 268) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 10249 bytes) -26/04/01 08:38:44 INFO TaskSetManager: Finished task 27.0 in stage 12.0 (TID 260) in 944 ms on 10.0.0.133 (executor driver) (28/37) -26/04/01 08:38:44 INFO Executor: Running task 35.0 in stage 12.0 (TID 268) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.14.parquet, range: 134217728-148540833, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.28.parquet, range: 134217728-148544471, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.15.parquet, range: 134217728-148558797, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.31.parquet, range: 134217728-148540689, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.10.parquet, range: 134217728-148543426, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.29.parquet, range: 134217728-148539628, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO Executor: Finished task 28.0 in stage 12.0 (TID 261). 2251 bytes result sent to driver -26/04/01 08:38:44 INFO TaskSetManager: Starting task 36.0 in stage 12.0 (TID 269) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9923 bytes) -26/04/01 08:38:44 INFO Executor: Running task 36.0 in stage 12.0 (TID 269) -26/04/01 08:38:44 INFO TaskSetManager: Finished task 28.0 in stage 12.0 (TID 261) in 976 ms on 10.0.0.133 (executor driver) (29/37) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.6.parquet, range: 134217728-148521236, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO Executor: Finished task 29.0 in stage 12.0 (TID 262). 2251 bytes result sent to driver -26/04/01 08:38:44 INFO TaskSetManager: Starting task 0.0 in stage 13.0 (TID 270) (10.0.0.133, executor driver, partition 0, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:44 INFO Executor: Running task 0.0 in stage 13.0 (TID 270) -26/04/01 08:38:44 INFO TaskSetManager: Finished task 29.0 in stage 12.0 (TID 262) in 993 ms on 10.0.0.133 (executor driver) (30/37) -26/04/01 08:38:44 INFO CodeGenerator: Code generated in 3.345584 ms -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.23.parquet, range: 134217728-148558546, partition values: [empty row] -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.13.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:44 INFO Executor: Finished task 30.0 in stage 12.0 (TID 263). 2251 bytes result sent to driver -26/04/01 08:38:44 INFO TaskSetManager: Starting task 1.0 in stage 13.0 (TID 271) (10.0.0.133, executor driver, partition 1, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:44 INFO Executor: Running task 1.0 in stage 13.0 (TID 271) -26/04/01 08:38:44 INFO TaskSetManager: Finished task 30.0 in stage 12.0 (TID 263) in 954 ms on 10.0.0.133 (executor driver) (31/37) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.7.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:44 INFO Executor: Finished task 31.0 in stage 12.0 (TID 264). 2251 bytes result sent to driver -26/04/01 08:38:44 INFO TaskSetManager: Starting task 2.0 in stage 13.0 (TID 272) (10.0.0.133, executor driver, partition 2, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:44 INFO Executor: Running task 2.0 in stage 13.0 (TID 272) -26/04/01 08:38:44 INFO TaskSetManager: Finished task 31.0 in stage 12.0 (TID 264) in 957 ms on 10.0.0.133 (executor driver) (32/37) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.25.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.8.parquet, range: 134217728-148542255, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO Executor: Finished task 32.0 in stage 12.0 (TID 265). 2251 bytes result sent to driver -26/04/01 08:38:44 INFO TaskSetManager: Starting task 3.0 in stage 13.0 (TID 273) (10.0.0.133, executor driver, partition 3, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:44 INFO TaskSetManager: Finished task 32.0 in stage 12.0 (TID 265) in 748 ms on 10.0.0.133 (executor driver) (33/37) -26/04/01 08:38:44 INFO Executor: Running task 3.0 in stage 13.0 (TID 273) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.24.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.25.parquet, range: 134217728-148557400, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.16.parquet, range: 134217728-148538785, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.4.parquet, range: 134217728-148516007, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.24.parquet, range: 134217728-148542182, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.17.parquet, range: 134217728-148515342, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.11.parquet, range: 134217728-148554765, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.9.parquet, range: 134217728-148538414, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.5.parquet, range: 134217728-148541392, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.3.parquet, range: 134217728-148499293, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.22.parquet, range: 134217728-148525436, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:44 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/partsupp/partsupp.18.parquet, range: 134217728-148522266, partition values: [empty row] -26/04/01 08:38:44 INFO FilterCompat: Filtering using predicate: and(noteq(ps_suppkey, null), noteq(ps_partkey, null)) -26/04/01 08:38:45 INFO Executor: Finished task 33.0 in stage 12.0 (TID 266). 2251 bytes result sent to driver -26/04/01 08:38:45 INFO TaskSetManager: Starting task 4.0 in stage 13.0 (TID 274) (10.0.0.133, executor driver, partition 4, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:45 INFO TaskSetManager: Finished task 33.0 in stage 12.0 (TID 266) in 767 ms on 10.0.0.133 (executor driver) (34/37) -26/04/01 08:38:45 INFO Executor: Running task 4.0 in stage 13.0 (TID 274) -26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.12.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:45 INFO Executor: Finished task 36.0 in stage 12.0 (TID 269). 2208 bytes result sent to driver -26/04/01 08:38:45 INFO TaskSetManager: Starting task 5.0 in stage 13.0 (TID 275) (10.0.0.133, executor driver, partition 5, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:45 INFO TaskSetManager: Finished task 36.0 in stage 12.0 (TID 269) in 482 ms on 10.0.0.133 (executor driver) (35/37) -26/04/01 08:38:45 INFO Executor: Running task 5.0 in stage 13.0 (TID 275) -26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.6.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:45 INFO Executor: Finished task 34.0 in stage 12.0 (TID 267). 2251 bytes result sent to driver -26/04/01 08:38:45 INFO TaskSetManager: Starting task 6.0 in stage 13.0 (TID 276) (10.0.0.133, executor driver, partition 6, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:45 INFO TaskSetManager: Finished task 34.0 in stage 12.0 (TID 267) in 764 ms on 10.0.0.133 (executor driver) (36/37) -26/04/01 08:38:45 INFO Executor: Running task 6.0 in stage 13.0 (TID 276) -26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.19.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:45 INFO Executor: Finished task 35.0 in stage 12.0 (TID 268). 2208 bytes result sent to driver -26/04/01 08:38:45 INFO TaskSetManager: Starting task 7.0 in stage 13.0 (TID 277) (10.0.0.133, executor driver, partition 7, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:45 INFO TaskSetManager: Finished task 35.0 in stage 12.0 (TID 268) in 690 ms on 10.0.0.133 (executor driver) (37/37) -26/04/01 08:38:45 INFO Executor: Running task 7.0 in stage 13.0 (TID 277) -26/04/01 08:38:45 INFO TaskSchedulerImpl: Removed TaskSet 12.0, whose tasks have all completed, from pool -26/04/01 08:38:45 INFO DAGScheduler: ShuffleMapStage 12 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 37.888 s -26/04/01 08:38:45 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:38:45 INFO DAGScheduler: running: Set(ShuffleMapStage 16, ShuffleMapStage 13) -26/04/01 08:38:45 INFO DAGScheduler: waiting: Set() -26/04/01 08:38:45 INFO DAGScheduler: failed: Set() -26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.26.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:45 INFO Executor: Finished task 2.0 in stage 13.0 (TID 272). 2208 bytes result sent to driver -26/04/01 08:38:45 INFO TaskSetManager: Starting task 8.0 in stage 13.0 (TID 278) (10.0.0.133, executor driver, partition 8, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:45 INFO Executor: Running task 8.0 in stage 13.0 (TID 278) -26/04/01 08:38:45 INFO TaskSetManager: Finished task 2.0 in stage 13.0 (TID 272) in 934 ms on 10.0.0.133 (executor driver) (1/64) -26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.4.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:45 INFO Executor: Finished task 0.0 in stage 13.0 (TID 270). 2251 bytes result sent to driver -26/04/01 08:38:45 INFO TaskSetManager: Starting task 9.0 in stage 13.0 (TID 279) (10.0.0.133, executor driver, partition 9, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:45 INFO Executor: Running task 9.0 in stage 13.0 (TID 279) -26/04/01 08:38:45 INFO TaskSetManager: Finished task 0.0 in stage 13.0 (TID 270) in 993 ms on 10.0.0.133 (executor driver) (2/64) -26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.10.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:45 INFO Executor: Finished task 3.0 in stage 13.0 (TID 273). 2251 bytes result sent to driver -26/04/01 08:38:45 INFO TaskSetManager: Starting task 10.0 in stage 13.0 (TID 280) (10.0.0.133, executor driver, partition 10, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:45 INFO TaskSetManager: Finished task 3.0 in stage 13.0 (TID 273) in 958 ms on 10.0.0.133 (executor driver) (3/64) -26/04/01 08:38:45 INFO Executor: Running task 10.0 in stage 13.0 (TID 280) -26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.5.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:45 INFO Executor: Finished task 1.0 in stage 13.0 (TID 271). 2251 bytes result sent to driver -26/04/01 08:38:45 INFO TaskSetManager: Starting task 11.0 in stage 13.0 (TID 281) (10.0.0.133, executor driver, partition 11, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:45 INFO TaskSetManager: Finished task 1.0 in stage 13.0 (TID 271) in 979 ms on 10.0.0.133 (executor driver) (4/64) -26/04/01 08:38:45 INFO Executor: Running task 11.0 in stage 13.0 (TID 281) -26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.11.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:45 INFO Executor: Finished task 4.0 in stage 13.0 (TID 274). 2251 bytes result sent to driver -26/04/01 08:38:45 INFO TaskSetManager: Starting task 12.0 in stage 13.0 (TID 282) (10.0.0.133, executor driver, partition 12, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:45 INFO Executor: Running task 12.0 in stage 13.0 (TID 282) -26/04/01 08:38:45 INFO TaskSetManager: Finished task 4.0 in stage 13.0 (TID 274) in 765 ms on 10.0.0.133 (executor driver) (5/64) -26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.18.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:45 INFO Executor: Finished task 5.0 in stage 13.0 (TID 275). 2251 bytes result sent to driver -26/04/01 08:38:45 INFO TaskSetManager: Starting task 13.0 in stage 13.0 (TID 283) (10.0.0.133, executor driver, partition 13, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:45 INFO Executor: Running task 13.0 in stage 13.0 (TID 283) -26/04/01 08:38:45 INFO TaskSetManager: Finished task 5.0 in stage 13.0 (TID 275) in 805 ms on 10.0.0.133 (executor driver) (6/64) -26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.27.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:45 INFO Executor: Finished task 6.0 in stage 13.0 (TID 276). 2251 bytes result sent to driver -26/04/01 08:38:45 INFO TaskSetManager: Starting task 14.0 in stage 13.0 (TID 284) (10.0.0.133, executor driver, partition 14, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:45 INFO Executor: Running task 14.0 in stage 13.0 (TID 284) -26/04/01 08:38:45 INFO TaskSetManager: Finished task 6.0 in stage 13.0 (TID 276) in 790 ms on 10.0.0.133 (executor driver) (7/64) -26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.32.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:45 INFO Executor: Finished task 7.0 in stage 13.0 (TID 277). 2251 bytes result sent to driver -26/04/01 08:38:45 INFO TaskSetManager: Starting task 15.0 in stage 13.0 (TID 285) (10.0.0.133, executor driver, partition 15, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:45 INFO TaskSetManager: Finished task 7.0 in stage 13.0 (TID 277) in 810 ms on 10.0.0.133 (executor driver) (8/64) -26/04/01 08:38:45 INFO Executor: Running task 15.0 in stage 13.0 (TID 285) -26/04/01 08:38:45 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.22.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:45 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:46 INFO Executor: Finished task 8.0 in stage 13.0 (TID 278). 2251 bytes result sent to driver -26/04/01 08:38:46 INFO TaskSetManager: Starting task 16.0 in stage 13.0 (TID 286) (10.0.0.133, executor driver, partition 16, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:46 INFO TaskSetManager: Finished task 8.0 in stage 13.0 (TID 278) in 874 ms on 10.0.0.133 (executor driver) (9/64) -26/04/01 08:38:46 INFO Executor: Running task 16.0 in stage 13.0 (TID 286) -26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.9.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:46 INFO Executor: Finished task 10.0 in stage 13.0 (TID 280). 2208 bytes result sent to driver -26/04/01 08:38:46 INFO TaskSetManager: Starting task 17.0 in stage 13.0 (TID 287) (10.0.0.133, executor driver, partition 17, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:46 INFO Executor: Finished task 11.0 in stage 13.0 (TID 281). 2208 bytes result sent to driver -26/04/01 08:38:46 INFO Executor: Running task 17.0 in stage 13.0 (TID 287) -26/04/01 08:38:46 INFO TaskSetManager: Starting task 18.0 in stage 13.0 (TID 288) (10.0.0.133, executor driver, partition 18, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:46 INFO Executor: Running task 18.0 in stage 13.0 (TID 288) -26/04/01 08:38:46 INFO TaskSetManager: Finished task 10.0 in stage 13.0 (TID 280) in 881 ms on 10.0.0.133 (executor driver) (10/64) -26/04/01 08:38:46 INFO TaskSetManager: Finished task 11.0 in stage 13.0 (TID 281) in 878 ms on 10.0.0.133 (executor driver) (11/64) -26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.14.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.1.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:46 INFO Executor: Finished task 9.0 in stage 13.0 (TID 279). 2208 bytes result sent to driver -26/04/01 08:38:46 INFO TaskSetManager: Starting task 19.0 in stage 13.0 (TID 289) (10.0.0.133, executor driver, partition 19, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:46 INFO Executor: Running task 19.0 in stage 13.0 (TID 289) -26/04/01 08:38:46 INFO TaskSetManager: Finished task 9.0 in stage 13.0 (TID 279) in 892 ms on 10.0.0.133 (executor driver) (12/64) -26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.15.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:46 INFO Executor: Finished task 12.0 in stage 13.0 (TID 282). 2208 bytes result sent to driver -26/04/01 08:38:46 INFO TaskSetManager: Starting task 20.0 in stage 13.0 (TID 290) (10.0.0.133, executor driver, partition 20, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:46 INFO TaskSetManager: Finished task 12.0 in stage 13.0 (TID 282) in 796 ms on 10.0.0.133 (executor driver) (13/64) -26/04/01 08:38:46 INFO Executor: Running task 20.0 in stage 13.0 (TID 290) -26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.23.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:46 INFO Executor: Finished task 13.0 in stage 13.0 (TID 283). 2208 bytes result sent to driver -26/04/01 08:38:46 INFO TaskSetManager: Starting task 21.0 in stage 13.0 (TID 291) (10.0.0.133, executor driver, partition 21, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:46 INFO TaskSetManager: Finished task 13.0 in stage 13.0 (TID 283) in 807 ms on 10.0.0.133 (executor driver) (14/64) -26/04/01 08:38:46 INFO Executor: Running task 21.0 in stage 13.0 (TID 291) -26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.8.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:46 INFO Executor: Finished task 14.0 in stage 13.0 (TID 284). 2208 bytes result sent to driver -26/04/01 08:38:46 INFO TaskSetManager: Starting task 22.0 in stage 13.0 (TID 292) (10.0.0.133, executor driver, partition 22, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:46 INFO TaskSetManager: Finished task 14.0 in stage 13.0 (TID 284) in 791 ms on 10.0.0.133 (executor driver) (15/64) -26/04/01 08:38:46 INFO Executor: Running task 22.0 in stage 13.0 (TID 292) -26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.28.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:46 INFO Executor: Finished task 15.0 in stage 13.0 (TID 285). 2208 bytes result sent to driver -26/04/01 08:38:46 INFO TaskSetManager: Starting task 23.0 in stage 13.0 (TID 293) (10.0.0.133, executor driver, partition 23, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:46 INFO TaskSetManager: Finished task 15.0 in stage 13.0 (TID 285) in 763 ms on 10.0.0.133 (executor driver) (16/64) -26/04/01 08:38:46 INFO Executor: Running task 23.0 in stage 13.0 (TID 293) -26/04/01 08:38:46 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.17.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:46 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:47 INFO Executor: Finished task 16.0 in stage 13.0 (TID 286). 2251 bytes result sent to driver -26/04/01 08:38:47 INFO TaskSetManager: Starting task 24.0 in stage 13.0 (TID 294) (10.0.0.133, executor driver, partition 24, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:47 INFO TaskSetManager: Finished task 16.0 in stage 13.0 (TID 286) in 634 ms on 10.0.0.133 (executor driver) (17/64) -26/04/01 08:38:47 INFO Executor: Running task 24.0 in stage 13.0 (TID 294) -26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.3.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:47 INFO Executor: Finished task 18.0 in stage 13.0 (TID 288). 2251 bytes result sent to driver -26/04/01 08:38:47 INFO TaskSetManager: Starting task 25.0 in stage 13.0 (TID 295) (10.0.0.133, executor driver, partition 25, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:47 INFO TaskSetManager: Finished task 18.0 in stage 13.0 (TID 288) in 873 ms on 10.0.0.133 (executor driver) (18/64) -26/04/01 08:38:47 INFO Executor: Running task 25.0 in stage 13.0 (TID 295) -26/04/01 08:38:47 INFO Executor: Finished task 19.0 in stage 13.0 (TID 289). 2251 bytes result sent to driver -26/04/01 08:38:47 INFO TaskSetManager: Starting task 26.0 in stage 13.0 (TID 296) (10.0.0.133, executor driver, partition 26, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:47 INFO TaskSetManager: Finished task 19.0 in stage 13.0 (TID 289) in 872 ms on 10.0.0.133 (executor driver) (19/64) -26/04/01 08:38:47 INFO Executor: Running task 26.0 in stage 13.0 (TID 296) -26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.31.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.21.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:47 INFO Executor: Finished task 17.0 in stage 13.0 (TID 287). 2251 bytes result sent to driver -26/04/01 08:38:47 INFO TaskSetManager: Starting task 27.0 in stage 13.0 (TID 297) (10.0.0.133, executor driver, partition 27, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:47 INFO Executor: Running task 27.0 in stage 13.0 (TID 297) -26/04/01 08:38:47 INFO TaskSetManager: Finished task 17.0 in stage 13.0 (TID 287) in 886 ms on 10.0.0.133 (executor driver) (20/64) -26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.20.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:47 INFO Executor: Finished task 20.0 in stage 13.0 (TID 290). 2294 bytes result sent to driver -26/04/01 08:38:47 INFO TaskSetManager: Starting task 28.0 in stage 13.0 (TID 298) (10.0.0.133, executor driver, partition 28, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:47 INFO TaskSetManager: Finished task 20.0 in stage 13.0 (TID 290) in 913 ms on 10.0.0.133 (executor driver) (21/64) -26/04/01 08:38:47 INFO Executor: Running task 28.0 in stage 13.0 (TID 298) -26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.30.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:47 INFO Executor: Finished task 23.0 in stage 13.0 (TID 293). 2251 bytes result sent to driver -26/04/01 08:38:47 INFO TaskSetManager: Starting task 29.0 in stage 13.0 (TID 299) (10.0.0.133, executor driver, partition 29, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:47 INFO TaskSetManager: Finished task 23.0 in stage 13.0 (TID 293) in 834 ms on 10.0.0.133 (executor driver) (22/64) -26/04/01 08:38:47 INFO Executor: Running task 29.0 in stage 13.0 (TID 299) -26/04/01 08:38:47 INFO Executor: Finished task 21.0 in stage 13.0 (TID 291). 2251 bytes result sent to driver -26/04/01 08:38:47 INFO TaskSetManager: Starting task 30.0 in stage 13.0 (TID 300) (10.0.0.133, executor driver, partition 30, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:47 INFO Executor: Running task 30.0 in stage 13.0 (TID 300) -26/04/01 08:38:47 INFO TaskSetManager: Finished task 21.0 in stage 13.0 (TID 291) in 877 ms on 10.0.0.133 (executor driver) (23/64) -26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.29.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.16.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:47 INFO Executor: Finished task 22.0 in stage 13.0 (TID 292). 2251 bytes result sent to driver -26/04/01 08:38:47 INFO TaskSetManager: Starting task 31.0 in stage 13.0 (TID 301) (10.0.0.133, executor driver, partition 31, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:47 INFO TaskSetManager: Finished task 22.0 in stage 13.0 (TID 292) in 870 ms on 10.0.0.133 (executor driver) (24/64) -26/04/01 08:38:47 INFO Executor: Running task 31.0 in stage 13.0 (TID 301) -26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.2.parquet, range: 0-134217728, partition values: [empty row] -26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:47 INFO Executor: Finished task 24.0 in stage 13.0 (TID 294). 2251 bytes result sent to driver -26/04/01 08:38:47 INFO TaskSetManager: Starting task 32.0 in stage 13.0 (TID 302) (10.0.0.133, executor driver, partition 32, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:47 INFO TaskSetManager: Finished task 24.0 in stage 13.0 (TID 294) in 746 ms on 10.0.0.133 (executor driver) (25/64) -26/04/01 08:38:47 INFO Executor: Running task 32.0 in stage 13.0 (TID 302) -26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.11.parquet, range: 134217728-233581632, partition values: [empty row] -26/04/01 08:38:47 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:47 INFO Executor: Finished task 26.0 in stage 13.0 (TID 296). 2251 bytes result sent to driver -26/04/01 08:38:47 INFO TaskSetManager: Starting task 33.0 in stage 13.0 (TID 303) (10.0.0.133, executor driver, partition 33, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:47 INFO TaskSetManager: Finished task 26.0 in stage 13.0 (TID 296) in 645 ms on 10.0.0.133 (executor driver) (26/64) -26/04/01 08:38:47 INFO Executor: Running task 33.0 in stage 13.0 (TID 303) -26/04/01 08:38:47 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.23.parquet, range: 134217728-233580600, partition values: [empty row] -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:48 INFO Executor: Finished task 25.0 in stage 13.0 (TID 295). 2251 bytes result sent to driver -26/04/01 08:38:48 INFO TaskSetManager: Starting task 34.0 in stage 13.0 (TID 304) (10.0.0.133, executor driver, partition 34, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:48 INFO TaskSetManager: Finished task 25.0 in stage 13.0 (TID 295) in 682 ms on 10.0.0.133 (executor driver) (27/64) -26/04/01 08:38:48 INFO Executor: Running task 34.0 in stage 13.0 (TID 304) -26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.30.parquet, range: 134217728-233577812, partition values: [empty row] -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:48 INFO Executor: Finished task 27.0 in stage 13.0 (TID 297). 2251 bytes result sent to driver -26/04/01 08:38:48 INFO TaskSetManager: Starting task 35.0 in stage 13.0 (TID 305) (10.0.0.133, executor driver, partition 35, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:48 INFO TaskSetManager: Finished task 27.0 in stage 13.0 (TID 297) in 696 ms on 10.0.0.133 (executor driver) (28/64) -26/04/01 08:38:48 INFO Executor: Running task 35.0 in stage 13.0 (TID 305) -26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.7.parquet, range: 134217728-233566955, partition values: [empty row] -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:48 INFO Executor: Finished task 28.0 in stage 13.0 (TID 298). 2251 bytes result sent to driver -26/04/01 08:38:48 INFO TaskSetManager: Starting task 36.0 in stage 13.0 (TID 306) (10.0.0.133, executor driver, partition 36, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:48 INFO TaskSetManager: Finished task 28.0 in stage 13.0 (TID 298) in 699 ms on 10.0.0.133 (executor driver) (29/64) -26/04/01 08:38:48 INFO Executor: Running task 36.0 in stage 13.0 (TID 306) -26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.12.parquet, range: 134217728-233565571, partition values: [empty row] -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:48 INFO Executor: Finished task 30.0 in stage 13.0 (TID 300). 2251 bytes result sent to driver -26/04/01 08:38:48 INFO TaskSetManager: Starting task 37.0 in stage 13.0 (TID 307) (10.0.0.133, executor driver, partition 37, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:48 INFO TaskSetManager: Finished task 30.0 in stage 13.0 (TID 300) in 759 ms on 10.0.0.133 (executor driver) (30/64) -26/04/01 08:38:48 INFO Executor: Running task 37.0 in stage 13.0 (TID 307) -26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.16.parquet, range: 134217728-233564116, partition values: [empty row] -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:48 INFO Executor: Finished task 31.0 in stage 13.0 (TID 301). 2251 bytes result sent to driver -26/04/01 08:38:48 INFO TaskSetManager: Starting task 38.0 in stage 13.0 (TID 308) (10.0.0.133, executor driver, partition 38, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:48 INFO Executor: Running task 38.0 in stage 13.0 (TID 308) -26/04/01 08:38:48 INFO TaskSetManager: Finished task 31.0 in stage 13.0 (TID 301) in 771 ms on 10.0.0.133 (executor driver) (31/64) -26/04/01 08:38:48 INFO Executor: Finished task 29.0 in stage 13.0 (TID 299). 2251 bytes result sent to driver -26/04/01 08:38:48 INFO TaskSetManager: Starting task 39.0 in stage 13.0 (TID 309) (10.0.0.133, executor driver, partition 39, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:48 INFO TaskSetManager: Finished task 29.0 in stage 13.0 (TID 299) in 773 ms on 10.0.0.133 (executor driver) (32/64) -26/04/01 08:38:48 INFO Executor: Running task 39.0 in stage 13.0 (TID 309) -26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.32.parquet, range: 134217728-233563384, partition values: [empty row] -26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.14.parquet, range: 134217728-233562716, partition values: [empty row] -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:48 INFO Executor: Finished task 32.0 in stage 13.0 (TID 302). 2208 bytes result sent to driver -26/04/01 08:38:48 INFO TaskSetManager: Starting task 40.0 in stage 13.0 (TID 310) (10.0.0.133, executor driver, partition 40, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:48 INFO TaskSetManager: Finished task 32.0 in stage 13.0 (TID 302) in 517 ms on 10.0.0.133 (executor driver) (33/64) -26/04/01 08:38:48 INFO Executor: Running task 40.0 in stage 13.0 (TID 310) -26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.5.parquet, range: 134217728-233556822, partition values: [empty row] -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:48 INFO Executor: Finished task 33.0 in stage 13.0 (TID 303). 2208 bytes result sent to driver -26/04/01 08:38:48 INFO TaskSetManager: Starting task 41.0 in stage 13.0 (TID 311) (10.0.0.133, executor driver, partition 41, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:48 INFO TaskSetManager: Finished task 33.0 in stage 13.0 (TID 303) in 525 ms on 10.0.0.133 (executor driver) (34/64) -26/04/01 08:38:48 INFO Executor: Running task 41.0 in stage 13.0 (TID 311) -26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.27.parquet, range: 134217728-233556807, partition values: [empty row] -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:48 INFO Executor: Finished task 34.0 in stage 13.0 (TID 304). 2208 bytes result sent to driver -26/04/01 08:38:48 INFO TaskSetManager: Starting task 42.0 in stage 13.0 (TID 312) (10.0.0.133, executor driver, partition 42, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:48 INFO TaskSetManager: Finished task 34.0 in stage 13.0 (TID 304) in 537 ms on 10.0.0.133 (executor driver) (35/64) -26/04/01 08:38:48 INFO Executor: Running task 42.0 in stage 13.0 (TID 312) -26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.19.parquet, range: 134217728-233556337, partition values: [empty row] -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:48 INFO Executor: Finished task 35.0 in stage 13.0 (TID 305). 2208 bytes result sent to driver -26/04/01 08:38:48 INFO TaskSetManager: Starting task 43.0 in stage 13.0 (TID 313) (10.0.0.133, executor driver, partition 43, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:48 INFO TaskSetManager: Finished task 35.0 in stage 13.0 (TID 305) in 520 ms on 10.0.0.133 (executor driver) (36/64) -26/04/01 08:38:48 INFO Executor: Running task 43.0 in stage 13.0 (TID 313) -26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.1.parquet, range: 134217728-233553544, partition values: [empty row] -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:48 INFO Executor: Finished task 36.0 in stage 13.0 (TID 306). 2208 bytes result sent to driver -26/04/01 08:38:48 INFO TaskSetManager: Starting task 44.0 in stage 13.0 (TID 314) (10.0.0.133, executor driver, partition 44, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:48 INFO TaskSetManager: Finished task 36.0 in stage 13.0 (TID 306) in 533 ms on 10.0.0.133 (executor driver) (37/64) -26/04/01 08:38:48 INFO Executor: Running task 44.0 in stage 13.0 (TID 314) -26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.6.parquet, range: 134217728-233549987, partition values: [empty row] -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:48 INFO Executor: Finished task 37.0 in stage 13.0 (TID 307). 2251 bytes result sent to driver -26/04/01 08:38:48 INFO TaskSetManager: Starting task 45.0 in stage 13.0 (TID 315) (10.0.0.133, executor driver, partition 45, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:48 INFO TaskSetManager: Finished task 37.0 in stage 13.0 (TID 307) in 536 ms on 10.0.0.133 (executor driver) (38/64) -26/04/01 08:38:48 INFO Executor: Running task 45.0 in stage 13.0 (TID 315) -26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.21.parquet, range: 134217728-233548593, partition values: [empty row] -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:48 INFO Executor: Finished task 39.0 in stage 13.0 (TID 309). 2251 bytes result sent to driver -26/04/01 08:38:48 INFO TaskSetManager: Starting task 46.0 in stage 13.0 (TID 316) (10.0.0.133, executor driver, partition 46, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:48 INFO Executor: Running task 46.0 in stage 13.0 (TID 316) -26/04/01 08:38:48 INFO TaskSetManager: Finished task 39.0 in stage 13.0 (TID 309) in 563 ms on 10.0.0.133 (executor driver) (39/64) -26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.28.parquet, range: 134217728-233545450, partition values: [empty row] -26/04/01 08:38:48 INFO Executor: Finished task 40.0 in stage 13.0 (TID 310). 2251 bytes result sent to driver -26/04/01 08:38:48 INFO TaskSetManager: Starting task 47.0 in stage 13.0 (TID 317) (10.0.0.133, executor driver, partition 47, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:48 INFO TaskSetManager: Finished task 40.0 in stage 13.0 (TID 310) in 549 ms on 10.0.0.133 (executor driver) (40/64) -26/04/01 08:38:48 INFO Executor: Running task 47.0 in stage 13.0 (TID 317) -26/04/01 08:38:48 INFO Executor: Finished task 38.0 in stage 13.0 (TID 308). 2251 bytes result sent to driver -26/04/01 08:38:48 INFO TaskSetManager: Starting task 48.0 in stage 13.0 (TID 318) (10.0.0.133, executor driver, partition 48, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:48 INFO TaskSetManager: Finished task 38.0 in stage 13.0 (TID 308) in 565 ms on 10.0.0.133 (executor driver) (41/64) -26/04/01 08:38:48 INFO Executor: Running task 48.0 in stage 13.0 (TID 318) -26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.4.parquet, range: 134217728-233545407, partition values: [empty row] -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:48 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.29.parquet, range: 134217728-233542751, partition values: [empty row] -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:48 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO Executor: Finished task 41.0 in stage 13.0 (TID 311). 2251 bytes result sent to driver -26/04/01 08:38:49 INFO TaskSetManager: Starting task 49.0 in stage 13.0 (TID 319) (10.0.0.133, executor driver, partition 49, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 41.0 in stage 13.0 (TID 311) in 482 ms on 10.0.0.133 (executor driver) (42/64) -26/04/01 08:38:49 INFO Executor: Running task 49.0 in stage 13.0 (TID 319) -26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.24.parquet, range: 134217728-233541467, partition values: [empty row] -26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO Executor: Finished task 43.0 in stage 13.0 (TID 313). 2251 bytes result sent to driver -26/04/01 08:38:49 INFO Executor: Finished task 42.0 in stage 13.0 (TID 312). 2251 bytes result sent to driver -26/04/01 08:38:49 INFO TaskSetManager: Starting task 50.0 in stage 13.0 (TID 320) (10.0.0.133, executor driver, partition 50, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:49 INFO TaskSetManager: Starting task 51.0 in stage 13.0 (TID 321) (10.0.0.133, executor driver, partition 51, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:49 INFO Executor: Running task 50.0 in stage 13.0 (TID 320) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 43.0 in stage 13.0 (TID 313) in 477 ms on 10.0.0.133 (executor driver) (43/64) -26/04/01 08:38:49 INFO Executor: Running task 51.0 in stage 13.0 (TID 321) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 42.0 in stage 13.0 (TID 312) in 488 ms on 10.0.0.133 (executor driver) (44/64) -26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.20.parquet, range: 134217728-233539473, partition values: [empty row] -26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.26.parquet, range: 134217728-233538804, partition values: [empty row] -26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO Executor: Finished task 44.0 in stage 13.0 (TID 314). 2251 bytes result sent to driver -26/04/01 08:38:49 INFO TaskSetManager: Starting task 52.0 in stage 13.0 (TID 322) (10.0.0.133, executor driver, partition 52, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:49 INFO Executor: Running task 52.0 in stage 13.0 (TID 322) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 44.0 in stage 13.0 (TID 314) in 457 ms on 10.0.0.133 (executor driver) (45/64) -26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.10.parquet, range: 134217728-233538598, partition values: [empty row] -26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO Executor: Finished task 45.0 in stage 13.0 (TID 315). 2208 bytes result sent to driver -26/04/01 08:38:49 INFO TaskSetManager: Starting task 53.0 in stage 13.0 (TID 323) (10.0.0.133, executor driver, partition 53, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 45.0 in stage 13.0 (TID 315) in 448 ms on 10.0.0.133 (executor driver) (46/64) -26/04/01 08:38:49 INFO Executor: Running task 53.0 in stage 13.0 (TID 323) -26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.13.parquet, range: 134217728-233537986, partition values: [empty row] -26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO Executor: Finished task 47.0 in stage 13.0 (TID 317). 2208 bytes result sent to driver -26/04/01 08:38:49 INFO TaskSetManager: Starting task 54.0 in stage 13.0 (TID 324) (10.0.0.133, executor driver, partition 54, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 47.0 in stage 13.0 (TID 317) in 505 ms on 10.0.0.133 (executor driver) (47/64) -26/04/01 08:38:49 INFO Executor: Running task 54.0 in stage 13.0 (TID 324) -26/04/01 08:38:49 INFO Executor: Finished task 46.0 in stage 13.0 (TID 316). 2208 bytes result sent to driver -26/04/01 08:38:49 INFO TaskSetManager: Starting task 55.0 in stage 13.0 (TID 325) (10.0.0.133, executor driver, partition 55, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 46.0 in stage 13.0 (TID 316) in 508 ms on 10.0.0.133 (executor driver) (48/64) -26/04/01 08:38:49 INFO Executor: Running task 55.0 in stage 13.0 (TID 325) -26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.8.parquet, range: 134217728-233536463, partition values: [empty row] -26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.22.parquet, range: 134217728-233536356, partition values: [empty row] -26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO Executor: Finished task 48.0 in stage 13.0 (TID 318). 2208 bytes result sent to driver -26/04/01 08:38:49 INFO TaskSetManager: Starting task 56.0 in stage 13.0 (TID 326) (10.0.0.133, executor driver, partition 56, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:49 INFO Executor: Running task 56.0 in stage 13.0 (TID 326) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 48.0 in stage 13.0 (TID 318) in 516 ms on 10.0.0.133 (executor driver) (49/64) -26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.9.parquet, range: 134217728-233535871, partition values: [empty row] -26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO Executor: Finished task 49.0 in stage 13.0 (TID 319). 2208 bytes result sent to driver -26/04/01 08:38:49 INFO TaskSetManager: Starting task 57.0 in stage 13.0 (TID 327) (10.0.0.133, executor driver, partition 57, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:49 INFO Executor: Running task 57.0 in stage 13.0 (TID 327) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 49.0 in stage 13.0 (TID 319) in 499 ms on 10.0.0.133 (executor driver) (50/64) -26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.31.parquet, range: 134217728-233535690, partition values: [empty row] -26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO Executor: Finished task 50.0 in stage 13.0 (TID 320). 2208 bytes result sent to driver -26/04/01 08:38:49 INFO Executor: Finished task 51.0 in stage 13.0 (TID 321). 2208 bytes result sent to driver -26/04/01 08:38:49 INFO TaskSetManager: Starting task 58.0 in stage 13.0 (TID 328) (10.0.0.133, executor driver, partition 58, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 50.0 in stage 13.0 (TID 320) in 516 ms on 10.0.0.133 (executor driver) (51/64) -26/04/01 08:38:49 INFO Executor: Running task 58.0 in stage 13.0 (TID 328) -26/04/01 08:38:49 INFO TaskSetManager: Starting task 59.0 in stage 13.0 (TID 329) (10.0.0.133, executor driver, partition 59, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:49 INFO Executor: Running task 59.0 in stage 13.0 (TID 329) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 51.0 in stage 13.0 (TID 321) in 516 ms on 10.0.0.133 (executor driver) (52/64) -26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.17.parquet, range: 134217728-233534488, partition values: [empty row] -26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.18.parquet, range: 134217728-233534883, partition values: [empty row] -26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO Executor: Finished task 52.0 in stage 13.0 (TID 322). 2251 bytes result sent to driver -26/04/01 08:38:49 INFO TaskSetManager: Starting task 60.0 in stage 13.0 (TID 330) (10.0.0.133, executor driver, partition 60, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:49 INFO Executor: Running task 60.0 in stage 13.0 (TID 330) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 52.0 in stage 13.0 (TID 322) in 500 ms on 10.0.0.133 (executor driver) (53/64) -26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.15.parquet, range: 134217728-233533044, partition values: [empty row] -26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO Executor: Finished task 53.0 in stage 13.0 (TID 323). 2251 bytes result sent to driver -26/04/01 08:38:49 INFO TaskSetManager: Starting task 61.0 in stage 13.0 (TID 331) (10.0.0.133, executor driver, partition 61, PROCESS_LOCAL, 9598 bytes) -26/04/01 08:38:49 INFO Executor: Running task 61.0 in stage 13.0 (TID 331) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 53.0 in stage 13.0 (TID 323) in 497 ms on 10.0.0.133 (executor driver) (54/64) -26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.25.parquet, range: 134217728-233532173, partition values: [empty row] -26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO Executor: Finished task 54.0 in stage 13.0 (TID 324). 2251 bytes result sent to driver -26/04/01 08:38:49 INFO TaskSetManager: Starting task 62.0 in stage 13.0 (TID 332) (10.0.0.133, executor driver, partition 62, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 54.0 in stage 13.0 (TID 324) in 579 ms on 10.0.0.133 (executor driver) (55/64) -26/04/01 08:38:49 INFO Executor: Running task 62.0 in stage 13.0 (TID 332) -26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.3.parquet, range: 134217728-233524702, partition values: [empty row] -26/04/01 08:38:49 INFO Executor: Finished task 55.0 in stage 13.0 (TID 325). 2251 bytes result sent to driver -26/04/01 08:38:49 INFO TaskSetManager: Starting task 63.0 in stage 13.0 (TID 333) (10.0.0.133, executor driver, partition 63, PROCESS_LOCAL, 9597 bytes) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 55.0 in stage 13.0 (TID 325) in 580 ms on 10.0.0.133 (executor driver) (56/64) -26/04/01 08:38:49 INFO Executor: Running task 63.0 in stage 13.0 (TID 333) -26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO FileScanRDD: Reading File path: file:///opt/tpch/sf100/orders/orders.2.parquet, range: 134217728-233511585, partition values: [empty row] -26/04/01 08:38:49 INFO FilterCompat: Filtering using predicate: noteq(o_orderkey, null) -26/04/01 08:38:49 INFO Executor: Finished task 56.0 in stage 13.0 (TID 326). 2251 bytes result sent to driver -26/04/01 08:38:49 INFO TaskSetManager: Starting task 0.0 in stage 16.0 (TID 334) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:49 INFO Executor: Running task 0.0 in stage 16.0 (TID 334) -26/04/01 08:38:49 INFO TaskSetManager: Finished task 56.0 in stage 13.0 (TID 326) in 600 ms on 10.0.0.133 (executor driver) (57/64) -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 8 ms -26/04/01 08:38:50 INFO CodeGenerator: Code generated in 3.042541 ms -26/04/01 08:38:50 INFO CodeGenerator: Code generated in 3.434042 ms -26/04/01 08:38:50 INFO CodeGenerator: Code generated in 1.618833 ms -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:50 INFO Executor: Finished task 57.0 in stage 13.0 (TID 327). 2251 bytes result sent to driver -26/04/01 08:38:50 INFO TaskSetManager: Starting task 1.0 in stage 16.0 (TID 335) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:50 INFO TaskSetManager: Finished task 57.0 in stage 13.0 (TID 327) in 577 ms on 10.0.0.133 (executor driver) (58/64) -26/04/01 08:38:50 INFO Executor: Running task 1.0 in stage 16.0 (TID 335) -26/04/01 08:38:50 INFO Executor: Finished task 58.0 in stage 13.0 (TID 328). 2251 bytes result sent to driver -26/04/01 08:38:50 INFO TaskSetManager: Starting task 2.0 in stage 16.0 (TID 336) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:50 INFO Executor: Running task 2.0 in stage 16.0 (TID 336) -26/04/01 08:38:50 INFO TaskSetManager: Finished task 58.0 in stage 13.0 (TID 328) in 507 ms on 10.0.0.133 (executor driver) (59/64) -26/04/01 08:38:50 INFO Executor: Finished task 59.0 in stage 13.0 (TID 329). 2251 bytes result sent to driver -26/04/01 08:38:50 INFO TaskSetManager: Starting task 3.0 in stage 16.0 (TID 337) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:50 INFO TaskSetManager: Finished task 59.0 in stage 13.0 (TID 329) in 508 ms on 10.0.0.133 (executor driver) (60/64) -26/04/01 08:38:50 INFO Executor: Running task 3.0 in stage 16.0 (TID 337) -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:50 INFO CodeGenerator: Code generated in 3.219 ms -26/04/01 08:38:50 INFO CodeGenerator: Code generated in 1.6445 ms -26/04/01 08:38:50 INFO CodeGenerator: Code generated in 1.843458 ms -26/04/01 08:38:50 INFO CodeGenerator: Code generated in 4.8145 ms -26/04/01 08:38:50 INFO CodeGenerator: Code generated in 2.479792 ms -26/04/01 08:38:50 INFO Executor: Finished task 60.0 in stage 13.0 (TID 330). 2251 bytes result sent to driver -26/04/01 08:38:50 INFO TaskSetManager: Starting task 4.0 in stage 16.0 (TID 338) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:50 INFO Executor: Running task 4.0 in stage 16.0 (TID 338) -26/04/01 08:38:50 INFO TaskSetManager: Finished task 60.0 in stage 13.0 (TID 330) in 515 ms on 10.0.0.133 (executor driver) (61/64) -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:50 INFO Executor: Finished task 61.0 in stage 13.0 (TID 331). 2208 bytes result sent to driver -26/04/01 08:38:50 INFO TaskSetManager: Starting task 5.0 in stage 16.0 (TID 339) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:50 INFO Executor: Running task 5.0 in stage 16.0 (TID 339) -26/04/01 08:38:50 INFO TaskSetManager: Finished task 61.0 in stage 13.0 (TID 331) in 465 ms on 10.0.0.133 (executor driver) (62/64) -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:50 INFO Executor: Finished task 62.0 in stage 13.0 (TID 332). 2208 bytes result sent to driver -26/04/01 08:38:50 INFO TaskSetManager: Starting task 6.0 in stage 16.0 (TID 340) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:50 INFO Executor: Running task 6.0 in stage 16.0 (TID 340) -26/04/01 08:38:50 INFO TaskSetManager: Finished task 62.0 in stage 13.0 (TID 332) in 462 ms on 10.0.0.133 (executor driver) (63/64) -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:50 INFO Executor: Finished task 63.0 in stage 13.0 (TID 333). 2208 bytes result sent to driver -26/04/01 08:38:50 INFO TaskSetManager: Starting task 7.0 in stage 16.0 (TID 341) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:50 INFO TaskSetManager: Finished task 63.0 in stage 13.0 (TID 333) in 474 ms on 10.0.0.133 (executor driver) (64/64) -26/04/01 08:38:50 INFO Executor: Running task 7.0 in stage 16.0 (TID 341) -26/04/01 08:38:50 INFO TaskSchedulerImpl: Removed TaskSet 13.0, whose tasks have all completed, from pool -26/04/01 08:38:50 INFO DAGScheduler: ShuffleMapStage 13 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 43.175 s -26/04/01 08:38:50 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:38:50 INFO DAGScheduler: running: Set(ShuffleMapStage 16) -26/04/01 08:38:50 INFO DAGScheduler: waiting: Set() -26/04/01 08:38:50 INFO DAGScheduler: failed: Set() -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:50 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO Executor: Finished task 3.0 in stage 16.0 (TID 337). 7266 bytes result sent to driver -26/04/01 08:38:51 INFO TaskSetManager: Starting task 8.0 in stage 16.0 (TID 342) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:51 INFO TaskSetManager: Finished task 3.0 in stage 16.0 (TID 337) in 1102 ms on 10.0.0.133 (executor driver) (1/200) -26/04/01 08:38:51 INFO Executor: Running task 8.0 in stage 16.0 (TID 342) -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO Executor: Finished task 5.0 in stage 16.0 (TID 339). 7266 bytes result sent to driver -26/04/01 08:38:51 INFO TaskSetManager: Starting task 9.0 in stage 16.0 (TID 343) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:51 INFO Executor: Running task 9.0 in stage 16.0 (TID 343) -26/04/01 08:38:51 INFO TaskSetManager: Finished task 5.0 in stage 16.0 (TID 339) in 1055 ms on 10.0.0.133 (executor driver) (2/200) -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO Executor: Finished task 4.0 in stage 16.0 (TID 338). 7266 bytes result sent to driver -26/04/01 08:38:51 INFO TaskSetManager: Starting task 10.0 in stage 16.0 (TID 344) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:51 INFO TaskSetManager: Finished task 4.0 in stage 16.0 (TID 338) in 1101 ms on 10.0.0.133 (executor driver) (3/200) -26/04/01 08:38:51 INFO Executor: Running task 10.0 in stage 16.0 (TID 344) -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO Executor: Finished task 2.0 in stage 16.0 (TID 336). 7266 bytes result sent to driver -26/04/01 08:38:51 INFO TaskSetManager: Starting task 11.0 in stage 16.0 (TID 345) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:51 INFO TaskSetManager: Finished task 2.0 in stage 16.0 (TID 336) in 1277 ms on 10.0.0.133 (executor driver) (4/200) -26/04/01 08:38:51 INFO Executor: Running task 11.0 in stage 16.0 (TID 345) -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (28.1 KiB) non-empty blocks including 8 (28.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO Executor: Finished task 6.0 in stage 16.0 (TID 340). 7266 bytes result sent to driver -26/04/01 08:38:51 INFO TaskSetManager: Starting task 12.0 in stage 16.0 (TID 346) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:51 INFO TaskSetManager: Finished task 6.0 in stage 16.0 (TID 340) in 948 ms on 10.0.0.133 (executor driver) (5/200) -26/04/01 08:38:51 INFO Executor: Running task 12.0 in stage 16.0 (TID 346) -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO Executor: Finished task 1.0 in stage 16.0 (TID 335). 7266 bytes result sent to driver -26/04/01 08:38:51 INFO TaskSetManager: Starting task 13.0 in stage 16.0 (TID 347) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:51 INFO Executor: Running task 13.0 in stage 16.0 (TID 347) -26/04/01 08:38:51 INFO TaskSetManager: Finished task 1.0 in stage 16.0 (TID 335) in 1297 ms on 10.0.0.133 (executor driver) (6/200) -26/04/01 08:38:51 INFO Executor: Finished task 0.0 in stage 16.0 (TID 334). 7266 bytes result sent to driver -26/04/01 08:38:51 INFO TaskSetManager: Starting task 14.0 in stage 16.0 (TID 348) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:51 INFO Executor: Running task 14.0 in stage 16.0 (TID 348) -26/04/01 08:38:51 INFO TaskSetManager: Finished task 0.0 in stage 16.0 (TID 334) in 1382 ms on 10.0.0.133 (executor driver) (7/200) -26/04/01 08:38:51 INFO Executor: Finished task 7.0 in stage 16.0 (TID 341). 7223 bytes result sent to driver -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO TaskSetManager: Starting task 15.0 in stage 16.0 (TID 349) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:51 INFO TaskSetManager: Finished task 7.0 in stage 16.0 (TID 341) in 938 ms on 10.0.0.133 (executor driver) (8/200) -26/04/01 08:38:51 INFO Executor: Running task 15.0 in stage 16.0 (TID 349) -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO Executor: Finished task 8.0 in stage 16.0 (TID 342). 7309 bytes result sent to driver -26/04/01 08:38:51 INFO TaskSetManager: Starting task 16.0 in stage 16.0 (TID 350) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:51 INFO TaskSetManager: Finished task 8.0 in stage 16.0 (TID 342) in 722 ms on 10.0.0.133 (executor driver) (9/200) -26/04/01 08:38:51 INFO Executor: Running task 16.0 in stage 16.0 (TID 350) -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:51 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO Executor: Finished task 10.0 in stage 16.0 (TID 344). 7309 bytes result sent to driver -26/04/01 08:38:52 INFO TaskSetManager: Starting task 17.0 in stage 16.0 (TID 351) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:52 INFO TaskSetManager: Finished task 10.0 in stage 16.0 (TID 344) in 795 ms on 10.0.0.133 (executor driver) (10/200) -26/04/01 08:38:52 INFO Executor: Running task 17.0 in stage 16.0 (TID 351) -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO Executor: Finished task 9.0 in stage 16.0 (TID 343). 7309 bytes result sent to driver -26/04/01 08:38:52 INFO TaskSetManager: Starting task 18.0 in stage 16.0 (TID 352) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:52 INFO TaskSetManager: Finished task 9.0 in stage 16.0 (TID 343) in 809 ms on 10.0.0.133 (executor driver) (11/200) -26/04/01 08:38:52 INFO Executor: Running task 18.0 in stage 16.0 (TID 352) -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO Executor: Finished task 11.0 in stage 16.0 (TID 345). 7309 bytes result sent to driver -26/04/01 08:38:52 INFO TaskSetManager: Starting task 19.0 in stage 16.0 (TID 353) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:52 INFO Executor: Running task 19.0 in stage 16.0 (TID 353) -26/04/01 08:38:52 INFO TaskSetManager: Finished task 11.0 in stage 16.0 (TID 345) in 793 ms on 10.0.0.133 (executor driver) (12/200) -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.5 KiB) non-empty blocks including 8 (30.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO Executor: Finished task 12.0 in stage 16.0 (TID 346). 7309 bytes result sent to driver -26/04/01 08:38:52 INFO TaskSetManager: Starting task 20.0 in stage 16.0 (TID 354) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:52 INFO TaskSetManager: Finished task 12.0 in stage 16.0 (TID 346) in 788 ms on 10.0.0.133 (executor driver) (13/200) -26/04/01 08:38:52 INFO Executor: Running task 20.0 in stage 16.0 (TID 354) -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO Executor: Finished task 13.0 in stage 16.0 (TID 347). 7266 bytes result sent to driver -26/04/01 08:38:52 INFO TaskSetManager: Starting task 21.0 in stage 16.0 (TID 355) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:52 INFO TaskSetManager: Finished task 13.0 in stage 16.0 (TID 347) in 792 ms on 10.0.0.133 (executor driver) (14/200) -26/04/01 08:38:52 INFO Executor: Running task 21.0 in stage 16.0 (TID 355) -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO Executor: Finished task 14.0 in stage 16.0 (TID 348). 7266 bytes result sent to driver -26/04/01 08:38:52 INFO TaskSetManager: Starting task 22.0 in stage 16.0 (TID 356) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:52 INFO TaskSetManager: Finished task 14.0 in stage 16.0 (TID 348) in 792 ms on 10.0.0.133 (executor driver) (15/200) -26/04/01 08:38:52 INFO Executor: Running task 22.0 in stage 16.0 (TID 356) -26/04/01 08:38:52 INFO Executor: Finished task 15.0 in stage 16.0 (TID 349). 7309 bytes result sent to driver -26/04/01 08:38:52 INFO TaskSetManager: Starting task 23.0 in stage 16.0 (TID 357) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:52 INFO Executor: Running task 23.0 in stage 16.0 (TID 357) -26/04/01 08:38:52 INFO TaskSetManager: Finished task 15.0 in stage 16.0 (TID 349) in 792 ms on 10.0.0.133 (executor driver) (16/200) -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO Executor: Finished task 16.0 in stage 16.0 (TID 350). 7266 bytes result sent to driver -26/04/01 08:38:52 INFO TaskSetManager: Starting task 24.0 in stage 16.0 (TID 358) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:52 INFO Executor: Running task 24.0 in stage 16.0 (TID 358) -26/04/01 08:38:52 INFO TaskSetManager: Finished task 16.0 in stage 16.0 (TID 350) in 728 ms on 10.0.0.133 (executor driver) (17/200) -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO Executor: Finished task 17.0 in stage 16.0 (TID 351). 7266 bytes result sent to driver -26/04/01 08:38:52 INFO TaskSetManager: Starting task 25.0 in stage 16.0 (TID 359) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:52 INFO TaskSetManager: Finished task 17.0 in stage 16.0 (TID 351) in 754 ms on 10.0.0.133 (executor driver) (18/200) -26/04/01 08:38:52 INFO Executor: Running task 25.0 in stage 16.0 (TID 359) -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO Executor: Finished task 18.0 in stage 16.0 (TID 352). 7223 bytes result sent to driver -26/04/01 08:38:52 INFO TaskSetManager: Starting task 26.0 in stage 16.0 (TID 360) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:52 INFO Executor: Running task 26.0 in stage 16.0 (TID 360) -26/04/01 08:38:52 INFO TaskSetManager: Finished task 18.0 in stage 16.0 (TID 352) in 766 ms on 10.0.0.133 (executor driver) (19/200) -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO Executor: Finished task 19.0 in stage 16.0 (TID 353). 7223 bytes result sent to driver -26/04/01 08:38:52 INFO TaskSetManager: Starting task 27.0 in stage 16.0 (TID 361) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:52 INFO TaskSetManager: Finished task 19.0 in stage 16.0 (TID 353) in 791 ms on 10.0.0.133 (executor driver) (20/200) -26/04/01 08:38:52 INFO Executor: Running task 27.0 in stage 16.0 (TID 361) -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO Executor: Finished task 21.0 in stage 16.0 (TID 355). 7223 bytes result sent to driver -26/04/01 08:38:52 INFO TaskSetManager: Starting task 28.0 in stage 16.0 (TID 362) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:52 INFO Executor: Running task 28.0 in stage 16.0 (TID 362) -26/04/01 08:38:52 INFO TaskSetManager: Finished task 21.0 in stage 16.0 (TID 355) in 790 ms on 10.0.0.133 (executor driver) (21/200) -26/04/01 08:38:52 INFO Executor: Finished task 20.0 in stage 16.0 (TID 354). 7223 bytes result sent to driver -26/04/01 08:38:52 INFO TaskSetManager: Starting task 29.0 in stage 16.0 (TID 363) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:52 INFO Executor: Running task 29.0 in stage 16.0 (TID 363) -26/04/01 08:38:52 INFO TaskSetManager: Finished task 20.0 in stage 16.0 (TID 354) in 797 ms on 10.0.0.133 (executor driver) (22/200) -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (30.2 KiB) non-empty blocks including 8 (30.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO Executor: Finished task 22.0 in stage 16.0 (TID 356). 7223 bytes result sent to driver -26/04/01 08:38:52 INFO TaskSetManager: Starting task 30.0 in stage 16.0 (TID 364) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:52 INFO TaskSetManager: Finished task 22.0 in stage 16.0 (TID 356) in 792 ms on 10.0.0.133 (executor driver) (23/200) -26/04/01 08:38:52 INFO Executor: Running task 30.0 in stage 16.0 (TID 364) -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO Executor: Finished task 23.0 in stage 16.0 (TID 357). 7223 bytes result sent to driver -26/04/01 08:38:52 INFO TaskSetManager: Starting task 31.0 in stage 16.0 (TID 365) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:52 INFO Executor: Running task 31.0 in stage 16.0 (TID 365) -26/04/01 08:38:52 INFO TaskSetManager: Finished task 23.0 in stage 16.0 (TID 357) in 793 ms on 10.0.0.133 (executor driver) (24/200) -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:52 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO Executor: Finished task 24.0 in stage 16.0 (TID 358). 7309 bytes result sent to driver -26/04/01 08:38:53 INFO TaskSetManager: Starting task 32.0 in stage 16.0 (TID 366) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:53 INFO Executor: Running task 32.0 in stage 16.0 (TID 366) -26/04/01 08:38:53 INFO TaskSetManager: Finished task 24.0 in stage 16.0 (TID 358) in 710 ms on 10.0.0.133 (executor driver) (25/200) -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO Executor: Finished task 25.0 in stage 16.0 (TID 359). 7309 bytes result sent to driver -26/04/01 08:38:53 INFO TaskSetManager: Starting task 33.0 in stage 16.0 (TID 367) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:53 INFO TaskSetManager: Finished task 25.0 in stage 16.0 (TID 359) in 871 ms on 10.0.0.133 (executor driver) (26/200) -26/04/01 08:38:53 INFO Executor: Running task 33.0 in stage 16.0 (TID 367) -26/04/01 08:38:53 INFO Executor: Finished task 26.0 in stage 16.0 (TID 360). 7309 bytes result sent to driver -26/04/01 08:38:53 INFO TaskSetManager: Starting task 34.0 in stage 16.0 (TID 368) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:53 INFO Executor: Running task 34.0 in stage 16.0 (TID 368) -26/04/01 08:38:53 INFO TaskSetManager: Finished task 26.0 in stage 16.0 (TID 360) in 851 ms on 10.0.0.133 (executor driver) (27/200) -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO Executor: Finished task 27.0 in stage 16.0 (TID 361). 7309 bytes result sent to driver -26/04/01 08:38:53 INFO TaskSetManager: Starting task 35.0 in stage 16.0 (TID 369) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:53 INFO Executor: Running task 35.0 in stage 16.0 (TID 369) -26/04/01 08:38:53 INFO TaskSetManager: Finished task 27.0 in stage 16.0 (TID 361) in 805 ms on 10.0.0.133 (executor driver) (28/200) -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO Executor: Finished task 28.0 in stage 16.0 (TID 362). 7309 bytes result sent to driver -26/04/01 08:38:53 INFO TaskSetManager: Starting task 36.0 in stage 16.0 (TID 370) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:53 INFO TaskSetManager: Finished task 28.0 in stage 16.0 (TID 362) in 801 ms on 10.0.0.133 (executor driver) (29/200) -26/04/01 08:38:53 INFO Executor: Running task 36.0 in stage 16.0 (TID 370) -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO Executor: Finished task 29.0 in stage 16.0 (TID 363). 7266 bytes result sent to driver -26/04/01 08:38:53 INFO TaskSetManager: Starting task 37.0 in stage 16.0 (TID 371) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:53 INFO TaskSetManager: Finished task 29.0 in stage 16.0 (TID 363) in 813 ms on 10.0.0.133 (executor driver) (30/200) -26/04/01 08:38:53 INFO Executor: Running task 37.0 in stage 16.0 (TID 371) -26/04/01 08:38:53 INFO Executor: Finished task 30.0 in stage 16.0 (TID 364). 7266 bytes result sent to driver -26/04/01 08:38:53 INFO TaskSetManager: Starting task 38.0 in stage 16.0 (TID 372) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:53 INFO Executor: Running task 38.0 in stage 16.0 (TID 372) -26/04/01 08:38:53 INFO TaskSetManager: Finished task 30.0 in stage 16.0 (TID 364) in 811 ms on 10.0.0.133 (executor driver) (31/200) -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (28.0 KiB) non-empty blocks including 8 (28.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO Executor: Finished task 31.0 in stage 16.0 (TID 365). 7266 bytes result sent to driver -26/04/01 08:38:53 INFO TaskSetManager: Starting task 39.0 in stage 16.0 (TID 373) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:53 INFO TaskSetManager: Finished task 31.0 in stage 16.0 (TID 365) in 812 ms on 10.0.0.133 (executor driver) (32/200) -26/04/01 08:38:53 INFO Executor: Running task 39.0 in stage 16.0 (TID 373) -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 8 (28.0 KiB) non-empty blocks including 8 (28.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:53 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO Executor: Finished task 32.0 in stage 16.0 (TID 366). 7266 bytes result sent to driver -26/04/01 08:38:54 INFO TaskSetManager: Starting task 40.0 in stage 16.0 (TID 374) (10.0.0.133, executor driver, partition 40, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:54 INFO Executor: Running task 40.0 in stage 16.0 (TID 374) -26/04/01 08:38:54 INFO TaskSetManager: Finished task 32.0 in stage 16.0 (TID 366) in 669 ms on 10.0.0.133 (executor driver) (33/200) -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO Executor: Finished task 33.0 in stage 16.0 (TID 367). 7266 bytes result sent to driver -26/04/01 08:38:54 INFO TaskSetManager: Starting task 41.0 in stage 16.0 (TID 375) (10.0.0.133, executor driver, partition 41, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:54 INFO Executor: Running task 41.0 in stage 16.0 (TID 375) -26/04/01 08:38:54 INFO TaskSetManager: Finished task 33.0 in stage 16.0 (TID 367) in 801 ms on 10.0.0.133 (executor driver) (34/200) -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO Executor: Finished task 34.0 in stage 16.0 (TID 368). 7266 bytes result sent to driver -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO TaskSetManager: Starting task 42.0 in stage 16.0 (TID 376) (10.0.0.133, executor driver, partition 42, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:54 INFO Executor: Running task 42.0 in stage 16.0 (TID 376) -26/04/01 08:38:54 INFO TaskSetManager: Finished task 34.0 in stage 16.0 (TID 368) in 804 ms on 10.0.0.133 (executor driver) (35/200) -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO Executor: Finished task 35.0 in stage 16.0 (TID 369). 7223 bytes result sent to driver -26/04/01 08:38:54 INFO TaskSetManager: Starting task 43.0 in stage 16.0 (TID 377) (10.0.0.133, executor driver, partition 43, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:54 INFO TaskSetManager: Finished task 35.0 in stage 16.0 (TID 369) in 798 ms on 10.0.0.133 (executor driver) (36/200) -26/04/01 08:38:54 INFO Executor: Running task 43.0 in stage 16.0 (TID 377) -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO Executor: Finished task 36.0 in stage 16.0 (TID 370). 7266 bytes result sent to driver -26/04/01 08:38:54 INFO TaskSetManager: Starting task 44.0 in stage 16.0 (TID 378) (10.0.0.133, executor driver, partition 44, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:54 INFO TaskSetManager: Finished task 36.0 in stage 16.0 (TID 370) in 788 ms on 10.0.0.133 (executor driver) (37/200) -26/04/01 08:38:54 INFO Executor: Running task 44.0 in stage 16.0 (TID 378) -26/04/01 08:38:54 INFO Executor: Finished task 38.0 in stage 16.0 (TID 372). 7223 bytes result sent to driver -26/04/01 08:38:54 INFO TaskSetManager: Starting task 45.0 in stage 16.0 (TID 379) (10.0.0.133, executor driver, partition 45, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:54 INFO Executor: Running task 45.0 in stage 16.0 (TID 379) -26/04/01 08:38:54 INFO TaskSetManager: Finished task 38.0 in stage 16.0 (TID 372) in 775 ms on 10.0.0.133 (executor driver) (38/200) -26/04/01 08:38:54 INFO Executor: Finished task 39.0 in stage 16.0 (TID 373). 7223 bytes result sent to driver -26/04/01 08:38:54 INFO TaskSetManager: Starting task 46.0 in stage 16.0 (TID 380) (10.0.0.133, executor driver, partition 46, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:54 INFO Executor: Running task 46.0 in stage 16.0 (TID 380) -26/04/01 08:38:54 INFO TaskSetManager: Finished task 39.0 in stage 16.0 (TID 373) in 774 ms on 10.0.0.133 (executor driver) (39/200) -26/04/01 08:38:54 INFO Executor: Finished task 37.0 in stage 16.0 (TID 371). 7266 bytes result sent to driver -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO TaskSetManager: Starting task 47.0 in stage 16.0 (TID 381) (10.0.0.133, executor driver, partition 47, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:54 INFO Executor: Running task 47.0 in stage 16.0 (TID 381) -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO TaskSetManager: Finished task 37.0 in stage 16.0 (TID 371) in 778 ms on 10.0.0.133 (executor driver) (40/200) -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO Executor: Finished task 40.0 in stage 16.0 (TID 374). 7266 bytes result sent to driver -26/04/01 08:38:54 INFO TaskSetManager: Starting task 48.0 in stage 16.0 (TID 382) (10.0.0.133, executor driver, partition 48, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:54 INFO Executor: Running task 48.0 in stage 16.0 (TID 382) -26/04/01 08:38:54 INFO TaskSetManager: Finished task 40.0 in stage 16.0 (TID 374) in 667 ms on 10.0.0.133 (executor driver) (41/200) -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:54 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO Executor: Finished task 42.0 in stage 16.0 (TID 376). 7266 bytes result sent to driver -26/04/01 08:38:55 INFO TaskSetManager: Starting task 49.0 in stage 16.0 (TID 383) (10.0.0.133, executor driver, partition 49, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:55 INFO TaskSetManager: Finished task 42.0 in stage 16.0 (TID 376) in 792 ms on 10.0.0.133 (executor driver) (42/200) -26/04/01 08:38:55 INFO Executor: Running task 49.0 in stage 16.0 (TID 383) -26/04/01 08:38:55 INFO Executor: Finished task 41.0 in stage 16.0 (TID 375). 7309 bytes result sent to driver -26/04/01 08:38:55 INFO TaskSetManager: Starting task 50.0 in stage 16.0 (TID 384) (10.0.0.133, executor driver, partition 50, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:55 INFO Executor: Running task 50.0 in stage 16.0 (TID 384) -26/04/01 08:38:55 INFO TaskSetManager: Finished task 41.0 in stage 16.0 (TID 375) in 796 ms on 10.0.0.133 (executor driver) (43/200) -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO Executor: Finished task 43.0 in stage 16.0 (TID 377). 7266 bytes result sent to driver -26/04/01 08:38:55 INFO TaskSetManager: Starting task 51.0 in stage 16.0 (TID 385) (10.0.0.133, executor driver, partition 51, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:55 INFO TaskSetManager: Finished task 43.0 in stage 16.0 (TID 377) in 816 ms on 10.0.0.133 (executor driver) (44/200) -26/04/01 08:38:55 INFO Executor: Running task 51.0 in stage 16.0 (TID 385) -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO Executor: Finished task 44.0 in stage 16.0 (TID 378). 7266 bytes result sent to driver -26/04/01 08:38:55 INFO TaskSetManager: Starting task 52.0 in stage 16.0 (TID 386) (10.0.0.133, executor driver, partition 52, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:55 INFO TaskSetManager: Finished task 44.0 in stage 16.0 (TID 378) in 825 ms on 10.0.0.133 (executor driver) (45/200) -26/04/01 08:38:55 INFO Executor: Running task 52.0 in stage 16.0 (TID 386) -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO Executor: Finished task 46.0 in stage 16.0 (TID 380). 7266 bytes result sent to driver -26/04/01 08:38:55 INFO TaskSetManager: Starting task 53.0 in stage 16.0 (TID 387) (10.0.0.133, executor driver, partition 53, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:55 INFO TaskSetManager: Finished task 46.0 in stage 16.0 (TID 380) in 833 ms on 10.0.0.133 (executor driver) (46/200) -26/04/01 08:38:55 INFO Executor: Running task 53.0 in stage 16.0 (TID 387) -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO Executor: Finished task 45.0 in stage 16.0 (TID 379). 7266 bytes result sent to driver -26/04/01 08:38:55 INFO TaskSetManager: Starting task 54.0 in stage 16.0 (TID 388) (10.0.0.133, executor driver, partition 54, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:55 INFO Executor: Running task 54.0 in stage 16.0 (TID 388) -26/04/01 08:38:55 INFO TaskSetManager: Finished task 45.0 in stage 16.0 (TID 379) in 837 ms on 10.0.0.133 (executor driver) (47/200) -26/04/01 08:38:55 INFO Executor: Finished task 47.0 in stage 16.0 (TID 381). 7266 bytes result sent to driver -26/04/01 08:38:55 INFO TaskSetManager: Starting task 55.0 in stage 16.0 (TID 389) (10.0.0.133, executor driver, partition 55, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:55 INFO TaskSetManager: Finished task 47.0 in stage 16.0 (TID 381) in 836 ms on 10.0.0.133 (executor driver) (48/200) -26/04/01 08:38:55 INFO Executor: Running task 55.0 in stage 16.0 (TID 389) -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO Executor: Finished task 48.0 in stage 16.0 (TID 382). 7266 bytes result sent to driver -26/04/01 08:38:55 INFO TaskSetManager: Starting task 56.0 in stage 16.0 (TID 390) (10.0.0.133, executor driver, partition 56, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:55 INFO Executor: Running task 56.0 in stage 16.0 (TID 390) -26/04/01 08:38:55 INFO TaskSetManager: Finished task 48.0 in stage 16.0 (TID 382) in 729 ms on 10.0.0.133 (executor driver) (49/200) -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:55 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO Executor: Finished task 49.0 in stage 16.0 (TID 383). 7266 bytes result sent to driver -26/04/01 08:38:56 INFO TaskSetManager: Starting task 57.0 in stage 16.0 (TID 391) (10.0.0.133, executor driver, partition 57, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:56 INFO Executor: Running task 57.0 in stage 16.0 (TID 391) -26/04/01 08:38:56 INFO TaskSetManager: Finished task 49.0 in stage 16.0 (TID 383) in 940 ms on 10.0.0.133 (executor driver) (50/200) -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO Executor: Finished task 51.0 in stage 16.0 (TID 385). 7223 bytes result sent to driver -26/04/01 08:38:56 INFO TaskSetManager: Starting task 58.0 in stage 16.0 (TID 392) (10.0.0.133, executor driver, partition 58, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:56 INFO Executor: Running task 58.0 in stage 16.0 (TID 392) -26/04/01 08:38:56 INFO TaskSetManager: Finished task 51.0 in stage 16.0 (TID 385) in 906 ms on 10.0.0.133 (executor driver) (51/200) -26/04/01 08:38:56 INFO Executor: Finished task 50.0 in stage 16.0 (TID 384). 7223 bytes result sent to driver -26/04/01 08:38:56 INFO TaskSetManager: Starting task 59.0 in stage 16.0 (TID 393) (10.0.0.133, executor driver, partition 59, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:56 INFO Executor: Running task 59.0 in stage 16.0 (TID 393) -26/04/01 08:38:56 INFO TaskSetManager: Finished task 50.0 in stage 16.0 (TID 384) in 942 ms on 10.0.0.133 (executor driver) (52/200) -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO Executor: Finished task 52.0 in stage 16.0 (TID 386). 7223 bytes result sent to driver -26/04/01 08:38:56 INFO TaskSetManager: Starting task 60.0 in stage 16.0 (TID 394) (10.0.0.133, executor driver, partition 60, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:56 INFO Executor: Running task 60.0 in stage 16.0 (TID 394) -26/04/01 08:38:56 INFO TaskSetManager: Finished task 52.0 in stage 16.0 (TID 386) in 898 ms on 10.0.0.133 (executor driver) (53/200) -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO Executor: Finished task 53.0 in stage 16.0 (TID 387). 7223 bytes result sent to driver -26/04/01 08:38:56 INFO TaskSetManager: Starting task 61.0 in stage 16.0 (TID 395) (10.0.0.133, executor driver, partition 61, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:56 INFO TaskSetManager: Finished task 53.0 in stage 16.0 (TID 387) in 891 ms on 10.0.0.133 (executor driver) (54/200) -26/04/01 08:38:56 INFO Executor: Running task 61.0 in stage 16.0 (TID 395) -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO Executor: Finished task 54.0 in stage 16.0 (TID 388). 7223 bytes result sent to driver -26/04/01 08:38:56 INFO TaskSetManager: Starting task 62.0 in stage 16.0 (TID 396) (10.0.0.133, executor driver, partition 62, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:56 INFO Executor: Running task 62.0 in stage 16.0 (TID 396) -26/04/01 08:38:56 INFO TaskSetManager: Finished task 54.0 in stage 16.0 (TID 388) in 890 ms on 10.0.0.133 (executor driver) (55/200) -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO Executor: Finished task 56.0 in stage 16.0 (TID 390). 7223 bytes result sent to driver -26/04/01 08:38:56 INFO TaskSetManager: Starting task 63.0 in stage 16.0 (TID 397) (10.0.0.133, executor driver, partition 63, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:56 INFO TaskSetManager: Finished task 56.0 in stage 16.0 (TID 390) in 873 ms on 10.0.0.133 (executor driver) (56/200) -26/04/01 08:38:56 INFO Executor: Running task 63.0 in stage 16.0 (TID 397) -26/04/01 08:38:56 INFO Executor: Finished task 55.0 in stage 16.0 (TID 389). 7223 bytes result sent to driver -26/04/01 08:38:56 INFO TaskSetManager: Starting task 64.0 in stage 16.0 (TID 398) (10.0.0.133, executor driver, partition 64, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:56 INFO TaskSetManager: Finished task 55.0 in stage 16.0 (TID 389) in 896 ms on 10.0.0.133 (executor driver) (57/200) -26/04/01 08:38:56 INFO Executor: Running task 64.0 in stage 16.0 (TID 398) -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:56 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO Executor: Finished task 57.0 in stage 16.0 (TID 391). 7309 bytes result sent to driver -26/04/01 08:38:57 INFO TaskSetManager: Starting task 65.0 in stage 16.0 (TID 399) (10.0.0.133, executor driver, partition 65, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 57.0 in stage 16.0 (TID 391) in 779 ms on 10.0.0.133 (executor driver) (58/200) -26/04/01 08:38:57 INFO Executor: Running task 65.0 in stage 16.0 (TID 399) -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO Executor: Finished task 58.0 in stage 16.0 (TID 392). 7266 bytes result sent to driver -26/04/01 08:38:57 INFO TaskSetManager: Starting task 66.0 in stage 16.0 (TID 400) (10.0.0.133, executor driver, partition 66, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO Executor: Running task 66.0 in stage 16.0 (TID 400) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 58.0 in stage 16.0 (TID 392) in 832 ms on 10.0.0.133 (executor driver) (59/200) -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO Executor: Finished task 59.0 in stage 16.0 (TID 393). 7266 bytes result sent to driver -26/04/01 08:38:57 INFO TaskSetManager: Starting task 67.0 in stage 16.0 (TID 401) (10.0.0.133, executor driver, partition 67, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 59.0 in stage 16.0 (TID 393) in 838 ms on 10.0.0.133 (executor driver) (60/200) -26/04/01 08:38:57 INFO Executor: Running task 67.0 in stage 16.0 (TID 401) -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO Executor: Finished task 60.0 in stage 16.0 (TID 394). 7266 bytes result sent to driver -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO TaskSetManager: Starting task 68.0 in stage 16.0 (TID 402) (10.0.0.133, executor driver, partition 68, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO Executor: Running task 68.0 in stage 16.0 (TID 402) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 60.0 in stage 16.0 (TID 394) in 836 ms on 10.0.0.133 (executor driver) (61/200) -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO Executor: Finished task 61.0 in stage 16.0 (TID 395). 7266 bytes result sent to driver -26/04/01 08:38:57 INFO TaskSetManager: Starting task 69.0 in stage 16.0 (TID 403) (10.0.0.133, executor driver, partition 69, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 61.0 in stage 16.0 (TID 395) in 838 ms on 10.0.0.133 (executor driver) (62/200) -26/04/01 08:38:57 INFO Executor: Running task 69.0 in stage 16.0 (TID 403) -26/04/01 08:38:57 INFO Executor: Finished task 63.0 in stage 16.0 (TID 397). 7309 bytes result sent to driver -26/04/01 08:38:57 INFO TaskSetManager: Starting task 70.0 in stage 16.0 (TID 404) (10.0.0.133, executor driver, partition 70, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO Executor: Running task 70.0 in stage 16.0 (TID 404) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 63.0 in stage 16.0 (TID 397) in 831 ms on 10.0.0.133 (executor driver) (63/200) -26/04/01 08:38:57 INFO Executor: Finished task 62.0 in stage 16.0 (TID 396). 7266 bytes result sent to driver -26/04/01 08:38:57 INFO TaskSetManager: Starting task 71.0 in stage 16.0 (TID 405) (10.0.0.133, executor driver, partition 71, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 62.0 in stage 16.0 (TID 396) in 837 ms on 10.0.0.133 (executor driver) (64/200) -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO Executor: Running task 71.0 in stage 16.0 (TID 405) -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO Executor: Finished task 64.0 in stage 16.0 (TID 398). 7266 bytes result sent to driver -26/04/01 08:38:57 INFO TaskSetManager: Starting task 72.0 in stage 16.0 (TID 406) (10.0.0.133, executor driver, partition 72, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 64.0 in stage 16.0 (TID 398) in 831 ms on 10.0.0.133 (executor driver) (65/200) -26/04/01 08:38:57 INFO Executor: Running task 72.0 in stage 16.0 (TID 406) -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO Executor: Finished task 65.0 in stage 16.0 (TID 399). 7266 bytes result sent to driver -26/04/01 08:38:57 INFO TaskSetManager: Starting task 73.0 in stage 16.0 (TID 407) (10.0.0.133, executor driver, partition 73, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO Executor: Running task 73.0 in stage 16.0 (TID 407) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 65.0 in stage 16.0 (TID 399) in 698 ms on 10.0.0.133 (executor driver) (66/200) -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO Executor: Finished task 66.0 in stage 16.0 (TID 400). 7266 bytes result sent to driver -26/04/01 08:38:57 INFO TaskSetManager: Starting task 74.0 in stage 16.0 (TID 408) (10.0.0.133, executor driver, partition 74, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO Executor: Running task 74.0 in stage 16.0 (TID 408) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 66.0 in stage 16.0 (TID 400) in 693 ms on 10.0.0.133 (executor driver) (67/200) -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO Executor: Finished task 67.0 in stage 16.0 (TID 401). 7223 bytes result sent to driver -26/04/01 08:38:57 INFO TaskSetManager: Starting task 75.0 in stage 16.0 (TID 409) (10.0.0.133, executor driver, partition 75, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO Executor: Running task 75.0 in stage 16.0 (TID 409) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 67.0 in stage 16.0 (TID 401) in 731 ms on 10.0.0.133 (executor driver) (68/200) -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO Executor: Finished task 68.0 in stage 16.0 (TID 402). 7223 bytes result sent to driver -26/04/01 08:38:57 INFO TaskSetManager: Starting task 76.0 in stage 16.0 (TID 410) (10.0.0.133, executor driver, partition 76, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 68.0 in stage 16.0 (TID 402) in 760 ms on 10.0.0.133 (executor driver) (69/200) -26/04/01 08:38:57 INFO Executor: Running task 76.0 in stage 16.0 (TID 410) -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO Executor: Finished task 69.0 in stage 16.0 (TID 403). 7223 bytes result sent to driver -26/04/01 08:38:57 INFO TaskSetManager: Starting task 77.0 in stage 16.0 (TID 411) (10.0.0.133, executor driver, partition 77, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO Executor: Running task 77.0 in stage 16.0 (TID 411) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 69.0 in stage 16.0 (TID 403) in 773 ms on 10.0.0.133 (executor driver) (70/200) -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO Executor: Finished task 70.0 in stage 16.0 (TID 404). 7266 bytes result sent to driver -26/04/01 08:38:57 INFO TaskSetManager: Starting task 78.0 in stage 16.0 (TID 412) (10.0.0.133, executor driver, partition 78, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO Executor: Running task 78.0 in stage 16.0 (TID 412) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 70.0 in stage 16.0 (TID 404) in 781 ms on 10.0.0.133 (executor driver) (71/200) -26/04/01 08:38:57 INFO Executor: Finished task 72.0 in stage 16.0 (TID 406). 7223 bytes result sent to driver -26/04/01 08:38:57 INFO TaskSetManager: Starting task 79.0 in stage 16.0 (TID 413) (10.0.0.133, executor driver, partition 79, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 72.0 in stage 16.0 (TID 406) in 780 ms on 10.0.0.133 (executor driver) (72/200) -26/04/01 08:38:57 INFO Executor: Running task 79.0 in stage 16.0 (TID 413) -26/04/01 08:38:57 INFO Executor: Finished task 71.0 in stage 16.0 (TID 405). 7223 bytes result sent to driver -26/04/01 08:38:57 INFO TaskSetManager: Starting task 80.0 in stage 16.0 (TID 414) (10.0.0.133, executor driver, partition 80, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:57 INFO Executor: Running task 80.0 in stage 16.0 (TID 414) -26/04/01 08:38:57 INFO TaskSetManager: Finished task 71.0 in stage 16.0 (TID 405) in 780 ms on 10.0.0.133 (executor driver) (73/200) -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:57 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO Executor: Finished task 73.0 in stage 16.0 (TID 407). 7309 bytes result sent to driver -26/04/01 08:38:58 INFO TaskSetManager: Starting task 81.0 in stage 16.0 (TID 415) (10.0.0.133, executor driver, partition 81, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:58 INFO TaskSetManager: Finished task 73.0 in stage 16.0 (TID 407) in 688 ms on 10.0.0.133 (executor driver) (74/200) -26/04/01 08:38:58 INFO Executor: Running task 81.0 in stage 16.0 (TID 415) -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO Executor: Finished task 74.0 in stage 16.0 (TID 408). 7266 bytes result sent to driver -26/04/01 08:38:58 INFO TaskSetManager: Starting task 82.0 in stage 16.0 (TID 416) (10.0.0.133, executor driver, partition 82, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:58 INFO TaskSetManager: Finished task 74.0 in stage 16.0 (TID 408) in 692 ms on 10.0.0.133 (executor driver) (75/200) -26/04/01 08:38:58 INFO Executor: Running task 82.0 in stage 16.0 (TID 416) -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO Executor: Finished task 75.0 in stage 16.0 (TID 409). 7266 bytes result sent to driver -26/04/01 08:38:58 INFO TaskSetManager: Starting task 83.0 in stage 16.0 (TID 417) (10.0.0.133, executor driver, partition 83, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:58 INFO TaskSetManager: Finished task 75.0 in stage 16.0 (TID 409) in 691 ms on 10.0.0.133 (executor driver) (76/200) -26/04/01 08:38:58 INFO Executor: Running task 83.0 in stage 16.0 (TID 417) -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO Executor: Finished task 76.0 in stage 16.0 (TID 410). 7309 bytes result sent to driver -26/04/01 08:38:58 INFO TaskSetManager: Starting task 84.0 in stage 16.0 (TID 418) (10.0.0.133, executor driver, partition 84, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:58 INFO Executor: Running task 84.0 in stage 16.0 (TID 418) -26/04/01 08:38:58 INFO TaskSetManager: Finished task 76.0 in stage 16.0 (TID 410) in 770 ms on 10.0.0.133 (executor driver) (77/200) -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO Executor: Finished task 77.0 in stage 16.0 (TID 411). 7266 bytes result sent to driver -26/04/01 08:38:58 INFO TaskSetManager: Starting task 85.0 in stage 16.0 (TID 419) (10.0.0.133, executor driver, partition 85, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:58 INFO TaskSetManager: Finished task 77.0 in stage 16.0 (TID 411) in 800 ms on 10.0.0.133 (executor driver) (78/200) -26/04/01 08:38:58 INFO Executor: Running task 85.0 in stage 16.0 (TID 419) -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO Executor: Finished task 78.0 in stage 16.0 (TID 412). 7266 bytes result sent to driver -26/04/01 08:38:58 INFO TaskSetManager: Starting task 86.0 in stage 16.0 (TID 420) (10.0.0.133, executor driver, partition 86, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:58 INFO TaskSetManager: Finished task 78.0 in stage 16.0 (TID 412) in 808 ms on 10.0.0.133 (executor driver) (79/200) -26/04/01 08:38:58 INFO Executor: Running task 86.0 in stage 16.0 (TID 420) -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO Executor: Finished task 79.0 in stage 16.0 (TID 413). 7266 bytes result sent to driver -26/04/01 08:38:58 INFO TaskSetManager: Starting task 87.0 in stage 16.0 (TID 421) (10.0.0.133, executor driver, partition 87, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:58 INFO TaskSetManager: Finished task 79.0 in stage 16.0 (TID 413) in 810 ms on 10.0.0.133 (executor driver) (80/200) -26/04/01 08:38:58 INFO Executor: Running task 87.0 in stage 16.0 (TID 421) -26/04/01 08:38:58 INFO Executor: Finished task 80.0 in stage 16.0 (TID 414). 7266 bytes result sent to driver -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO TaskSetManager: Starting task 88.0 in stage 16.0 (TID 422) (10.0.0.133, executor driver, partition 88, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:58 INFO Executor: Running task 88.0 in stage 16.0 (TID 422) -26/04/01 08:38:58 INFO TaskSetManager: Finished task 80.0 in stage 16.0 (TID 414) in 812 ms on 10.0.0.133 (executor driver) (81/200) -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:58 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO Executor: Finished task 81.0 in stage 16.0 (TID 415). 7223 bytes result sent to driver -26/04/01 08:38:59 INFO TaskSetManager: Starting task 89.0 in stage 16.0 (TID 423) (10.0.0.133, executor driver, partition 89, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:59 INFO Executor: Running task 89.0 in stage 16.0 (TID 423) -26/04/01 08:38:59 INFO TaskSetManager: Finished task 81.0 in stage 16.0 (TID 415) in 695 ms on 10.0.0.133 (executor driver) (82/200) -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO Executor: Finished task 82.0 in stage 16.0 (TID 416). 7223 bytes result sent to driver -26/04/01 08:38:59 INFO TaskSetManager: Starting task 90.0 in stage 16.0 (TID 424) (10.0.0.133, executor driver, partition 90, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:59 INFO Executor: Running task 90.0 in stage 16.0 (TID 424) -26/04/01 08:38:59 INFO TaskSetManager: Finished task 82.0 in stage 16.0 (TID 416) in 688 ms on 10.0.0.133 (executor driver) (83/200) -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO Executor: Finished task 83.0 in stage 16.0 (TID 417). 7223 bytes result sent to driver -26/04/01 08:38:59 INFO TaskSetManager: Starting task 91.0 in stage 16.0 (TID 425) (10.0.0.133, executor driver, partition 91, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:59 INFO TaskSetManager: Finished task 83.0 in stage 16.0 (TID 417) in 704 ms on 10.0.0.133 (executor driver) (84/200) -26/04/01 08:38:59 INFO Executor: Running task 91.0 in stage 16.0 (TID 425) -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO Executor: Finished task 84.0 in stage 16.0 (TID 418). 7266 bytes result sent to driver -26/04/01 08:38:59 INFO TaskSetManager: Starting task 92.0 in stage 16.0 (TID 426) (10.0.0.133, executor driver, partition 92, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:59 INFO TaskSetManager: Finished task 84.0 in stage 16.0 (TID 418) in 666 ms on 10.0.0.133 (executor driver) (85/200) -26/04/01 08:38:59 INFO Executor: Running task 92.0 in stage 16.0 (TID 426) -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO Executor: Finished task 85.0 in stage 16.0 (TID 419). 7223 bytes result sent to driver -26/04/01 08:38:59 INFO TaskSetManager: Starting task 93.0 in stage 16.0 (TID 427) (10.0.0.133, executor driver, partition 93, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:59 INFO TaskSetManager: Finished task 85.0 in stage 16.0 (TID 419) in 686 ms on 10.0.0.133 (executor driver) (86/200) -26/04/01 08:38:59 INFO Executor: Running task 93.0 in stage 16.0 (TID 427) -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO Executor: Finished task 86.0 in stage 16.0 (TID 420). 7223 bytes result sent to driver -26/04/01 08:38:59 INFO TaskSetManager: Starting task 94.0 in stage 16.0 (TID 428) (10.0.0.133, executor driver, partition 94, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:59 INFO TaskSetManager: Finished task 86.0 in stage 16.0 (TID 420) in 684 ms on 10.0.0.133 (executor driver) (87/200) -26/04/01 08:38:59 INFO Executor: Running task 94.0 in stage 16.0 (TID 428) -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO Executor: Finished task 87.0 in stage 16.0 (TID 421). 7223 bytes result sent to driver -26/04/01 08:38:59 INFO TaskSetManager: Starting task 95.0 in stage 16.0 (TID 429) (10.0.0.133, executor driver, partition 95, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:59 INFO Executor: Running task 95.0 in stage 16.0 (TID 429) -26/04/01 08:38:59 INFO TaskSetManager: Finished task 87.0 in stage 16.0 (TID 421) in 698 ms on 10.0.0.133 (executor driver) (88/200) -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO Executor: Finished task 88.0 in stage 16.0 (TID 422). 7223 bytes result sent to driver -26/04/01 08:38:59 INFO TaskSetManager: Starting task 96.0 in stage 16.0 (TID 430) (10.0.0.133, executor driver, partition 96, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:59 INFO Executor: Running task 96.0 in stage 16.0 (TID 430) -26/04/01 08:38:59 INFO TaskSetManager: Finished task 88.0 in stage 16.0 (TID 422) in 698 ms on 10.0.0.133 (executor driver) (89/200) -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO Executor: Finished task 89.0 in stage 16.0 (TID 423). 7266 bytes result sent to driver -26/04/01 08:38:59 INFO TaskSetManager: Starting task 97.0 in stage 16.0 (TID 431) (10.0.0.133, executor driver, partition 97, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:59 INFO Executor: Running task 97.0 in stage 16.0 (TID 431) -26/04/01 08:38:59 INFO TaskSetManager: Finished task 89.0 in stage 16.0 (TID 423) in 654 ms on 10.0.0.133 (executor driver) (90/200) -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (28.1 KiB) non-empty blocks including 8 (28.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO Executor: Finished task 90.0 in stage 16.0 (TID 424). 7266 bytes result sent to driver -26/04/01 08:38:59 INFO TaskSetManager: Starting task 98.0 in stage 16.0 (TID 432) (10.0.0.133, executor driver, partition 98, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:59 INFO Executor: Running task 98.0 in stage 16.0 (TID 432) -26/04/01 08:38:59 INFO TaskSetManager: Finished task 90.0 in stage 16.0 (TID 424) in 669 ms on 10.0.0.133 (executor driver) (91/200) -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (29.5 KiB) non-empty blocks including 8 (29.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO Executor: Finished task 91.0 in stage 16.0 (TID 425). 7266 bytes result sent to driver -26/04/01 08:38:59 INFO TaskSetManager: Starting task 99.0 in stage 16.0 (TID 433) (10.0.0.133, executor driver, partition 99, NODE_LOCAL, 9270 bytes) -26/04/01 08:38:59 INFO Executor: Running task 99.0 in stage 16.0 (TID 433) -26/04/01 08:38:59 INFO TaskSetManager: Finished task 91.0 in stage 16.0 (TID 425) in 702 ms on 10.0.0.133 (executor driver) (92/200) -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:38:59 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:38:59 INFO Executor: Finished task 92.0 in stage 16.0 (TID 426). 7266 bytes result sent to driver -26/04/01 08:39:00 INFO TaskSetManager: Starting task 100.0 in stage 16.0 (TID 434) (10.0.0.133, executor driver, partition 100, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:00 INFO Executor: Running task 100.0 in stage 16.0 (TID 434) -26/04/01 08:39:00 INFO TaskSetManager: Finished task 92.0 in stage 16.0 (TID 426) in 697 ms on 10.0.0.133 (executor driver) (93/200) -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO Executor: Finished task 93.0 in stage 16.0 (TID 427). 7266 bytes result sent to driver -26/04/01 08:39:00 INFO TaskSetManager: Starting task 101.0 in stage 16.0 (TID 435) (10.0.0.133, executor driver, partition 101, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:00 INFO TaskSetManager: Finished task 93.0 in stage 16.0 (TID 427) in 833 ms on 10.0.0.133 (executor driver) (94/200) -26/04/01 08:39:00 INFO Executor: Running task 101.0 in stage 16.0 (TID 435) -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO Executor: Finished task 94.0 in stage 16.0 (TID 428). 7266 bytes result sent to driver -26/04/01 08:39:00 INFO TaskSetManager: Starting task 102.0 in stage 16.0 (TID 436) (10.0.0.133, executor driver, partition 102, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:00 INFO Executor: Running task 102.0 in stage 16.0 (TID 436) -26/04/01 08:39:00 INFO TaskSetManager: Finished task 94.0 in stage 16.0 (TID 428) in 828 ms on 10.0.0.133 (executor driver) (95/200) -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO Executor: Finished task 96.0 in stage 16.0 (TID 430). 7266 bytes result sent to driver -26/04/01 08:39:00 INFO TaskSetManager: Starting task 103.0 in stage 16.0 (TID 437) (10.0.0.133, executor driver, partition 103, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:00 INFO TaskSetManager: Finished task 96.0 in stage 16.0 (TID 430) in 814 ms on 10.0.0.133 (executor driver) (96/200) -26/04/01 08:39:00 INFO Executor: Running task 103.0 in stage 16.0 (TID 437) -26/04/01 08:39:00 INFO Executor: Finished task 95.0 in stage 16.0 (TID 429). 7266 bytes result sent to driver -26/04/01 08:39:00 INFO TaskSetManager: Starting task 104.0 in stage 16.0 (TID 438) (10.0.0.133, executor driver, partition 104, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:00 INFO TaskSetManager: Finished task 95.0 in stage 16.0 (TID 429) in 816 ms on 10.0.0.133 (executor driver) (97/200) -26/04/01 08:39:00 INFO Executor: Running task 104.0 in stage 16.0 (TID 438) -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO Executor: Finished task 97.0 in stage 16.0 (TID 431). 7223 bytes result sent to driver -26/04/01 08:39:00 INFO TaskSetManager: Starting task 105.0 in stage 16.0 (TID 439) (10.0.0.133, executor driver, partition 105, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:00 INFO Executor: Running task 105.0 in stage 16.0 (TID 439) -26/04/01 08:39:00 INFO TaskSetManager: Finished task 97.0 in stage 16.0 (TID 431) in 727 ms on 10.0.0.133 (executor driver) (98/200) -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO Executor: Finished task 98.0 in stage 16.0 (TID 432). 7223 bytes result sent to driver -26/04/01 08:39:00 INFO TaskSetManager: Starting task 106.0 in stage 16.0 (TID 440) (10.0.0.133, executor driver, partition 106, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:00 INFO Executor: Running task 106.0 in stage 16.0 (TID 440) -26/04/01 08:39:00 INFO TaskSetManager: Finished task 98.0 in stage 16.0 (TID 432) in 675 ms on 10.0.0.133 (executor driver) (99/200) -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO Executor: Finished task 99.0 in stage 16.0 (TID 433). 7223 bytes result sent to driver -26/04/01 08:39:00 INFO TaskSetManager: Starting task 107.0 in stage 16.0 (TID 441) (10.0.0.133, executor driver, partition 107, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:00 INFO Executor: Running task 107.0 in stage 16.0 (TID 441) -26/04/01 08:39:00 INFO TaskSetManager: Finished task 99.0 in stage 16.0 (TID 433) in 657 ms on 10.0.0.133 (executor driver) (100/200) -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO Executor: Finished task 100.0 in stage 16.0 (TID 434). 7266 bytes result sent to driver -26/04/01 08:39:00 INFO TaskSetManager: Starting task 108.0 in stage 16.0 (TID 442) (10.0.0.133, executor driver, partition 108, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:00 INFO TaskSetManager: Finished task 100.0 in stage 16.0 (TID 434) in 666 ms on 10.0.0.133 (executor driver) (101/200) -26/04/01 08:39:00 INFO Executor: Running task 108.0 in stage 16.0 (TID 442) -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO Executor: Finished task 101.0 in stage 16.0 (TID 435). 7266 bytes result sent to driver -26/04/01 08:39:00 INFO TaskSetManager: Starting task 109.0 in stage 16.0 (TID 443) (10.0.0.133, executor driver, partition 109, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:00 INFO TaskSetManager: Finished task 101.0 in stage 16.0 (TID 435) in 705 ms on 10.0.0.133 (executor driver) (102/200) -26/04/01 08:39:00 INFO Executor: Running task 109.0 in stage 16.0 (TID 443) -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO Executor: Finished task 102.0 in stage 16.0 (TID 436). 7223 bytes result sent to driver -26/04/01 08:39:00 INFO TaskSetManager: Starting task 110.0 in stage 16.0 (TID 444) (10.0.0.133, executor driver, partition 110, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:00 INFO Executor: Running task 110.0 in stage 16.0 (TID 444) -26/04/01 08:39:00 INFO TaskSetManager: Finished task 102.0 in stage 16.0 (TID 436) in 707 ms on 10.0.0.133 (executor driver) (103/200) -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO Executor: Finished task 103.0 in stage 16.0 (TID 437). 7223 bytes result sent to driver -26/04/01 08:39:00 INFO TaskSetManager: Starting task 111.0 in stage 16.0 (TID 445) (10.0.0.133, executor driver, partition 111, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:00 INFO Executor: Running task 111.0 in stage 16.0 (TID 445) -26/04/01 08:39:00 INFO TaskSetManager: Finished task 103.0 in stage 16.0 (TID 437) in 708 ms on 10.0.0.133 (executor driver) (104/200) -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO Executor: Finished task 104.0 in stage 16.0 (TID 438). 7223 bytes result sent to driver -26/04/01 08:39:00 INFO TaskSetManager: Starting task 112.0 in stage 16.0 (TID 446) (10.0.0.133, executor driver, partition 112, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:00 INFO Executor: Running task 112.0 in stage 16.0 (TID 446) -26/04/01 08:39:00 INFO TaskSetManager: Finished task 104.0 in stage 16.0 (TID 438) in 710 ms on 10.0.0.133 (executor driver) (105/200) -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:00 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO Executor: Finished task 105.0 in stage 16.0 (TID 439). 7266 bytes result sent to driver -26/04/01 08:39:01 INFO TaskSetManager: Starting task 113.0 in stage 16.0 (TID 447) (10.0.0.133, executor driver, partition 113, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:01 INFO Executor: Running task 113.0 in stage 16.0 (TID 447) -26/04/01 08:39:01 INFO TaskSetManager: Finished task 105.0 in stage 16.0 (TID 439) in 661 ms on 10.0.0.133 (executor driver) (106/200) -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO Executor: Finished task 106.0 in stage 16.0 (TID 440). 7223 bytes result sent to driver -26/04/01 08:39:01 INFO TaskSetManager: Starting task 114.0 in stage 16.0 (TID 448) (10.0.0.133, executor driver, partition 114, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:01 INFO Executor: Running task 114.0 in stage 16.0 (TID 448) -26/04/01 08:39:01 INFO TaskSetManager: Finished task 106.0 in stage 16.0 (TID 440) in 655 ms on 10.0.0.133 (executor driver) (107/200) -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO Executor: Finished task 107.0 in stage 16.0 (TID 441). 7223 bytes result sent to driver -26/04/01 08:39:01 INFO TaskSetManager: Starting task 115.0 in stage 16.0 (TID 449) (10.0.0.133, executor driver, partition 115, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:01 INFO TaskSetManager: Finished task 107.0 in stage 16.0 (TID 441) in 661 ms on 10.0.0.133 (executor driver) (108/200) -26/04/01 08:39:01 INFO Executor: Running task 115.0 in stage 16.0 (TID 449) -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO Executor: Finished task 108.0 in stage 16.0 (TID 442). 7266 bytes result sent to driver -26/04/01 08:39:01 INFO TaskSetManager: Starting task 116.0 in stage 16.0 (TID 450) (10.0.0.133, executor driver, partition 116, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:01 INFO TaskSetManager: Finished task 108.0 in stage 16.0 (TID 442) in 677 ms on 10.0.0.133 (executor driver) (109/200) -26/04/01 08:39:01 INFO Executor: Running task 116.0 in stage 16.0 (TID 450) -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO Executor: Finished task 109.0 in stage 16.0 (TID 443). 7266 bytes result sent to driver -26/04/01 08:39:01 INFO TaskSetManager: Starting task 117.0 in stage 16.0 (TID 451) (10.0.0.133, executor driver, partition 117, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:01 INFO Executor: Running task 117.0 in stage 16.0 (TID 451) -26/04/01 08:39:01 INFO TaskSetManager: Finished task 109.0 in stage 16.0 (TID 443) in 723 ms on 10.0.0.133 (executor driver) (110/200) -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO Executor: Finished task 110.0 in stage 16.0 (TID 444). 7266 bytes result sent to driver -26/04/01 08:39:01 INFO TaskSetManager: Starting task 118.0 in stage 16.0 (TID 452) (10.0.0.133, executor driver, partition 118, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:01 INFO Executor: Running task 118.0 in stage 16.0 (TID 452) -26/04/01 08:39:01 INFO TaskSetManager: Finished task 110.0 in stage 16.0 (TID 444) in 719 ms on 10.0.0.133 (executor driver) (111/200) -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO Executor: Finished task 111.0 in stage 16.0 (TID 445). 7266 bytes result sent to driver -26/04/01 08:39:01 INFO TaskSetManager: Starting task 119.0 in stage 16.0 (TID 453) (10.0.0.133, executor driver, partition 119, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:01 INFO TaskSetManager: Finished task 111.0 in stage 16.0 (TID 445) in 717 ms on 10.0.0.133 (executor driver) (112/200) -26/04/01 08:39:01 INFO Executor: Running task 119.0 in stage 16.0 (TID 453) -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO Executor: Finished task 112.0 in stage 16.0 (TID 446). 7266 bytes result sent to driver -26/04/01 08:39:01 INFO TaskSetManager: Starting task 120.0 in stage 16.0 (TID 454) (10.0.0.133, executor driver, partition 120, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:01 INFO TaskSetManager: Finished task 112.0 in stage 16.0 (TID 446) in 715 ms on 10.0.0.133 (executor driver) (113/200) -26/04/01 08:39:01 INFO Executor: Running task 120.0 in stage 16.0 (TID 454) -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO Executor: Finished task 113.0 in stage 16.0 (TID 447). 7266 bytes result sent to driver -26/04/01 08:39:01 INFO TaskSetManager: Starting task 121.0 in stage 16.0 (TID 455) (10.0.0.133, executor driver, partition 121, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:01 INFO Executor: Running task 121.0 in stage 16.0 (TID 455) -26/04/01 08:39:01 INFO TaskSetManager: Finished task 113.0 in stage 16.0 (TID 447) in 701 ms on 10.0.0.133 (executor driver) (114/200) -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO Executor: Finished task 114.0 in stage 16.0 (TID 448). 7266 bytes result sent to driver -26/04/01 08:39:01 INFO TaskSetManager: Starting task 122.0 in stage 16.0 (TID 456) (10.0.0.133, executor driver, partition 122, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO Executor: Running task 122.0 in stage 16.0 (TID 456) -26/04/01 08:39:01 INFO TaskSetManager: Finished task 114.0 in stage 16.0 (TID 448) in 699 ms on 10.0.0.133 (executor driver) (115/200) -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO Executor: Finished task 115.0 in stage 16.0 (TID 449). 7266 bytes result sent to driver -26/04/01 08:39:01 INFO TaskSetManager: Starting task 123.0 in stage 16.0 (TID 457) (10.0.0.133, executor driver, partition 123, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:01 INFO Executor: Running task 123.0 in stage 16.0 (TID 457) -26/04/01 08:39:01 INFO TaskSetManager: Finished task 115.0 in stage 16.0 (TID 449) in 686 ms on 10.0.0.133 (executor driver) (116/200) -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:01 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO Executor: Finished task 116.0 in stage 16.0 (TID 450). 7223 bytes result sent to driver -26/04/01 08:39:02 INFO TaskSetManager: Starting task 124.0 in stage 16.0 (TID 458) (10.0.0.133, executor driver, partition 124, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:02 INFO Executor: Running task 124.0 in stage 16.0 (TID 458) -26/04/01 08:39:02 INFO TaskSetManager: Finished task 116.0 in stage 16.0 (TID 450) in 672 ms on 10.0.0.133 (executor driver) (117/200) -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO Executor: Finished task 117.0 in stage 16.0 (TID 451). 7223 bytes result sent to driver -26/04/01 08:39:02 INFO TaskSetManager: Starting task 125.0 in stage 16.0 (TID 459) (10.0.0.133, executor driver, partition 125, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:02 INFO TaskSetManager: Finished task 117.0 in stage 16.0 (TID 451) in 713 ms on 10.0.0.133 (executor driver) (118/200) -26/04/01 08:39:02 INFO Executor: Running task 125.0 in stage 16.0 (TID 459) -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO Executor: Finished task 118.0 in stage 16.0 (TID 452). 7223 bytes result sent to driver -26/04/01 08:39:02 INFO TaskSetManager: Starting task 126.0 in stage 16.0 (TID 460) (10.0.0.133, executor driver, partition 126, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:02 INFO TaskSetManager: Finished task 118.0 in stage 16.0 (TID 452) in 710 ms on 10.0.0.133 (executor driver) (119/200) -26/04/01 08:39:02 INFO Executor: Running task 126.0 in stage 16.0 (TID 460) -26/04/01 08:39:02 INFO Executor: Finished task 120.0 in stage 16.0 (TID 454). 7223 bytes result sent to driver -26/04/01 08:39:02 INFO TaskSetManager: Starting task 127.0 in stage 16.0 (TID 461) (10.0.0.133, executor driver, partition 127, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:02 INFO TaskSetManager: Finished task 120.0 in stage 16.0 (TID 454) in 708 ms on 10.0.0.133 (executor driver) (120/200) -26/04/01 08:39:02 INFO Executor: Running task 127.0 in stage 16.0 (TID 461) -26/04/01 08:39:02 INFO Executor: Finished task 119.0 in stage 16.0 (TID 453). 7223 bytes result sent to driver -26/04/01 08:39:02 INFO TaskSetManager: Starting task 128.0 in stage 16.0 (TID 462) (10.0.0.133, executor driver, partition 128, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO TaskSetManager: Finished task 119.0 in stage 16.0 (TID 453) in 709 ms on 10.0.0.133 (executor driver) (121/200) -26/04/01 08:39:02 INFO Executor: Running task 128.0 in stage 16.0 (TID 462) -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO Executor: Finished task 121.0 in stage 16.0 (TID 455). 7223 bytes result sent to driver -26/04/01 08:39:02 INFO TaskSetManager: Starting task 129.0 in stage 16.0 (TID 463) (10.0.0.133, executor driver, partition 129, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:02 INFO Executor: Running task 129.0 in stage 16.0 (TID 463) -26/04/01 08:39:02 INFO TaskSetManager: Finished task 121.0 in stage 16.0 (TID 455) in 652 ms on 10.0.0.133 (executor driver) (122/200) -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO Executor: Finished task 122.0 in stage 16.0 (TID 456). 7223 bytes result sent to driver -26/04/01 08:39:02 INFO TaskSetManager: Starting task 130.0 in stage 16.0 (TID 464) (10.0.0.133, executor driver, partition 130, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:02 INFO TaskSetManager: Finished task 122.0 in stage 16.0 (TID 456) in 657 ms on 10.0.0.133 (executor driver) (123/200) -26/04/01 08:39:02 INFO Executor: Running task 130.0 in stage 16.0 (TID 464) -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO Executor: Finished task 123.0 in stage 16.0 (TID 457). 7266 bytes result sent to driver -26/04/01 08:39:02 INFO TaskSetManager: Starting task 131.0 in stage 16.0 (TID 465) (10.0.0.133, executor driver, partition 131, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:02 INFO Executor: Running task 131.0 in stage 16.0 (TID 465) -26/04/01 08:39:02 INFO TaskSetManager: Finished task 123.0 in stage 16.0 (TID 457) in 655 ms on 10.0.0.133 (executor driver) (124/200) -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO Executor: Finished task 124.0 in stage 16.0 (TID 458). 7223 bytes result sent to driver -26/04/01 08:39:02 INFO TaskSetManager: Starting task 132.0 in stage 16.0 (TID 466) (10.0.0.133, executor driver, partition 132, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:02 INFO TaskSetManager: Finished task 124.0 in stage 16.0 (TID 458) in 666 ms on 10.0.0.133 (executor driver) (125/200) -26/04/01 08:39:02 INFO Executor: Running task 132.0 in stage 16.0 (TID 466) -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:02 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO Executor: Finished task 126.0 in stage 16.0 (TID 460). 7266 bytes result sent to driver -26/04/01 08:39:03 INFO TaskSetManager: Starting task 133.0 in stage 16.0 (TID 467) (10.0.0.133, executor driver, partition 133, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO Executor: Finished task 127.0 in stage 16.0 (TID 461). 7309 bytes result sent to driver -26/04/01 08:39:03 INFO Executor: Running task 133.0 in stage 16.0 (TID 467) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 126.0 in stage 16.0 (TID 460) in 697 ms on 10.0.0.133 (executor driver) (126/200) -26/04/01 08:39:03 INFO TaskSetManager: Starting task 134.0 in stage 16.0 (TID 468) (10.0.0.133, executor driver, partition 134, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO Executor: Running task 134.0 in stage 16.0 (TID 468) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 127.0 in stage 16.0 (TID 461) in 697 ms on 10.0.0.133 (executor driver) (127/200) -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO Executor: Finished task 125.0 in stage 16.0 (TID 459). 7309 bytes result sent to driver -26/04/01 08:39:03 INFO TaskSetManager: Starting task 135.0 in stage 16.0 (TID 469) (10.0.0.133, executor driver, partition 135, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO Executor: Running task 135.0 in stage 16.0 (TID 469) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 125.0 in stage 16.0 (TID 459) in 749 ms on 10.0.0.133 (executor driver) (128/200) -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO Executor: Finished task 128.0 in stage 16.0 (TID 462). 7266 bytes result sent to driver -26/04/01 08:39:03 INFO TaskSetManager: Starting task 136.0 in stage 16.0 (TID 470) (10.0.0.133, executor driver, partition 136, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO Executor: Running task 136.0 in stage 16.0 (TID 470) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 128.0 in stage 16.0 (TID 462) in 759 ms on 10.0.0.133 (executor driver) (129/200) -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO Executor: Finished task 129.0 in stage 16.0 (TID 463). 7266 bytes result sent to driver -26/04/01 08:39:03 INFO TaskSetManager: Starting task 137.0 in stage 16.0 (TID 471) (10.0.0.133, executor driver, partition 137, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 129.0 in stage 16.0 (TID 463) in 672 ms on 10.0.0.133 (executor driver) (130/200) -26/04/01 08:39:03 INFO Executor: Running task 137.0 in stage 16.0 (TID 471) -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO Executor: Finished task 130.0 in stage 16.0 (TID 464). 7266 bytes result sent to driver -26/04/01 08:39:03 INFO TaskSetManager: Starting task 138.0 in stage 16.0 (TID 472) (10.0.0.133, executor driver, partition 138, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 130.0 in stage 16.0 (TID 464) in 676 ms on 10.0.0.133 (executor driver) (131/200) -26/04/01 08:39:03 INFO Executor: Running task 138.0 in stage 16.0 (TID 472) -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO Executor: Finished task 131.0 in stage 16.0 (TID 465). 7266 bytes result sent to driver -26/04/01 08:39:03 INFO TaskSetManager: Starting task 139.0 in stage 16.0 (TID 473) (10.0.0.133, executor driver, partition 139, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 131.0 in stage 16.0 (TID 465) in 696 ms on 10.0.0.133 (executor driver) (132/200) -26/04/01 08:39:03 INFO Executor: Running task 139.0 in stage 16.0 (TID 473) -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO Executor: Finished task 132.0 in stage 16.0 (TID 466). 7266 bytes result sent to driver -26/04/01 08:39:03 INFO TaskSetManager: Starting task 140.0 in stage 16.0 (TID 474) (10.0.0.133, executor driver, partition 140, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 132.0 in stage 16.0 (TID 466) in 645 ms on 10.0.0.133 (executor driver) (133/200) -26/04/01 08:39:03 INFO Executor: Running task 140.0 in stage 16.0 (TID 474) -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO Executor: Finished task 133.0 in stage 16.0 (TID 467). 7223 bytes result sent to driver -26/04/01 08:39:03 INFO TaskSetManager: Starting task 141.0 in stage 16.0 (TID 475) (10.0.0.133, executor driver, partition 141, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO Executor: Running task 141.0 in stage 16.0 (TID 475) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 133.0 in stage 16.0 (TID 467) in 695 ms on 10.0.0.133 (executor driver) (134/200) -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO Executor: Finished task 134.0 in stage 16.0 (TID 468). 7223 bytes result sent to driver -26/04/01 08:39:03 INFO TaskSetManager: Starting task 142.0 in stage 16.0 (TID 476) (10.0.0.133, executor driver, partition 142, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO Executor: Running task 142.0 in stage 16.0 (TID 476) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 134.0 in stage 16.0 (TID 468) in 699 ms on 10.0.0.133 (executor driver) (135/200) -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO Executor: Finished task 135.0 in stage 16.0 (TID 469). 7223 bytes result sent to driver -26/04/01 08:39:03 INFO TaskSetManager: Starting task 143.0 in stage 16.0 (TID 477) (10.0.0.133, executor driver, partition 143, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO Executor: Running task 143.0 in stage 16.0 (TID 477) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 135.0 in stage 16.0 (TID 469) in 677 ms on 10.0.0.133 (executor driver) (136/200) -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO Executor: Finished task 136.0 in stage 16.0 (TID 470). 7223 bytes result sent to driver -26/04/01 08:39:03 INFO TaskSetManager: Starting task 144.0 in stage 16.0 (TID 478) (10.0.0.133, executor driver, partition 144, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO Executor: Running task 144.0 in stage 16.0 (TID 478) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 136.0 in stage 16.0 (TID 470) in 676 ms on 10.0.0.133 (executor driver) (137/200) -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO Executor: Finished task 137.0 in stage 16.0 (TID 471). 7223 bytes result sent to driver -26/04/01 08:39:03 INFO TaskSetManager: Starting task 145.0 in stage 16.0 (TID 479) (10.0.0.133, executor driver, partition 145, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 137.0 in stage 16.0 (TID 471) in 677 ms on 10.0.0.133 (executor driver) (138/200) -26/04/01 08:39:03 INFO Executor: Running task 145.0 in stage 16.0 (TID 479) -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO Executor: Finished task 138.0 in stage 16.0 (TID 472). 7223 bytes result sent to driver -26/04/01 08:39:03 INFO TaskSetManager: Starting task 146.0 in stage 16.0 (TID 480) (10.0.0.133, executor driver, partition 146, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 138.0 in stage 16.0 (TID 472) in 676 ms on 10.0.0.133 (executor driver) (139/200) -26/04/01 08:39:03 INFO Executor: Running task 146.0 in stage 16.0 (TID 480) -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO Executor: Finished task 139.0 in stage 16.0 (TID 473). 7223 bytes result sent to driver -26/04/01 08:39:03 INFO TaskSetManager: Starting task 147.0 in stage 16.0 (TID 481) (10.0.0.133, executor driver, partition 147, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO Executor: Running task 147.0 in stage 16.0 (TID 481) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 139.0 in stage 16.0 (TID 473) in 673 ms on 10.0.0.133 (executor driver) (140/200) -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO Executor: Finished task 140.0 in stage 16.0 (TID 474). 7223 bytes result sent to driver -26/04/01 08:39:03 INFO TaskSetManager: Starting task 148.0 in stage 16.0 (TID 482) (10.0.0.133, executor driver, partition 148, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:03 INFO TaskSetManager: Finished task 140.0 in stage 16.0 (TID 474) in 656 ms on 10.0.0.133 (executor driver) (141/200) -26/04/01 08:39:03 INFO Executor: Running task 148.0 in stage 16.0 (TID 482) -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:03 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO Executor: Finished task 141.0 in stage 16.0 (TID 475). 7266 bytes result sent to driver -26/04/01 08:39:04 INFO TaskSetManager: Starting task 149.0 in stage 16.0 (TID 483) (10.0.0.133, executor driver, partition 149, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:04 INFO TaskSetManager: Finished task 141.0 in stage 16.0 (TID 475) in 676 ms on 10.0.0.133 (executor driver) (142/200) -26/04/01 08:39:04 INFO Executor: Running task 149.0 in stage 16.0 (TID 483) -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO Executor: Finished task 142.0 in stage 16.0 (TID 476). 7266 bytes result sent to driver -26/04/01 08:39:04 INFO TaskSetManager: Starting task 150.0 in stage 16.0 (TID 484) (10.0.0.133, executor driver, partition 150, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:04 INFO Executor: Running task 150.0 in stage 16.0 (TID 484) -26/04/01 08:39:04 INFO TaskSetManager: Finished task 142.0 in stage 16.0 (TID 476) in 681 ms on 10.0.0.133 (executor driver) (143/200) -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO Executor: Finished task 143.0 in stage 16.0 (TID 477). 7266 bytes result sent to driver -26/04/01 08:39:04 INFO TaskSetManager: Starting task 151.0 in stage 16.0 (TID 485) (10.0.0.133, executor driver, partition 151, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:04 INFO TaskSetManager: Finished task 143.0 in stage 16.0 (TID 477) in 677 ms on 10.0.0.133 (executor driver) (144/200) -26/04/01 08:39:04 INFO Executor: Running task 151.0 in stage 16.0 (TID 485) -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO Executor: Finished task 144.0 in stage 16.0 (TID 478). 7266 bytes result sent to driver -26/04/01 08:39:04 INFO TaskSetManager: Starting task 152.0 in stage 16.0 (TID 486) (10.0.0.133, executor driver, partition 152, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:04 INFO TaskSetManager: Finished task 144.0 in stage 16.0 (TID 478) in 688 ms on 10.0.0.133 (executor driver) (145/200) -26/04/01 08:39:04 INFO Executor: Running task 152.0 in stage 16.0 (TID 486) -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO Executor: Finished task 145.0 in stage 16.0 (TID 479). 7266 bytes result sent to driver -26/04/01 08:39:04 INFO TaskSetManager: Starting task 153.0 in stage 16.0 (TID 487) (10.0.0.133, executor driver, partition 153, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:04 INFO TaskSetManager: Finished task 145.0 in stage 16.0 (TID 479) in 677 ms on 10.0.0.133 (executor driver) (146/200) -26/04/01 08:39:04 INFO Executor: Running task 153.0 in stage 16.0 (TID 487) -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO Executor: Finished task 146.0 in stage 16.0 (TID 480). 7266 bytes result sent to driver -26/04/01 08:39:04 INFO TaskSetManager: Starting task 154.0 in stage 16.0 (TID 488) (10.0.0.133, executor driver, partition 154, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:04 INFO TaskSetManager: Finished task 146.0 in stage 16.0 (TID 480) in 682 ms on 10.0.0.133 (executor driver) (147/200) -26/04/01 08:39:04 INFO Executor: Running task 154.0 in stage 16.0 (TID 488) -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO Executor: Finished task 147.0 in stage 16.0 (TID 481). 7266 bytes result sent to driver -26/04/01 08:39:04 INFO TaskSetManager: Starting task 155.0 in stage 16.0 (TID 489) (10.0.0.133, executor driver, partition 155, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:04 INFO TaskSetManager: Finished task 147.0 in stage 16.0 (TID 481) in 667 ms on 10.0.0.133 (executor driver) (148/200) -26/04/01 08:39:04 INFO Executor: Running task 155.0 in stage 16.0 (TID 489) -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO Executor: Finished task 148.0 in stage 16.0 (TID 482). 7266 bytes result sent to driver -26/04/01 08:39:04 INFO TaskSetManager: Starting task 156.0 in stage 16.0 (TID 490) (10.0.0.133, executor driver, partition 156, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:04 INFO Executor: Running task 156.0 in stage 16.0 (TID 490) -26/04/01 08:39:04 INFO TaskSetManager: Finished task 148.0 in stage 16.0 (TID 482) in 671 ms on 10.0.0.133 (executor driver) (149/200) -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:04 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 149.0 in stage 16.0 (TID 483). 7223 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 157.0 in stage 16.0 (TID 491) (10.0.0.133, executor driver, partition 157, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO Executor: Running task 157.0 in stage 16.0 (TID 491) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 149.0 in stage 16.0 (TID 483) in 730 ms on 10.0.0.133 (executor driver) (150/200) -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 150.0 in stage 16.0 (TID 484). 7223 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 158.0 in stage 16.0 (TID 492) (10.0.0.133, executor driver, partition 158, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO Executor: Running task 158.0 in stage 16.0 (TID 492) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 150.0 in stage 16.0 (TID 484) in 745 ms on 10.0.0.133 (executor driver) (151/200) -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 151.0 in stage 16.0 (TID 485). 7223 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 159.0 in stage 16.0 (TID 493) (10.0.0.133, executor driver, partition 159, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO Executor: Running task 159.0 in stage 16.0 (TID 493) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 151.0 in stage 16.0 (TID 485) in 729 ms on 10.0.0.133 (executor driver) (152/200) -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 152.0 in stage 16.0 (TID 486). 7223 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 160.0 in stage 16.0 (TID 494) (10.0.0.133, executor driver, partition 160, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 152.0 in stage 16.0 (TID 486) in 732 ms on 10.0.0.133 (executor driver) (153/200) -26/04/01 08:39:05 INFO Executor: Running task 160.0 in stage 16.0 (TID 494) -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 153.0 in stage 16.0 (TID 487). 7223 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 161.0 in stage 16.0 (TID 495) (10.0.0.133, executor driver, partition 161, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 153.0 in stage 16.0 (TID 487) in 706 ms on 10.0.0.133 (executor driver) (154/200) -26/04/01 08:39:05 INFO Executor: Running task 161.0 in stage 16.0 (TID 495) -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 154.0 in stage 16.0 (TID 488). 7223 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 162.0 in stage 16.0 (TID 496) (10.0.0.133, executor driver, partition 162, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 154.0 in stage 16.0 (TID 488) in 709 ms on 10.0.0.133 (executor driver) (155/200) -26/04/01 08:39:05 INFO Executor: Running task 162.0 in stage 16.0 (TID 496) -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 155.0 in stage 16.0 (TID 489). 7223 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 163.0 in stage 16.0 (TID 497) (10.0.0.133, executor driver, partition 163, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 155.0 in stage 16.0 (TID 489) in 663 ms on 10.0.0.133 (executor driver) (156/200) -26/04/01 08:39:05 INFO Executor: Running task 163.0 in stage 16.0 (TID 497) -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 156.0 in stage 16.0 (TID 490). 7223 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 164.0 in stage 16.0 (TID 498) (10.0.0.133, executor driver, partition 164, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO Executor: Running task 164.0 in stage 16.0 (TID 498) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 156.0 in stage 16.0 (TID 490) in 660 ms on 10.0.0.133 (executor driver) (157/200) -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 157.0 in stage 16.0 (TID 491). 7266 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 165.0 in stage 16.0 (TID 499) (10.0.0.133, executor driver, partition 165, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO Executor: Running task 165.0 in stage 16.0 (TID 499) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 157.0 in stage 16.0 (TID 491) in 689 ms on 10.0.0.133 (executor driver) (158/200) -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 158.0 in stage 16.0 (TID 492). 7309 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 166.0 in stage 16.0 (TID 500) (10.0.0.133, executor driver, partition 166, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO Executor: Running task 166.0 in stage 16.0 (TID 500) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 158.0 in stage 16.0 (TID 492) in 689 ms on 10.0.0.133 (executor driver) (159/200) -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 159.0 in stage 16.0 (TID 493). 7266 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 167.0 in stage 16.0 (TID 501) (10.0.0.133, executor driver, partition 167, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 159.0 in stage 16.0 (TID 493) in 695 ms on 10.0.0.133 (executor driver) (160/200) -26/04/01 08:39:05 INFO Executor: Running task 167.0 in stage 16.0 (TID 501) -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 160.0 in stage 16.0 (TID 494). 7266 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 168.0 in stage 16.0 (TID 502) (10.0.0.133, executor driver, partition 168, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 160.0 in stage 16.0 (TID 494) in 677 ms on 10.0.0.133 (executor driver) (161/200) -26/04/01 08:39:05 INFO Executor: Running task 168.0 in stage 16.0 (TID 502) -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.3 KiB) non-empty blocks including 8 (28.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 161.0 in stage 16.0 (TID 495). 7266 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 169.0 in stage 16.0 (TID 503) (10.0.0.133, executor driver, partition 169, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 161.0 in stage 16.0 (TID 495) in 702 ms on 10.0.0.133 (executor driver) (162/200) -26/04/01 08:39:05 INFO Executor: Running task 169.0 in stage 16.0 (TID 503) -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 163.0 in stage 16.0 (TID 497). 7266 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 170.0 in stage 16.0 (TID 504) (10.0.0.133, executor driver, partition 170, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO Executor: Running task 170.0 in stage 16.0 (TID 504) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 163.0 in stage 16.0 (TID 497) in 693 ms on 10.0.0.133 (executor driver) (163/200) -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 162.0 in stage 16.0 (TID 496). 7266 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 171.0 in stage 16.0 (TID 505) (10.0.0.133, executor driver, partition 171, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO Executor: Running task 171.0 in stage 16.0 (TID 505) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 162.0 in stage 16.0 (TID 496) in 717 ms on 10.0.0.133 (executor driver) (164/200) -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:05 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:05 INFO Executor: Finished task 164.0 in stage 16.0 (TID 498). 7266 bytes result sent to driver -26/04/01 08:39:05 INFO TaskSetManager: Starting task 172.0 in stage 16.0 (TID 506) (10.0.0.133, executor driver, partition 172, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:05 INFO Executor: Running task 172.0 in stage 16.0 (TID 506) -26/04/01 08:39:05 INFO TaskSetManager: Finished task 164.0 in stage 16.0 (TID 498) in 688 ms on 10.0.0.133 (executor driver) (165/200) -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO Executor: Finished task 165.0 in stage 16.0 (TID 499). 7223 bytes result sent to driver -26/04/01 08:39:06 INFO TaskSetManager: Starting task 173.0 in stage 16.0 (TID 507) (10.0.0.133, executor driver, partition 173, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:06 INFO Executor: Running task 173.0 in stage 16.0 (TID 507) -26/04/01 08:39:06 INFO TaskSetManager: Finished task 165.0 in stage 16.0 (TID 499) in 746 ms on 10.0.0.133 (executor driver) (166/200) -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO Executor: Finished task 167.0 in stage 16.0 (TID 501). 7223 bytes result sent to driver -26/04/01 08:39:06 INFO TaskSetManager: Starting task 174.0 in stage 16.0 (TID 508) (10.0.0.133, executor driver, partition 174, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:06 INFO Executor: Running task 174.0 in stage 16.0 (TID 508) -26/04/01 08:39:06 INFO TaskSetManager: Finished task 167.0 in stage 16.0 (TID 501) in 722 ms on 10.0.0.133 (executor driver) (167/200) -26/04/01 08:39:06 INFO Executor: Finished task 166.0 in stage 16.0 (TID 500). 7223 bytes result sent to driver -26/04/01 08:39:06 INFO TaskSetManager: Starting task 175.0 in stage 16.0 (TID 509) (10.0.0.133, executor driver, partition 175, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:06 INFO Executor: Running task 175.0 in stage 16.0 (TID 509) -26/04/01 08:39:06 INFO TaskSetManager: Finished task 166.0 in stage 16.0 (TID 500) in 736 ms on 10.0.0.133 (executor driver) (168/200) -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO Executor: Finished task 168.0 in stage 16.0 (TID 502). 7223 bytes result sent to driver -26/04/01 08:39:06 INFO TaskSetManager: Starting task 176.0 in stage 16.0 (TID 510) (10.0.0.133, executor driver, partition 176, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:06 INFO Executor: Running task 176.0 in stage 16.0 (TID 510) -26/04/01 08:39:06 INFO TaskSetManager: Finished task 168.0 in stage 16.0 (TID 502) in 721 ms on 10.0.0.133 (executor driver) (169/200) -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO Executor: Finished task 169.0 in stage 16.0 (TID 503). 7266 bytes result sent to driver -26/04/01 08:39:06 INFO TaskSetManager: Starting task 177.0 in stage 16.0 (TID 511) (10.0.0.133, executor driver, partition 177, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:06 INFO TaskSetManager: Finished task 169.0 in stage 16.0 (TID 503) in 776 ms on 10.0.0.133 (executor driver) (170/200) -26/04/01 08:39:06 INFO Executor: Running task 177.0 in stage 16.0 (TID 511) -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO Executor: Finished task 170.0 in stage 16.0 (TID 504). 7223 bytes result sent to driver -26/04/01 08:39:06 INFO TaskSetManager: Starting task 178.0 in stage 16.0 (TID 512) (10.0.0.133, executor driver, partition 178, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:06 INFO TaskSetManager: Finished task 170.0 in stage 16.0 (TID 504) in 776 ms on 10.0.0.133 (executor driver) (171/200) -26/04/01 08:39:06 INFO Executor: Running task 178.0 in stage 16.0 (TID 512) -26/04/01 08:39:06 INFO Executor: Finished task 171.0 in stage 16.0 (TID 505). 7223 bytes result sent to driver -26/04/01 08:39:06 INFO TaskSetManager: Starting task 179.0 in stage 16.0 (TID 513) (10.0.0.133, executor driver, partition 179, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:06 INFO Executor: Running task 179.0 in stage 16.0 (TID 513) -26/04/01 08:39:06 INFO TaskSetManager: Finished task 171.0 in stage 16.0 (TID 505) in 774 ms on 10.0.0.133 (executor driver) (172/200) -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO Executor: Finished task 172.0 in stage 16.0 (TID 506). 7223 bytes result sent to driver -26/04/01 08:39:06 INFO TaskSetManager: Starting task 180.0 in stage 16.0 (TID 514) (10.0.0.133, executor driver, partition 180, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:06 INFO Executor: Running task 180.0 in stage 16.0 (TID 514) -26/04/01 08:39:06 INFO TaskSetManager: Finished task 172.0 in stage 16.0 (TID 506) in 763 ms on 10.0.0.133 (executor driver) (173/200) -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:06 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO Executor: Finished task 173.0 in stage 16.0 (TID 507). 7266 bytes result sent to driver -26/04/01 08:39:07 INFO TaskSetManager: Starting task 181.0 in stage 16.0 (TID 515) (10.0.0.133, executor driver, partition 181, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:07 INFO Executor: Running task 181.0 in stage 16.0 (TID 515) -26/04/01 08:39:07 INFO TaskSetManager: Finished task 173.0 in stage 16.0 (TID 507) in 724 ms on 10.0.0.133 (executor driver) (174/200) -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (28.7 KiB) non-empty blocks including 8 (28.7 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO Executor: Finished task 174.0 in stage 16.0 (TID 508). 7266 bytes result sent to driver -26/04/01 08:39:07 INFO TaskSetManager: Starting task 182.0 in stage 16.0 (TID 516) (10.0.0.133, executor driver, partition 182, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:07 INFO Executor: Running task 182.0 in stage 16.0 (TID 516) -26/04/01 08:39:07 INFO TaskSetManager: Finished task 174.0 in stage 16.0 (TID 508) in 721 ms on 10.0.0.133 (executor driver) (175/200) -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO Executor: Finished task 176.0 in stage 16.0 (TID 510). 7266 bytes result sent to driver -26/04/01 08:39:07 INFO TaskSetManager: Starting task 183.0 in stage 16.0 (TID 517) (10.0.0.133, executor driver, partition 183, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:07 INFO TaskSetManager: Finished task 176.0 in stage 16.0 (TID 510) in 718 ms on 10.0.0.133 (executor driver) (176/200) -26/04/01 08:39:07 INFO Executor: Running task 183.0 in stage 16.0 (TID 517) -26/04/01 08:39:07 INFO Executor: Finished task 175.0 in stage 16.0 (TID 509). 7266 bytes result sent to driver -26/04/01 08:39:07 INFO TaskSetManager: Starting task 184.0 in stage 16.0 (TID 518) (10.0.0.133, executor driver, partition 184, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:07 INFO TaskSetManager: Finished task 175.0 in stage 16.0 (TID 509) in 727 ms on 10.0.0.133 (executor driver) (177/200) -26/04/01 08:39:07 INFO Executor: Running task 184.0 in stage 16.0 (TID 518) -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO Executor: Finished task 177.0 in stage 16.0 (TID 511). 7266 bytes result sent to driver -26/04/01 08:39:07 INFO TaskSetManager: Starting task 185.0 in stage 16.0 (TID 519) (10.0.0.133, executor driver, partition 185, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:07 INFO Executor: Running task 185.0 in stage 16.0 (TID 519) -26/04/01 08:39:07 INFO TaskSetManager: Finished task 177.0 in stage 16.0 (TID 511) in 691 ms on 10.0.0.133 (executor driver) (178/200) -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO Executor: Finished task 178.0 in stage 16.0 (TID 512). 7309 bytes result sent to driver -26/04/01 08:39:07 INFO TaskSetManager: Starting task 186.0 in stage 16.0 (TID 520) (10.0.0.133, executor driver, partition 186, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:07 INFO Executor: Running task 186.0 in stage 16.0 (TID 520) -26/04/01 08:39:07 INFO TaskSetManager: Finished task 178.0 in stage 16.0 (TID 512) in 695 ms on 10.0.0.133 (executor driver) (179/200) -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (28.4 KiB) non-empty blocks including 8 (28.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO Executor: Finished task 179.0 in stage 16.0 (TID 513). 7266 bytes result sent to driver -26/04/01 08:39:07 INFO TaskSetManager: Starting task 187.0 in stage 16.0 (TID 521) (10.0.0.133, executor driver, partition 187, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:07 INFO Executor: Running task 187.0 in stage 16.0 (TID 521) -26/04/01 08:39:07 INFO TaskSetManager: Finished task 179.0 in stage 16.0 (TID 513) in 696 ms on 10.0.0.133 (executor driver) (180/200) -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO Executor: Finished task 180.0 in stage 16.0 (TID 514). 7266 bytes result sent to driver -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO TaskSetManager: Starting task 188.0 in stage 16.0 (TID 522) (10.0.0.133, executor driver, partition 188, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:07 INFO Executor: Running task 188.0 in stage 16.0 (TID 522) -26/04/01 08:39:07 INFO TaskSetManager: Finished task 180.0 in stage 16.0 (TID 514) in 693 ms on 10.0.0.133 (executor driver) (181/200) -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (30.2 KiB) non-empty blocks including 8 (30.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO Executor: Finished task 181.0 in stage 16.0 (TID 515). 7223 bytes result sent to driver -26/04/01 08:39:07 INFO TaskSetManager: Starting task 189.0 in stage 16.0 (TID 523) (10.0.0.133, executor driver, partition 189, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:07 INFO Executor: Running task 189.0 in stage 16.0 (TID 523) -26/04/01 08:39:07 INFO TaskSetManager: Finished task 181.0 in stage 16.0 (TID 515) in 671 ms on 10.0.0.133 (executor driver) (182/200) -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO Executor: Finished task 182.0 in stage 16.0 (TID 516). 7223 bytes result sent to driver -26/04/01 08:39:07 INFO TaskSetManager: Starting task 190.0 in stage 16.0 (TID 524) (10.0.0.133, executor driver, partition 190, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:07 INFO Executor: Running task 190.0 in stage 16.0 (TID 524) -26/04/01 08:39:07 INFO TaskSetManager: Finished task 182.0 in stage 16.0 (TID 516) in 681 ms on 10.0.0.133 (executor driver) (183/200) -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:07 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO Executor: Finished task 183.0 in stage 16.0 (TID 517). 7223 bytes result sent to driver -26/04/01 08:39:08 INFO TaskSetManager: Starting task 191.0 in stage 16.0 (TID 525) (10.0.0.133, executor driver, partition 191, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:08 INFO Executor: Running task 191.0 in stage 16.0 (TID 525) -26/04/01 08:39:08 INFO TaskSetManager: Finished task 183.0 in stage 16.0 (TID 517) in 686 ms on 10.0.0.133 (executor driver) (184/200) -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO Executor: Finished task 184.0 in stage 16.0 (TID 518). 7266 bytes result sent to driver -26/04/01 08:39:08 INFO TaskSetManager: Starting task 192.0 in stage 16.0 (TID 526) (10.0.0.133, executor driver, partition 192, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:08 INFO TaskSetManager: Finished task 184.0 in stage 16.0 (TID 518) in 692 ms on 10.0.0.133 (executor driver) (185/200) -26/04/01 08:39:08 INFO Executor: Running task 192.0 in stage 16.0 (TID 526) -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (28.1 KiB) non-empty blocks including 8 (28.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO Executor: Finished task 185.0 in stage 16.0 (TID 519). 7266 bytes result sent to driver -26/04/01 08:39:08 INFO TaskSetManager: Starting task 193.0 in stage 16.0 (TID 527) (10.0.0.133, executor driver, partition 193, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:08 INFO Executor: Running task 193.0 in stage 16.0 (TID 527) -26/04/01 08:39:08 INFO TaskSetManager: Finished task 185.0 in stage 16.0 (TID 519) in 707 ms on 10.0.0.133 (executor driver) (186/200) -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO Executor: Finished task 186.0 in stage 16.0 (TID 520). 7223 bytes result sent to driver -26/04/01 08:39:08 INFO TaskSetManager: Starting task 194.0 in stage 16.0 (TID 528) (10.0.0.133, executor driver, partition 194, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:08 INFO Executor: Running task 194.0 in stage 16.0 (TID 528) -26/04/01 08:39:08 INFO TaskSetManager: Finished task 186.0 in stage 16.0 (TID 520) in 708 ms on 10.0.0.133 (executor driver) (187/200) -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO Executor: Finished task 187.0 in stage 16.0 (TID 521). 7223 bytes result sent to driver -26/04/01 08:39:08 INFO TaskSetManager: Starting task 195.0 in stage 16.0 (TID 529) (10.0.0.133, executor driver, partition 195, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:08 INFO TaskSetManager: Finished task 187.0 in stage 16.0 (TID 521) in 718 ms on 10.0.0.133 (executor driver) (188/200) -26/04/01 08:39:08 INFO Executor: Running task 195.0 in stage 16.0 (TID 529) -26/04/01 08:39:08 INFO Executor: Finished task 188.0 in stage 16.0 (TID 522). 7223 bytes result sent to driver -26/04/01 08:39:08 INFO TaskSetManager: Starting task 196.0 in stage 16.0 (TID 530) (10.0.0.133, executor driver, partition 196, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:08 INFO Executor: Running task 196.0 in stage 16.0 (TID 530) -26/04/01 08:39:08 INFO TaskSetManager: Finished task 188.0 in stage 16.0 (TID 522) in 716 ms on 10.0.0.133 (executor driver) (189/200) -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (30.1 KiB) non-empty blocks including 8 (30.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO Executor: Finished task 189.0 in stage 16.0 (TID 523). 7266 bytes result sent to driver -26/04/01 08:39:08 INFO TaskSetManager: Starting task 197.0 in stage 16.0 (TID 531) (10.0.0.133, executor driver, partition 197, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:08 INFO Executor: Running task 197.0 in stage 16.0 (TID 531) -26/04/01 08:39:08 INFO TaskSetManager: Finished task 189.0 in stage 16.0 (TID 523) in 701 ms on 10.0.0.133 (executor driver) (190/200) -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (29.1 KiB) non-empty blocks including 8 (29.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO Executor: Finished task 190.0 in stage 16.0 (TID 524). 7266 bytes result sent to driver -26/04/01 08:39:08 INFO TaskSetManager: Starting task 198.0 in stage 16.0 (TID 532) (10.0.0.133, executor driver, partition 198, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:08 INFO Executor: Running task 198.0 in stage 16.0 (TID 532) -26/04/01 08:39:08 INFO TaskSetManager: Finished task 190.0 in stage 16.0 (TID 524) in 700 ms on 10.0.0.133 (executor driver) (191/200) -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (29.4 KiB) non-empty blocks including 8 (29.4 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO Executor: Finished task 191.0 in stage 16.0 (TID 525). 7266 bytes result sent to driver -26/04/01 08:39:08 INFO TaskSetManager: Starting task 199.0 in stage 16.0 (TID 533) (10.0.0.133, executor driver, partition 199, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:08 INFO TaskSetManager: Finished task 191.0 in stage 16.0 (TID 525) in 699 ms on 10.0.0.133 (executor driver) (192/200) -26/04/01 08:39:08 INFO Executor: Running task 199.0 in stage 16.0 (TID 533) -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 8 (29.8 KiB) non-empty blocks including 8 (29.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Getting 208 (90.2 MiB) non-empty blocks including 208 (90.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:08 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:08 INFO Executor: Finished task 192.0 in stage 16.0 (TID 526). 7266 bytes result sent to driver -26/04/01 08:39:08 INFO TaskSetManager: Finished task 192.0 in stage 16.0 (TID 526) in 695 ms on 10.0.0.133 (executor driver) (193/200) -26/04/01 08:39:08 INFO Executor: Finished task 193.0 in stage 16.0 (TID 527). 7266 bytes result sent to driver -26/04/01 08:39:08 INFO TaskSetManager: Finished task 193.0 in stage 16.0 (TID 527) in 668 ms on 10.0.0.133 (executor driver) (194/200) -26/04/01 08:39:08 INFO Executor: Finished task 194.0 in stage 16.0 (TID 528). 7266 bytes result sent to driver -26/04/01 08:39:08 INFO TaskSetManager: Finished task 194.0 in stage 16.0 (TID 528) in 694 ms on 10.0.0.133 (executor driver) (195/200) -26/04/01 08:39:08 INFO Executor: Finished task 195.0 in stage 16.0 (TID 529). 7266 bytes result sent to driver -26/04/01 08:39:08 INFO TaskSetManager: Finished task 195.0 in stage 16.0 (TID 529) in 696 ms on 10.0.0.133 (executor driver) (196/200) -26/04/01 08:39:08 INFO Executor: Finished task 196.0 in stage 16.0 (TID 530). 7266 bytes result sent to driver -26/04/01 08:39:08 INFO TaskSetManager: Finished task 196.0 in stage 16.0 (TID 530) in 698 ms on 10.0.0.133 (executor driver) (197/200) -26/04/01 08:39:09 INFO Executor: Finished task 197.0 in stage 16.0 (TID 531). 7223 bytes result sent to driver -26/04/01 08:39:09 INFO TaskSetManager: Finished task 197.0 in stage 16.0 (TID 531) in 779 ms on 10.0.0.133 (executor driver) (198/200) -26/04/01 08:39:09 INFO Executor: Finished task 198.0 in stage 16.0 (TID 532). 7223 bytes result sent to driver -26/04/01 08:39:09 INFO TaskSetManager: Finished task 198.0 in stage 16.0 (TID 532) in 774 ms on 10.0.0.133 (executor driver) (199/200) -26/04/01 08:39:09 INFO Executor: Finished task 199.0 in stage 16.0 (TID 533). 7223 bytes result sent to driver -26/04/01 08:39:09 INFO TaskSetManager: Finished task 199.0 in stage 16.0 (TID 533) in 783 ms on 10.0.0.133 (executor driver) (200/200) -26/04/01 08:39:09 INFO TaskSchedulerImpl: Removed TaskSet 16.0, whose tasks have all completed, from pool -26/04/01 08:39:09 INFO DAGScheduler: ShuffleMapStage 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 28.434 s -26/04/01 08:39:09 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:39:09 INFO DAGScheduler: running: Set() -26/04/01 08:39:09 INFO DAGScheduler: waiting: Set() -26/04/01 08:39:09 INFO DAGScheduler: failed: Set() -26/04/01 08:39:09 INFO ShufflePartitionsUtil: For shuffle(5, 2), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 08:39:09 INFO CodeGenerator: Code generated in 4.300083 ms -26/04/01 08:39:09 INFO CodeGenerator: Code generated in 2.088959 ms -26/04/01 08:39:09 INFO CodeGenerator: Code generated in 2.233375 ms -26/04/01 08:39:09 INFO DAGScheduler: Registering RDD 53 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 6 -26/04/01 08:39:09 INFO DAGScheduler: Got map stage job 15 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 13 output partitions -26/04/01 08:39:09 INFO DAGScheduler: Final stage: ShuffleMapStage 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:39:09 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 19, ShuffleMapStage 20) -26/04/01 08:39:09 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:09 INFO DAGScheduler: Submitting ShuffleMapStage 21 (MapPartitionsRDD[53] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:39:09 INFO MemoryStore: Block broadcast_22 stored as values in memory (estimated size 65.8 KiB, free 8.6 GiB) -26/04/01 08:39:09 INFO MemoryStore: Block broadcast_22_piece0 stored as bytes in memory (estimated size 30.0 KiB, free 8.6 GiB) -26/04/01 08:39:09 INFO BlockManagerInfo: Added broadcast_22_piece0 in memory on 10.0.0.133:59288 (size: 30.0 KiB, free: 8.6 GiB) -26/04/01 08:39:09 INFO SparkContext: Created broadcast 22 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:09 INFO DAGScheduler: Submitting 13 missing tasks from ShuffleMapStage 21 (MapPartitionsRDD[53] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) -26/04/01 08:39:09 INFO TaskSchedulerImpl: Adding task set 21.0 with 13 tasks resource profile 0 -26/04/01 08:39:09 INFO TaskSetManager: Starting task 0.0 in stage 21.0 (TID 534) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:09 INFO TaskSetManager: Starting task 1.0 in stage 21.0 (TID 535) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:09 INFO TaskSetManager: Starting task 2.0 in stage 21.0 (TID 536) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:09 INFO TaskSetManager: Starting task 3.0 in stage 21.0 (TID 537) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:09 INFO TaskSetManager: Starting task 4.0 in stage 21.0 (TID 538) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:09 INFO TaskSetManager: Starting task 5.0 in stage 21.0 (TID 539) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:09 INFO TaskSetManager: Starting task 6.0 in stage 21.0 (TID 540) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:09 INFO TaskSetManager: Starting task 7.0 in stage 21.0 (TID 541) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:09 INFO Executor: Running task 1.0 in stage 21.0 (TID 535) -26/04/01 08:39:09 INFO Executor: Running task 0.0 in stage 21.0 (TID 534) -26/04/01 08:39:09 INFO Executor: Running task 3.0 in stage 21.0 (TID 537) -26/04/01 08:39:09 INFO Executor: Running task 7.0 in stage 21.0 (TID 541) -26/04/01 08:39:09 INFO Executor: Running task 5.0 in stage 21.0 (TID 539) -26/04/01 08:39:09 INFO Executor: Running task 6.0 in stage 21.0 (TID 540) -26/04/01 08:39:09 INFO Executor: Running task 4.0 in stage 21.0 (TID 538) -26/04/01 08:39:09 INFO Executor: Running task 2.0 in stage 21.0 (TID 536) -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (59.6 MiB) non-empty blocks including 200 (59.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (59.7 MiB) non-empty blocks including 200 (59.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 4 ms -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (59.5 MiB) non-empty blocks including 200 (59.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (63.0 MiB) non-empty blocks including 200 (63.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (59.5 MiB) non-empty blocks including 200 (59.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (62.9 MiB) non-empty blocks including 200 (62.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (63.2 MiB) non-empty blocks including 200 (63.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 200 (63.1 MiB) non-empty blocks including 200 (63.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 5 ms -26/04/01 08:39:09 INFO CodeGenerator: Code generated in 2.935459 ms -26/04/01 08:39:09 INFO CodeGenerator: Code generated in 1.423291 ms -26/04/01 08:39:09 INFO CodeGenerator: Code generated in 1.563958 ms -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (699.8 KiB) non-empty blocks including 8 (699.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (741.9 KiB) non-empty blocks including 8 (741.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (743.3 KiB) non-empty blocks including 8 (743.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (698.8 KiB) non-empty blocks including 8 (698.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (704.9 KiB) non-empty blocks including 8 (704.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (701.5 KiB) non-empty blocks including 8 (701.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (741.5 KiB) non-empty blocks including 8 (741.5 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Getting 8 (743.9 KiB) non-empty blocks including 8 (743.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:09 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:09 INFO CodeGenerator: Code generated in 2.077167 ms -26/04/01 08:39:09 INFO CodeGenerator: Code generated in 1.392583 ms -26/04/01 08:39:09 INFO CodeGenerator: Code generated in 1.329041 ms -26/04/01 08:39:09 INFO CodeGenerator: Code generated in 3.886667 ms -26/04/01 08:39:09 INFO CodeGenerator: Code generated in 2.125125 ms -26/04/01 08:39:12 INFO Executor: Finished task 6.0 in stage 21.0 (TID 540). 10893 bytes result sent to driver -26/04/01 08:39:12 INFO TaskSetManager: Starting task 8.0 in stage 21.0 (TID 542) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:12 INFO Executor: Running task 8.0 in stage 21.0 (TID 542) -26/04/01 08:39:12 INFO TaskSetManager: Finished task 6.0 in stage 21.0 (TID 540) in 3084 ms on 10.0.0.133 (executor driver) (1/13) -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 200 (59.6 MiB) non-empty blocks including 200 (59.6 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 8 (703.8 KiB) non-empty blocks including 8 (703.8 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:12 INFO Executor: Finished task 3.0 in stage 21.0 (TID 537). 10893 bytes result sent to driver -26/04/01 08:39:12 INFO TaskSetManager: Starting task 9.0 in stage 21.0 (TID 543) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:12 INFO Executor: Running task 9.0 in stage 21.0 (TID 543) -26/04/01 08:39:12 INFO TaskSetManager: Finished task 3.0 in stage 21.0 (TID 537) in 3349 ms on 10.0.0.133 (executor driver) (2/13) -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 200 (63.3 MiB) non-empty blocks including 200 (63.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 8 (742.1 KiB) non-empty blocks including 8 (742.1 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:12 INFO Executor: Finished task 0.0 in stage 21.0 (TID 534). 10893 bytes result sent to driver -26/04/01 08:39:12 INFO TaskSetManager: Starting task 10.0 in stage 21.0 (TID 544) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:12 INFO TaskSetManager: Finished task 0.0 in stage 21.0 (TID 534) in 3413 ms on 10.0.0.133 (executor driver) (3/13) -26/04/01 08:39:12 INFO Executor: Running task 10.0 in stage 21.0 (TID 544) -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 200 (59.4 MiB) non-empty blocks including 200 (59.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 8 (699.9 KiB) non-empty blocks including 8 (699.9 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:12 INFO Executor: Finished task 5.0 in stage 21.0 (TID 539). 10893 bytes result sent to driver -26/04/01 08:39:12 INFO TaskSetManager: Starting task 11.0 in stage 21.0 (TID 545) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:12 INFO TaskSetManager: Finished task 5.0 in stage 21.0 (TID 539) in 3447 ms on 10.0.0.133 (executor driver) (4/13) -26/04/01 08:39:12 INFO Executor: Running task 11.0 in stage 21.0 (TID 545) -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 200 (59.5 MiB) non-empty blocks including 200 (59.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 8 (696.0 KiB) non-empty blocks including 8 (696.0 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:12 INFO Executor: Finished task 1.0 in stage 21.0 (TID 535). 10893 bytes result sent to driver -26/04/01 08:39:12 INFO TaskSetManager: Starting task 12.0 in stage 21.0 (TID 546) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:12 INFO Executor: Running task 12.0 in stage 21.0 (TID 546) -26/04/01 08:39:12 INFO TaskSetManager: Finished task 1.0 in stage 21.0 (TID 535) in 3461 ms on 10.0.0.133 (executor driver) (5/13) -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 200 (59.5 MiB) non-empty blocks including 200 (59.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Getting 8 (699.3 KiB) non-empty blocks including 8 (699.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:12 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:13 INFO Executor: Finished task 2.0 in stage 21.0 (TID 536). 10893 bytes result sent to driver -26/04/01 08:39:13 INFO TaskSetManager: Finished task 2.0 in stage 21.0 (TID 536) in 3472 ms on 10.0.0.133 (executor driver) (6/13) -26/04/01 08:39:13 INFO Executor: Finished task 4.0 in stage 21.0 (TID 538). 10893 bytes result sent to driver -26/04/01 08:39:13 INFO TaskSetManager: Finished task 4.0 in stage 21.0 (TID 538) in 3474 ms on 10.0.0.133 (executor driver) (7/13) -26/04/01 08:39:13 INFO Executor: Finished task 7.0 in stage 21.0 (TID 541). 10893 bytes result sent to driver -26/04/01 08:39:13 INFO TaskSetManager: Finished task 7.0 in stage 21.0 (TID 541) in 3474 ms on 10.0.0.133 (executor driver) (8/13) -26/04/01 08:39:14 INFO Executor: Finished task 8.0 in stage 21.0 (TID 542). 10893 bytes result sent to driver -26/04/01 08:39:14 INFO TaskSetManager: Finished task 8.0 in stage 21.0 (TID 542) in 1539 ms on 10.0.0.133 (executor driver) (9/13) -26/04/01 08:39:14 INFO Executor: Finished task 11.0 in stage 21.0 (TID 545). 10850 bytes result sent to driver -26/04/01 08:39:14 INFO TaskSetManager: Finished task 11.0 in stage 21.0 (TID 545) in 1601 ms on 10.0.0.133 (executor driver) (10/13) -26/04/01 08:39:14 INFO Executor: Finished task 10.0 in stage 21.0 (TID 544). 10893 bytes result sent to driver -26/04/01 08:39:14 INFO TaskSetManager: Finished task 10.0 in stage 21.0 (TID 544) in 1654 ms on 10.0.0.133 (executor driver) (11/13) -26/04/01 08:39:14 INFO Executor: Finished task 9.0 in stage 21.0 (TID 543). 10893 bytes result sent to driver -26/04/01 08:39:14 INFO TaskSetManager: Finished task 9.0 in stage 21.0 (TID 543) in 1767 ms on 10.0.0.133 (executor driver) (12/13) -26/04/01 08:39:14 INFO Executor: Finished task 12.0 in stage 21.0 (TID 546). 10893 bytes result sent to driver -26/04/01 08:39:14 INFO TaskSetManager: Finished task 12.0 in stage 21.0 (TID 546) in 1661 ms on 10.0.0.133 (executor driver) (13/13) -26/04/01 08:39:14 INFO TaskSchedulerImpl: Removed TaskSet 21.0, whose tasks have all completed, from pool -26/04/01 08:39:14 INFO DAGScheduler: ShuffleMapStage 21 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 5.122 s -26/04/01 08:39:14 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:39:14 INFO DAGScheduler: running: Set() -26/04/01 08:39:14 INFO DAGScheduler: waiting: Set() -26/04/01 08:39:14 INFO DAGScheduler: failed: Set() -26/04/01 08:39:14 INFO ShufflePartitionsUtil: For shuffle(6, 3), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 08:39:14 INFO CodeGenerator: Code generated in 3.866792 ms -26/04/01 08:39:14 INFO CodeGenerator: Code generated in 2.012292 ms -26/04/01 08:39:14 INFO CodeGenerator: Code generated in 1.779375 ms -26/04/01 08:39:14 INFO DAGScheduler: Registering RDD 60 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 7 -26/04/01 08:39:14 INFO DAGScheduler: Got map stage job 16 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 34 output partitions -26/04/01 08:39:14 INFO DAGScheduler: Final stage: ShuffleMapStage 28 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:39:14 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 27, ShuffleMapStage 26) -26/04/01 08:39:14 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:14 INFO DAGScheduler: Submitting ShuffleMapStage 28 (MapPartitionsRDD[60] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:39:14 INFO MemoryStore: Block broadcast_23 stored as values in memory (estimated size 77.9 KiB, free 8.6 GiB) -26/04/01 08:39:14 INFO MemoryStore: Block broadcast_23_piece0 stored as bytes in memory (estimated size 33.7 KiB, free 8.6 GiB) -26/04/01 08:39:14 INFO BlockManagerInfo: Added broadcast_23_piece0 in memory on 10.0.0.133:59288 (size: 33.7 KiB, free: 8.6 GiB) -26/04/01 08:39:14 INFO SparkContext: Created broadcast 23 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:14 INFO DAGScheduler: Submitting 34 missing tasks from ShuffleMapStage 28 (MapPartitionsRDD[60] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:39:14 INFO TaskSchedulerImpl: Adding task set 28.0 with 34 tasks resource profile 0 -26/04/01 08:39:14 INFO TaskSetManager: Starting task 0.0 in stage 28.0 (TID 547) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:14 INFO TaskSetManager: Starting task 1.0 in stage 28.0 (TID 548) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:14 INFO TaskSetManager: Starting task 2.0 in stage 28.0 (TID 549) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:14 INFO TaskSetManager: Starting task 3.0 in stage 28.0 (TID 550) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:14 INFO TaskSetManager: Starting task 4.0 in stage 28.0 (TID 551) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:14 INFO TaskSetManager: Starting task 5.0 in stage 28.0 (TID 552) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:14 INFO TaskSetManager: Starting task 6.0 in stage 28.0 (TID 553) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:14 INFO TaskSetManager: Starting task 7.0 in stage 28.0 (TID 554) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:14 INFO Executor: Running task 2.0 in stage 28.0 (TID 549) -26/04/01 08:39:14 INFO Executor: Running task 5.0 in stage 28.0 (TID 552) -26/04/01 08:39:14 INFO Executor: Running task 3.0 in stage 28.0 (TID 550) -26/04/01 08:39:14 INFO Executor: Running task 1.0 in stage 28.0 (TID 548) -26/04/01 08:39:14 INFO Executor: Running task 6.0 in stage 28.0 (TID 553) -26/04/01 08:39:14 INFO Executor: Running task 4.0 in stage 28.0 (TID 551) -26/04/01 08:39:14 INFO Executor: Running task 0.0 in stage 28.0 (TID 547) -26/04/01 08:39:14 INFO Executor: Running task 7.0 in stage 28.0 (TID 554) -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO CodeGenerator: Code generated in 2.429166 ms -26/04/01 08:39:14 INFO CodeGenerator: Code generated in 1.740458 ms -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:14 INFO CodeGenerator: Code generated in 2.482916 ms -26/04/01 08:39:14 INFO CodeGenerator: Code generated in 1.349833 ms -26/04/01 08:39:14 INFO CodeGenerator: Code generated in 3.366125 ms -26/04/01 08:39:15 INFO BlockManagerInfo: Removed broadcast_22_piece0 on 10.0.0.133:59288 in memory (size: 30.0 KiB, free: 8.6 GiB) -26/04/01 08:39:16 INFO Executor: Finished task 0.0 in stage 28.0 (TID 547). 14477 bytes result sent to driver -26/04/01 08:39:16 INFO TaskSetManager: Starting task 8.0 in stage 28.0 (TID 555) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:16 INFO TaskSetManager: Finished task 0.0 in stage 28.0 (TID 547) in 1938 ms on 10.0.0.133 (executor driver) (1/34) -26/04/01 08:39:16 INFO Executor: Running task 8.0 in stage 28.0 (TID 555) -26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:16 INFO Executor: Finished task 5.0 in stage 28.0 (TID 552). 14477 bytes result sent to driver -26/04/01 08:39:16 INFO TaskSetManager: Starting task 9.0 in stage 28.0 (TID 556) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:16 INFO Executor: Running task 9.0 in stage 28.0 (TID 556) -26/04/01 08:39:16 INFO TaskSetManager: Finished task 5.0 in stage 28.0 (TID 552) in 2307 ms on 10.0.0.133 (executor driver) (2/34) -26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Getting 13 (24.9 MiB) non-empty blocks including 13 (24.9 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:16 INFO Executor: Finished task 2.0 in stage 28.0 (TID 549). 14477 bytes result sent to driver -26/04/01 08:39:16 INFO TaskSetManager: Starting task 10.0 in stage 28.0 (TID 557) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:16 INFO TaskSetManager: Finished task 2.0 in stage 28.0 (TID 549) in 2314 ms on 10.0.0.133 (executor driver) (3/34) -26/04/01 08:39:16 INFO Executor: Running task 10.0 in stage 28.0 (TID 557) -26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:16 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:17 INFO Executor: Finished task 3.0 in stage 28.0 (TID 550). 14477 bytes result sent to driver -26/04/01 08:39:17 INFO TaskSetManager: Starting task 11.0 in stage 28.0 (TID 558) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:17 INFO Executor: Running task 11.0 in stage 28.0 (TID 558) -26/04/01 08:39:17 INFO TaskSetManager: Finished task 3.0 in stage 28.0 (TID 550) in 2329 ms on 10.0.0.133 (executor driver) (4/34) -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 13 (25.0 MiB) non-empty blocks including 13 (25.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:17 INFO Executor: Finished task 4.0 in stage 28.0 (TID 551). 14477 bytes result sent to driver -26/04/01 08:39:17 INFO TaskSetManager: Starting task 12.0 in stage 28.0 (TID 559) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:17 INFO TaskSetManager: Finished task 4.0 in stage 28.0 (TID 551) in 2345 ms on 10.0.0.133 (executor driver) (5/34) -26/04/01 08:39:17 INFO Executor: Running task 12.0 in stage 28.0 (TID 559) -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:17 INFO Executor: Finished task 7.0 in stage 28.0 (TID 554). 14477 bytes result sent to driver -26/04/01 08:39:17 INFO TaskSetManager: Starting task 13.0 in stage 28.0 (TID 560) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:17 INFO Executor: Running task 13.0 in stage 28.0 (TID 560) -26/04/01 08:39:17 INFO TaskSetManager: Finished task 7.0 in stage 28.0 (TID 554) in 2348 ms on 10.0.0.133 (executor driver) (6/34) -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 13 (25.4 MiB) non-empty blocks including 13 (25.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:17 INFO Executor: Finished task 1.0 in stage 28.0 (TID 548). 14477 bytes result sent to driver -26/04/01 08:39:17 INFO TaskSetManager: Starting task 14.0 in stage 28.0 (TID 561) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:17 INFO Executor: Running task 14.0 in stage 28.0 (TID 561) -26/04/01 08:39:17 INFO TaskSetManager: Finished task 1.0 in stage 28.0 (TID 548) in 2354 ms on 10.0.0.133 (executor driver) (7/34) -26/04/01 08:39:17 INFO Executor: Finished task 6.0 in stage 28.0 (TID 553). 14477 bytes result sent to driver -26/04/01 08:39:17 INFO TaskSetManager: Starting task 15.0 in stage 28.0 (TID 562) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:17 INFO TaskSetManager: Finished task 6.0 in stage 28.0 (TID 553) in 2355 ms on 10.0.0.133 (executor driver) (8/34) -26/04/01 08:39:17 INFO Executor: Running task 15.0 in stage 28.0 (TID 562) -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:17 INFO Executor: Finished task 8.0 in stage 28.0 (TID 555). 14434 bytes result sent to driver -26/04/01 08:39:17 INFO TaskSetManager: Starting task 16.0 in stage 28.0 (TID 563) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:17 INFO TaskSetManager: Finished task 8.0 in stage 28.0 (TID 555) in 1211 ms on 10.0.0.133 (executor driver) (9/34) -26/04/01 08:39:17 INFO Executor: Running task 16.0 in stage 28.0 (TID 563) -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:17 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:18 INFO Executor: Finished task 10.0 in stage 28.0 (TID 557). 14477 bytes result sent to driver -26/04/01 08:39:18 INFO TaskSetManager: Starting task 17.0 in stage 28.0 (TID 564) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:18 INFO TaskSetManager: Finished task 10.0 in stage 28.0 (TID 557) in 1303 ms on 10.0.0.133 (executor driver) (10/34) -26/04/01 08:39:18 INFO Executor: Running task 17.0 in stage 28.0 (TID 564) -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:18 INFO Executor: Finished task 9.0 in stage 28.0 (TID 556). 14477 bytes result sent to driver -26/04/01 08:39:18 INFO TaskSetManager: Starting task 18.0 in stage 28.0 (TID 565) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:18 INFO Executor: Running task 18.0 in stage 28.0 (TID 565) -26/04/01 08:39:18 INFO TaskSetManager: Finished task 9.0 in stage 28.0 (TID 556) in 1426 ms on 10.0.0.133 (executor driver) (11/34) -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:18 INFO Executor: Finished task 12.0 in stage 28.0 (TID 559). 14477 bytes result sent to driver -26/04/01 08:39:18 INFO TaskSetManager: Starting task 19.0 in stage 28.0 (TID 566) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:18 INFO Executor: Running task 19.0 in stage 28.0 (TID 566) -26/04/01 08:39:18 INFO TaskSetManager: Finished task 12.0 in stage 28.0 (TID 559) in 1448 ms on 10.0.0.133 (executor driver) (12/34) -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:18 INFO Executor: Finished task 11.0 in stage 28.0 (TID 558). 14477 bytes result sent to driver -26/04/01 08:39:18 INFO TaskSetManager: Starting task 20.0 in stage 28.0 (TID 567) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:18 INFO Executor: Running task 20.0 in stage 28.0 (TID 567) -26/04/01 08:39:18 INFO TaskSetManager: Finished task 11.0 in stage 28.0 (TID 558) in 1478 ms on 10.0.0.133 (executor driver) (13/34) -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:18 INFO Executor: Finished task 13.0 in stage 28.0 (TID 560). 14434 bytes result sent to driver -26/04/01 08:39:18 INFO TaskSetManager: Starting task 21.0 in stage 28.0 (TID 568) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:18 INFO Executor: Running task 21.0 in stage 28.0 (TID 568) -26/04/01 08:39:18 INFO TaskSetManager: Finished task 13.0 in stage 28.0 (TID 560) in 1508 ms on 10.0.0.133 (executor driver) (14/34) -26/04/01 08:39:18 INFO Executor: Finished task 14.0 in stage 28.0 (TID 561). 14434 bytes result sent to driver -26/04/01 08:39:18 INFO TaskSetManager: Starting task 22.0 in stage 28.0 (TID 569) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:18 INFO TaskSetManager: Finished task 14.0 in stage 28.0 (TID 561) in 1502 ms on 10.0.0.133 (executor driver) (15/34) -26/04/01 08:39:18 INFO Executor: Running task 22.0 in stage 28.0 (TID 569) -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:18 INFO Executor: Finished task 15.0 in stage 28.0 (TID 562). 14434 bytes result sent to driver -26/04/01 08:39:18 INFO TaskSetManager: Starting task 23.0 in stage 28.0 (TID 570) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:18 INFO TaskSetManager: Finished task 15.0 in stage 28.0 (TID 562) in 1505 ms on 10.0.0.133 (executor driver) (16/34) -26/04/01 08:39:18 INFO Executor: Running task 23.0 in stage 28.0 (TID 570) -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:18 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:19 INFO Executor: Finished task 16.0 in stage 28.0 (TID 563). 14477 bytes result sent to driver -26/04/01 08:39:19 INFO TaskSetManager: Starting task 24.0 in stage 28.0 (TID 571) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:19 INFO Executor: Running task 24.0 in stage 28.0 (TID 571) -26/04/01 08:39:19 INFO TaskSetManager: Finished task 16.0 in stage 28.0 (TID 563) in 1216 ms on 10.0.0.133 (executor driver) (17/34) -26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:19 INFO Executor: Finished task 17.0 in stage 28.0 (TID 564). 14477 bytes result sent to driver -26/04/01 08:39:19 INFO TaskSetManager: Starting task 25.0 in stage 28.0 (TID 572) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:19 INFO Executor: Running task 25.0 in stage 28.0 (TID 572) -26/04/01 08:39:19 INFO TaskSetManager: Finished task 17.0 in stage 28.0 (TID 564) in 1481 ms on 10.0.0.133 (executor driver) (18/34) -26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:19 INFO Executor: Finished task 18.0 in stage 28.0 (TID 565). 14477 bytes result sent to driver -26/04/01 08:39:19 INFO TaskSetManager: Starting task 26.0 in stage 28.0 (TID 573) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:19 INFO Executor: Running task 26.0 in stage 28.0 (TID 573) -26/04/01 08:39:19 INFO TaskSetManager: Finished task 18.0 in stage 28.0 (TID 565) in 1386 ms on 10.0.0.133 (executor driver) (19/34) -26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:19 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:20 INFO Executor: Finished task 20.0 in stage 28.0 (TID 567). 14477 bytes result sent to driver -26/04/01 08:39:20 INFO TaskSetManager: Starting task 27.0 in stage 28.0 (TID 574) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:20 INFO Executor: Running task 27.0 in stage 28.0 (TID 574) -26/04/01 08:39:20 INFO TaskSetManager: Finished task 20.0 in stage 28.0 (TID 567) in 1600 ms on 10.0.0.133 (executor driver) (20/34) -26/04/01 08:39:20 INFO Executor: Finished task 19.0 in stage 28.0 (TID 566). 14477 bytes result sent to driver -26/04/01 08:39:20 INFO TaskSetManager: Starting task 28.0 in stage 28.0 (TID 575) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:20 INFO TaskSetManager: Finished task 19.0 in stage 28.0 (TID 566) in 1613 ms on 10.0.0.133 (executor driver) (21/34) -26/04/01 08:39:20 INFO Executor: Running task 28.0 in stage 28.0 (TID 575) -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 13 (25.0 MiB) non-empty blocks including 13 (25.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:20 INFO Executor: Finished task 21.0 in stage 28.0 (TID 568). 14477 bytes result sent to driver -26/04/01 08:39:20 INFO TaskSetManager: Starting task 29.0 in stage 28.0 (TID 576) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:20 INFO TaskSetManager: Finished task 21.0 in stage 28.0 (TID 568) in 1625 ms on 10.0.0.133 (executor driver) (22/34) -26/04/01 08:39:20 INFO Executor: Running task 29.0 in stage 28.0 (TID 576) -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 13 (25.2 MiB) non-empty blocks including 13 (25.2 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:20 INFO Executor: Finished task 22.0 in stage 28.0 (TID 569). 14477 bytes result sent to driver -26/04/01 08:39:20 INFO TaskSetManager: Starting task 30.0 in stage 28.0 (TID 577) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:20 INFO TaskSetManager: Finished task 22.0 in stage 28.0 (TID 569) in 1632 ms on 10.0.0.133 (executor driver) (23/34) -26/04/01 08:39:20 INFO Executor: Running task 30.0 in stage 28.0 (TID 577) -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:20 INFO Executor: Finished task 23.0 in stage 28.0 (TID 570). 14434 bytes result sent to driver -26/04/01 08:39:20 INFO TaskSetManager: Starting task 31.0 in stage 28.0 (TID 578) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:20 INFO Executor: Running task 31.0 in stage 28.0 (TID 578) -26/04/01 08:39:20 INFO TaskSetManager: Finished task 23.0 in stage 28.0 (TID 570) in 1638 ms on 10.0.0.133 (executor driver) (24/34) -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 13 (25.3 MiB) non-empty blocks including 13 (25.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:20 INFO Executor: Finished task 24.0 in stage 28.0 (TID 571). 14477 bytes result sent to driver -26/04/01 08:39:20 INFO TaskSetManager: Starting task 32.0 in stage 28.0 (TID 579) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:20 INFO Executor: Running task 32.0 in stage 28.0 (TID 579) -26/04/01 08:39:20 INFO TaskSetManager: Finished task 24.0 in stage 28.0 (TID 571) in 1344 ms on 10.0.0.133 (executor driver) (25/34) -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 13 (25.1 MiB) non-empty blocks including 13 (25.1 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Getting 37 (37.0 MiB) non-empty blocks including 37 (37.0 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:20 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO Executor: Finished task 26.0 in stage 28.0 (TID 573). 14477 bytes result sent to driver -26/04/01 08:39:21 INFO TaskSetManager: Starting task 33.0 in stage 28.0 (TID 580) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:21 INFO TaskSetManager: Finished task 26.0 in stage 28.0 (TID 573) in 1497 ms on 10.0.0.133 (executor driver) (26/34) -26/04/01 08:39:21 INFO Executor: Running task 33.0 in stage 28.0 (TID 580) -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 13 (8.5 MiB) non-empty blocks including 13 (8.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 2 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 37 (12.3 MiB) non-empty blocks including 37 (12.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO Executor: Finished task 25.0 in stage 28.0 (TID 572). 14477 bytes result sent to driver -26/04/01 08:39:21 INFO TaskSetManager: Finished task 25.0 in stage 28.0 (TID 572) in 1541 ms on 10.0.0.133 (executor driver) (27/34) -26/04/01 08:39:21 INFO Executor: Finished task 27.0 in stage 28.0 (TID 574). 14477 bytes result sent to driver -26/04/01 08:39:21 INFO TaskSetManager: Finished task 27.0 in stage 28.0 (TID 574) in 1476 ms on 10.0.0.133 (executor driver) (28/34) -26/04/01 08:39:21 INFO Executor: Finished task 28.0 in stage 28.0 (TID 575). 14477 bytes result sent to driver -26/04/01 08:39:21 INFO TaskSetManager: Finished task 28.0 in stage 28.0 (TID 575) in 1523 ms on 10.0.0.133 (executor driver) (29/34) -26/04/01 08:39:21 INFO Executor: Finished task 29.0 in stage 28.0 (TID 576). 14477 bytes result sent to driver -26/04/01 08:39:21 INFO TaskSetManager: Finished task 29.0 in stage 28.0 (TID 576) in 1486 ms on 10.0.0.133 (executor driver) (30/34) -26/04/01 08:39:21 INFO Executor: Finished task 30.0 in stage 28.0 (TID 577). 14477 bytes result sent to driver -26/04/01 08:39:21 INFO TaskSetManager: Finished task 30.0 in stage 28.0 (TID 577) in 1572 ms on 10.0.0.133 (executor driver) (31/34) -26/04/01 08:39:21 INFO Executor: Finished task 31.0 in stage 28.0 (TID 578). 14434 bytes result sent to driver -26/04/01 08:39:21 INFO TaskSetManager: Finished task 31.0 in stage 28.0 (TID 578) in 1575 ms on 10.0.0.133 (executor driver) (32/34) -26/04/01 08:39:21 INFO Executor: Finished task 33.0 in stage 28.0 (TID 580). 14477 bytes result sent to driver -26/04/01 08:39:21 INFO TaskSetManager: Finished task 33.0 in stage 28.0 (TID 580) in 474 ms on 10.0.0.133 (executor driver) (33/34) -26/04/01 08:39:21 INFO Executor: Finished task 32.0 in stage 28.0 (TID 579). 14477 bytes result sent to driver -26/04/01 08:39:21 INFO TaskSetManager: Finished task 32.0 in stage 28.0 (TID 579) in 1407 ms on 10.0.0.133 (executor driver) (34/34) -26/04/01 08:39:21 INFO TaskSchedulerImpl: Removed TaskSet 28.0, whose tasks have all completed, from pool -26/04/01 08:39:21 INFO DAGScheduler: ShuffleMapStage 28 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 7.116 s -26/04/01 08:39:21 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:39:21 INFO DAGScheduler: running: Set() -26/04/01 08:39:21 INFO DAGScheduler: waiting: Set() -26/04/01 08:39:21 INFO DAGScheduler: failed: Set() -26/04/01 08:39:21 INFO ShufflePartitionsUtil: For shuffle(7, 4), advisory target size: 67108864, actual target size 67108864, minimum partition size: 1048576 -26/04/01 08:39:21 INFO HashAggregateExec: spark.sql.codegen.aggregate.map.twolevel.enabled is set to true, but current version of codegened fast hashmap does not support this aggregate. -26/04/01 08:39:21 INFO CodeGenerator: Code generated in 11.426584 ms -26/04/01 08:39:21 INFO CodeGenerator: Code generated in 1.968209 ms -26/04/01 08:39:21 INFO CodeGenerator: Code generated in 1.643583 ms -26/04/01 08:39:21 INFO DAGScheduler: Registering RDD 67 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 8 -26/04/01 08:39:21 INFO DAGScheduler: Got map stage job 17 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 40 output partitions -26/04/01 08:39:21 INFO DAGScheduler: Final stage: ShuffleMapStage 37 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:39:21 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 35, ShuffleMapStage 36) -26/04/01 08:39:21 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:21 INFO DAGScheduler: Submitting ShuffleMapStage 37 (MapPartitionsRDD[67] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:39:21 INFO MemoryStore: Block broadcast_24 stored as values in memory (estimated size 110.1 KiB, free 8.6 GiB) -26/04/01 08:39:21 INFO MemoryStore: Block broadcast_24_piece0 stored as bytes in memory (estimated size 46.7 KiB, free 8.6 GiB) -26/04/01 08:39:21 INFO BlockManagerInfo: Added broadcast_24_piece0 in memory on 10.0.0.133:59288 (size: 46.7 KiB, free: 8.6 GiB) -26/04/01 08:39:21 INFO SparkContext: Created broadcast 24 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:21 INFO DAGScheduler: Submitting 40 missing tasks from ShuffleMapStage 37 (MapPartitionsRDD[67] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) -26/04/01 08:39:21 INFO TaskSchedulerImpl: Adding task set 37.0 with 40 tasks resource profile 0 -26/04/01 08:39:21 INFO TaskSetManager: Starting task 0.0 in stage 37.0 (TID 581) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:21 INFO TaskSetManager: Starting task 1.0 in stage 37.0 (TID 582) (10.0.0.133, executor driver, partition 1, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:21 INFO TaskSetManager: Starting task 2.0 in stage 37.0 (TID 583) (10.0.0.133, executor driver, partition 2, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:21 INFO TaskSetManager: Starting task 3.0 in stage 37.0 (TID 584) (10.0.0.133, executor driver, partition 3, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:21 INFO TaskSetManager: Starting task 4.0 in stage 37.0 (TID 585) (10.0.0.133, executor driver, partition 4, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:21 INFO TaskSetManager: Starting task 5.0 in stage 37.0 (TID 586) (10.0.0.133, executor driver, partition 5, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:21 INFO TaskSetManager: Starting task 6.0 in stage 37.0 (TID 587) (10.0.0.133, executor driver, partition 6, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:21 INFO TaskSetManager: Starting task 7.0 in stage 37.0 (TID 588) (10.0.0.133, executor driver, partition 7, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:21 INFO Executor: Running task 2.0 in stage 37.0 (TID 583) -26/04/01 08:39:21 INFO Executor: Running task 0.0 in stage 37.0 (TID 581) -26/04/01 08:39:21 INFO Executor: Running task 1.0 in stage 37.0 (TID 582) -26/04/01 08:39:21 INFO Executor: Running task 4.0 in stage 37.0 (TID 585) -26/04/01 08:39:21 INFO Executor: Running task 5.0 in stage 37.0 (TID 586) -26/04/01 08:39:21 INFO Executor: Running task 6.0 in stage 37.0 (TID 587) -26/04/01 08:39:21 INFO Executor: Running task 7.0 in stage 37.0 (TID 588) -26/04/01 08:39:21 INFO Executor: Running task 3.0 in stage 37.0 (TID 584) -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 1 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO CodeGenerator: Code generated in 2.273916 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.5 MiB) non-empty blocks including 64 (36.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:21 INFO CodeGenerator: Code generated in 2.775625 ms -26/04/01 08:39:21 INFO CodeGenerator: Code generated in 6.171875 ms -26/04/01 08:39:21 INFO CodeGenerator: Code generated in 1.881667 ms -26/04/01 08:39:21 INFO CodeGenerator: Code generated in 0.8755 ms -26/04/01 08:39:21 INFO CodeGenerator: Code generated in 1.532458 ms -26/04/01 08:39:21 INFO CodeGenerator: Code generated in 2.958333 ms -26/04/01 08:39:22 INFO BlockManagerInfo: Removed broadcast_23_piece0 on 10.0.0.133:59288 in memory (size: 33.7 KiB, free: 8.6 GiB) -26/04/01 08:39:22 INFO Executor: Finished task 4.0 in stage 37.0 (TID 585). 19388 bytes result sent to driver -26/04/01 08:39:22 INFO TaskSetManager: Starting task 8.0 in stage 37.0 (TID 589) (10.0.0.133, executor driver, partition 8, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:22 INFO Executor: Running task 8.0 in stage 37.0 (TID 589) -26/04/01 08:39:22 INFO TaskSetManager: Finished task 4.0 in stage 37.0 (TID 585) in 1051 ms on 10.0.0.133 (executor driver) (1/40) -26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:22 INFO Executor: Finished task 1.0 in stage 37.0 (TID 582). 19388 bytes result sent to driver -26/04/01 08:39:22 INFO TaskSetManager: Starting task 9.0 in stage 37.0 (TID 590) (10.0.0.133, executor driver, partition 9, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:22 INFO Executor: Finished task 6.0 in stage 37.0 (TID 587). 19388 bytes result sent to driver -26/04/01 08:39:22 INFO Executor: Running task 9.0 in stage 37.0 (TID 590) -26/04/01 08:39:22 INFO TaskSetManager: Starting task 10.0 in stage 37.0 (TID 591) (10.0.0.133, executor driver, partition 10, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:22 INFO TaskSetManager: Finished task 1.0 in stage 37.0 (TID 582) in 1137 ms on 10.0.0.133 (executor driver) (2/40) -26/04/01 08:39:22 INFO Executor: Running task 10.0 in stage 37.0 (TID 591) -26/04/01 08:39:22 INFO TaskSetManager: Finished task 6.0 in stage 37.0 (TID 587) in 1137 ms on 10.0.0.133 (executor driver) (3/40) -26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:22 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO Executor: Finished task 5.0 in stage 37.0 (TID 586). 19388 bytes result sent to driver -26/04/01 08:39:23 INFO TaskSetManager: Starting task 11.0 in stage 37.0 (TID 592) (10.0.0.133, executor driver, partition 11, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:23 INFO TaskSetManager: Finished task 5.0 in stage 37.0 (TID 586) in 1213 ms on 10.0.0.133 (executor driver) (4/40) -26/04/01 08:39:23 INFO Executor: Running task 11.0 in stage 37.0 (TID 592) -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO Executor: Finished task 0.0 in stage 37.0 (TID 581). 19388 bytes result sent to driver -26/04/01 08:39:23 INFO TaskSetManager: Starting task 12.0 in stage 37.0 (TID 593) (10.0.0.133, executor driver, partition 12, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:23 INFO Executor: Running task 12.0 in stage 37.0 (TID 593) -26/04/01 08:39:23 INFO TaskSetManager: Finished task 0.0 in stage 37.0 (TID 581) in 1314 ms on 10.0.0.133 (executor driver) (5/40) -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO Executor: Finished task 3.0 in stage 37.0 (TID 584). 19388 bytes result sent to driver -26/04/01 08:39:23 INFO TaskSetManager: Starting task 13.0 in stage 37.0 (TID 594) (10.0.0.133, executor driver, partition 13, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:23 INFO Executor: Running task 13.0 in stage 37.0 (TID 594) -26/04/01 08:39:23 INFO TaskSetManager: Finished task 3.0 in stage 37.0 (TID 584) in 1358 ms on 10.0.0.133 (executor driver) (6/40) -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO Executor: Finished task 2.0 in stage 37.0 (TID 583). 19388 bytes result sent to driver -26/04/01 08:39:23 INFO TaskSetManager: Starting task 14.0 in stage 37.0 (TID 595) (10.0.0.133, executor driver, partition 14, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:23 INFO TaskSetManager: Finished task 2.0 in stage 37.0 (TID 583) in 1412 ms on 10.0.0.133 (executor driver) (7/40) -26/04/01 08:39:23 INFO Executor: Running task 14.0 in stage 37.0 (TID 595) -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO Executor: Finished task 7.0 in stage 37.0 (TID 588). 19388 bytes result sent to driver -26/04/01 08:39:23 INFO TaskSetManager: Starting task 15.0 in stage 37.0 (TID 596) (10.0.0.133, executor driver, partition 15, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:23 INFO TaskSetManager: Finished task 7.0 in stage 37.0 (TID 588) in 1420 ms on 10.0.0.133 (executor driver) (8/40) -26/04/01 08:39:23 INFO Executor: Running task 15.0 in stage 37.0 (TID 596) -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO Executor: Finished task 8.0 in stage 37.0 (TID 589). 19388 bytes result sent to driver -26/04/01 08:39:23 INFO TaskSetManager: Starting task 16.0 in stage 37.0 (TID 597) (10.0.0.133, executor driver, partition 16, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:23 INFO Executor: Running task 16.0 in stage 37.0 (TID 597) -26/04/01 08:39:23 INFO TaskSetManager: Finished task 8.0 in stage 37.0 (TID 589) in 843 ms on 10.0.0.133 (executor driver) (9/40) -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO Executor: Finished task 9.0 in stage 37.0 (TID 590). 19388 bytes result sent to driver -26/04/01 08:39:23 INFO TaskSetManager: Starting task 17.0 in stage 37.0 (TID 598) (10.0.0.133, executor driver, partition 17, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:23 INFO Executor: Running task 17.0 in stage 37.0 (TID 598) -26/04/01 08:39:23 INFO TaskSetManager: Finished task 9.0 in stage 37.0 (TID 590) in 842 ms on 10.0.0.133 (executor driver) (10/40) -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO Executor: Finished task 10.0 in stage 37.0 (TID 591). 19388 bytes result sent to driver -26/04/01 08:39:23 INFO TaskSetManager: Starting task 18.0 in stage 37.0 (TID 599) (10.0.0.133, executor driver, partition 18, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:23 INFO Executor: Running task 18.0 in stage 37.0 (TID 599) -26/04/01 08:39:23 INFO TaskSetManager: Finished task 10.0 in stage 37.0 (TID 591) in 864 ms on 10.0.0.133 (executor driver) (11/40) -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO Executor: Finished task 11.0 in stage 37.0 (TID 592). 19388 bytes result sent to driver -26/04/01 08:39:23 INFO TaskSetManager: Starting task 19.0 in stage 37.0 (TID 600) (10.0.0.133, executor driver, partition 19, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:23 INFO Executor: Running task 19.0 in stage 37.0 (TID 600) -26/04/01 08:39:23 INFO TaskSetManager: Finished task 11.0 in stage 37.0 (TID 592) in 840 ms on 10.0.0.133 (executor driver) (12/40) -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:23 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO Executor: Finished task 12.0 in stage 37.0 (TID 593). 19388 bytes result sent to driver -26/04/01 08:39:24 INFO TaskSetManager: Starting task 20.0 in stage 37.0 (TID 601) (10.0.0.133, executor driver, partition 20, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:24 INFO Executor: Running task 20.0 in stage 37.0 (TID 601) -26/04/01 08:39:24 INFO TaskSetManager: Finished task 12.0 in stage 37.0 (TID 593) in 903 ms on 10.0.0.133 (executor driver) (13/40) -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO Executor: Finished task 13.0 in stage 37.0 (TID 594). 19388 bytes result sent to driver -26/04/01 08:39:24 INFO TaskSetManager: Starting task 21.0 in stage 37.0 (TID 602) (10.0.0.133, executor driver, partition 21, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:24 INFO TaskSetManager: Finished task 13.0 in stage 37.0 (TID 594) in 875 ms on 10.0.0.133 (executor driver) (14/40) -26/04/01 08:39:24 INFO Executor: Running task 21.0 in stage 37.0 (TID 602) -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO Executor: Finished task 14.0 in stage 37.0 (TID 595). 19388 bytes result sent to driver -26/04/01 08:39:24 INFO TaskSetManager: Starting task 22.0 in stage 37.0 (TID 603) (10.0.0.133, executor driver, partition 22, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:24 INFO Executor: Running task 22.0 in stage 37.0 (TID 603) -26/04/01 08:39:24 INFO TaskSetManager: Finished task 14.0 in stage 37.0 (TID 595) in 827 ms on 10.0.0.133 (executor driver) (15/40) -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO Executor: Finished task 15.0 in stage 37.0 (TID 596). 19388 bytes result sent to driver -26/04/01 08:39:24 INFO TaskSetManager: Starting task 23.0 in stage 37.0 (TID 604) (10.0.0.133, executor driver, partition 23, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:24 INFO Executor: Running task 23.0 in stage 37.0 (TID 604) -26/04/01 08:39:24 INFO TaskSetManager: Finished task 15.0 in stage 37.0 (TID 596) in 842 ms on 10.0.0.133 (executor driver) (16/40) -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO Executor: Finished task 16.0 in stage 37.0 (TID 597). 19388 bytes result sent to driver -26/04/01 08:39:24 INFO TaskSetManager: Starting task 24.0 in stage 37.0 (TID 605) (10.0.0.133, executor driver, partition 24, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:24 INFO TaskSetManager: Finished task 16.0 in stage 37.0 (TID 597) in 830 ms on 10.0.0.133 (executor driver) (17/40) -26/04/01 08:39:24 INFO Executor: Running task 24.0 in stage 37.0 (TID 605) -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO Executor: Finished task 17.0 in stage 37.0 (TID 598). 19388 bytes result sent to driver -26/04/01 08:39:24 INFO TaskSetManager: Starting task 25.0 in stage 37.0 (TID 606) (10.0.0.133, executor driver, partition 25, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:24 INFO Executor: Running task 25.0 in stage 37.0 (TID 606) -26/04/01 08:39:24 INFO TaskSetManager: Finished task 17.0 in stage 37.0 (TID 598) in 802 ms on 10.0.0.133 (executor driver) (18/40) -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO Executor: Finished task 18.0 in stage 37.0 (TID 599). 19388 bytes result sent to driver -26/04/01 08:39:24 INFO TaskSetManager: Starting task 26.0 in stage 37.0 (TID 607) (10.0.0.133, executor driver, partition 26, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:24 INFO Executor: Running task 26.0 in stage 37.0 (TID 607) -26/04/01 08:39:24 INFO TaskSetManager: Finished task 18.0 in stage 37.0 (TID 599) in 808 ms on 10.0.0.133 (executor driver) (19/40) -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO Executor: Finished task 19.0 in stage 37.0 (TID 600). 19388 bytes result sent to driver -26/04/01 08:39:24 INFO TaskSetManager: Starting task 27.0 in stage 37.0 (TID 608) (10.0.0.133, executor driver, partition 27, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:24 INFO Executor: Running task 27.0 in stage 37.0 (TID 608) -26/04/01 08:39:24 INFO TaskSetManager: Finished task 19.0 in stage 37.0 (TID 600) in 802 ms on 10.0.0.133 (executor driver) (20/40) -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO Executor: Finished task 20.0 in stage 37.0 (TID 601). 19388 bytes result sent to driver -26/04/01 08:39:24 INFO TaskSetManager: Starting task 28.0 in stage 37.0 (TID 609) (10.0.0.133, executor driver, partition 28, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:24 INFO TaskSetManager: Finished task 20.0 in stage 37.0 (TID 601) in 825 ms on 10.0.0.133 (executor driver) (21/40) -26/04/01 08:39:24 INFO Executor: Running task 28.0 in stage 37.0 (TID 609) -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO Executor: Finished task 21.0 in stage 37.0 (TID 602). 19388 bytes result sent to driver -26/04/01 08:39:24 INFO TaskSetManager: Starting task 29.0 in stage 37.0 (TID 610) (10.0.0.133, executor driver, partition 29, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:24 INFO Executor: Running task 29.0 in stage 37.0 (TID 610) -26/04/01 08:39:24 INFO TaskSetManager: Finished task 21.0 in stage 37.0 (TID 602) in 820 ms on 10.0.0.133 (executor driver) (22/40) -26/04/01 08:39:24 INFO Executor: Finished task 22.0 in stage 37.0 (TID 603). 19388 bytes result sent to driver -26/04/01 08:39:24 INFO TaskSetManager: Starting task 30.0 in stage 37.0 (TID 611) (10.0.0.133, executor driver, partition 30, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:24 INFO Executor: Running task 30.0 in stage 37.0 (TID 611) -26/04/01 08:39:24 INFO TaskSetManager: Finished task 22.0 in stage 37.0 (TID 603) in 815 ms on 10.0.0.133 (executor driver) (23/40) -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO Executor: Finished task 23.0 in stage 37.0 (TID 604). 19388 bytes result sent to driver -26/04/01 08:39:24 INFO TaskSetManager: Starting task 31.0 in stage 37.0 (TID 612) (10.0.0.133, executor driver, partition 31, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:24 INFO TaskSetManager: Finished task 23.0 in stage 37.0 (TID 604) in 816 ms on 10.0.0.133 (executor driver) (24/40) -26/04/01 08:39:24 INFO Executor: Running task 31.0 in stage 37.0 (TID 612) -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:24 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO Executor: Finished task 24.0 in stage 37.0 (TID 605). 19388 bytes result sent to driver -26/04/01 08:39:25 INFO TaskSetManager: Starting task 32.0 in stage 37.0 (TID 613) (10.0.0.133, executor driver, partition 32, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:25 INFO TaskSetManager: Finished task 24.0 in stage 37.0 (TID 605) in 807 ms on 10.0.0.133 (executor driver) (25/40) -26/04/01 08:39:25 INFO Executor: Running task 32.0 in stage 37.0 (TID 613) -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO Executor: Finished task 25.0 in stage 37.0 (TID 606). 19388 bytes result sent to driver -26/04/01 08:39:25 INFO TaskSetManager: Starting task 33.0 in stage 37.0 (TID 614) (10.0.0.133, executor driver, partition 33, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:25 INFO Executor: Running task 33.0 in stage 37.0 (TID 614) -26/04/01 08:39:25 INFO TaskSetManager: Finished task 25.0 in stage 37.0 (TID 606) in 815 ms on 10.0.0.133 (executor driver) (26/40) -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO Executor: Finished task 26.0 in stage 37.0 (TID 607). 19388 bytes result sent to driver -26/04/01 08:39:25 INFO TaskSetManager: Starting task 34.0 in stage 37.0 (TID 615) (10.0.0.133, executor driver, partition 34, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:25 INFO Executor: Running task 34.0 in stage 37.0 (TID 615) -26/04/01 08:39:25 INFO TaskSetManager: Finished task 26.0 in stage 37.0 (TID 607) in 801 ms on 10.0.0.133 (executor driver) (27/40) -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.5 MiB) non-empty blocks including 64 (36.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO Executor: Finished task 27.0 in stage 37.0 (TID 608). 19388 bytes result sent to driver -26/04/01 08:39:25 INFO TaskSetManager: Starting task 35.0 in stage 37.0 (TID 616) (10.0.0.133, executor driver, partition 35, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:25 INFO Executor: Running task 35.0 in stage 37.0 (TID 616) -26/04/01 08:39:25 INFO TaskSetManager: Finished task 27.0 in stage 37.0 (TID 608) in 819 ms on 10.0.0.133 (executor driver) (28/40) -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.4 MiB) non-empty blocks including 64 (36.4 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO Executor: Finished task 29.0 in stage 37.0 (TID 610). 19388 bytes result sent to driver -26/04/01 08:39:25 INFO TaskSetManager: Starting task 36.0 in stage 37.0 (TID 617) (10.0.0.133, executor driver, partition 36, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:25 INFO TaskSetManager: Finished task 29.0 in stage 37.0 (TID 610) in 818 ms on 10.0.0.133 (executor driver) (29/40) -26/04/01 08:39:25 INFO Executor: Running task 36.0 in stage 37.0 (TID 617) -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO Executor: Finished task 28.0 in stage 37.0 (TID 609). 19388 bytes result sent to driver -26/04/01 08:39:25 INFO TaskSetManager: Starting task 37.0 in stage 37.0 (TID 618) (10.0.0.133, executor driver, partition 37, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:25 INFO Executor: Running task 37.0 in stage 37.0 (TID 618) -26/04/01 08:39:25 INFO TaskSetManager: Finished task 28.0 in stage 37.0 (TID 609) in 832 ms on 10.0.0.133 (executor driver) (30/40) -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO Executor: Finished task 30.0 in stage 37.0 (TID 611). 19388 bytes result sent to driver -26/04/01 08:39:25 INFO TaskSetManager: Starting task 38.0 in stage 37.0 (TID 619) (10.0.0.133, executor driver, partition 38, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:25 INFO Executor: Running task 38.0 in stage 37.0 (TID 619) -26/04/01 08:39:25 INFO TaskSetManager: Finished task 30.0 in stage 37.0 (TID 611) in 823 ms on 10.0.0.133 (executor driver) (31/40) -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.5 MiB) non-empty blocks including 64 (36.5 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO Executor: Finished task 31.0 in stage 37.0 (TID 612). 19388 bytes result sent to driver -26/04/01 08:39:25 INFO TaskSetManager: Starting task 39.0 in stage 37.0 (TID 620) (10.0.0.133, executor driver, partition 39, NODE_LOCAL, 9270 bytes) -26/04/01 08:39:25 INFO Executor: Running task 39.0 in stage 37.0 (TID 620) -26/04/01 08:39:25 INFO TaskSetManager: Finished task 31.0 in stage 37.0 (TID 612) in 803 ms on 10.0.0.133 (executor driver) (32/40) -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 34 (23.7 MiB) non-empty blocks including 34 (23.7 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Getting 64 (36.3 MiB) non-empty blocks including 64 (36.3 MiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:25 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:26 INFO Executor: Finished task 32.0 in stage 37.0 (TID 613). 19388 bytes result sent to driver -26/04/01 08:39:26 INFO TaskSetManager: Finished task 32.0 in stage 37.0 (TID 613) in 788 ms on 10.0.0.133 (executor driver) (33/40) -26/04/01 08:39:26 INFO Executor: Finished task 34.0 in stage 37.0 (TID 615). 19388 bytes result sent to driver -26/04/01 08:39:26 INFO TaskSetManager: Finished task 34.0 in stage 37.0 (TID 615) in 788 ms on 10.0.0.133 (executor driver) (34/40) -26/04/01 08:39:26 INFO Executor: Finished task 33.0 in stage 37.0 (TID 614). 19388 bytes result sent to driver -26/04/01 08:39:26 INFO TaskSetManager: Finished task 33.0 in stage 37.0 (TID 614) in 833 ms on 10.0.0.133 (executor driver) (35/40) -26/04/01 08:39:26 INFO Executor: Finished task 35.0 in stage 37.0 (TID 616). 19388 bytes result sent to driver -26/04/01 08:39:26 INFO TaskSetManager: Finished task 35.0 in stage 37.0 (TID 616) in 828 ms on 10.0.0.133 (executor driver) (36/40) -26/04/01 08:39:26 INFO Executor: Finished task 37.0 in stage 37.0 (TID 618). 19388 bytes result sent to driver -26/04/01 08:39:26 INFO TaskSetManager: Finished task 37.0 in stage 37.0 (TID 618) in 799 ms on 10.0.0.133 (executor driver) (37/40) -26/04/01 08:39:26 INFO Executor: Finished task 36.0 in stage 37.0 (TID 617). 19388 bytes result sent to driver -26/04/01 08:39:26 INFO TaskSetManager: Finished task 36.0 in stage 37.0 (TID 617) in 809 ms on 10.0.0.133 (executor driver) (38/40) -26/04/01 08:39:26 INFO Executor: Finished task 39.0 in stage 37.0 (TID 620). 19388 bytes result sent to driver -26/04/01 08:39:26 INFO TaskSetManager: Finished task 39.0 in stage 37.0 (TID 620) in 805 ms on 10.0.0.133 (executor driver) (39/40) -26/04/01 08:39:26 INFO Executor: Finished task 38.0 in stage 37.0 (TID 619). 19388 bytes result sent to driver -26/04/01 08:39:26 INFO TaskSetManager: Finished task 38.0 in stage 37.0 (TID 619) in 813 ms on 10.0.0.133 (executor driver) (40/40) -26/04/01 08:39:26 INFO TaskSchedulerImpl: Removed TaskSet 37.0, whose tasks have all completed, from pool -26/04/01 08:39:26 INFO DAGScheduler: ShuffleMapStage 37 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 4.693 s -26/04/01 08:39:26 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:39:26 INFO DAGScheduler: running: Set() -26/04/01 08:39:26 INFO DAGScheduler: waiting: Set() -26/04/01 08:39:26 INFO DAGScheduler: failed: Set() -26/04/01 08:39:26 INFO ShufflePartitionsUtil: For shuffle(8), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 08:39:26 INFO HashAggregateExec: spark.sql.codegen.aggregate.map.twolevel.enabled is set to true, but current version of codegened fast hashmap does not support this aggregate. -26/04/01 08:39:26 INFO CodeGenerator: Code generated in 4.282291 ms -26/04/01 08:39:26 INFO CodeGenerator: Code generated in 1.394541 ms -26/04/01 08:39:26 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:39:26 INFO DAGScheduler: Got job 18 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 08:39:26 INFO DAGScheduler: Final stage: ResultStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:39:26 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 46) -26/04/01 08:39:26 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:26 INFO DAGScheduler: Submitting ResultStage 47 (MapPartitionsRDD[72] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:39:26 INFO MemoryStore: Block broadcast_25 stored as values in memory (estimated size 107.4 KiB, free 8.6 GiB) -26/04/01 08:39:26 INFO MemoryStore: Block broadcast_25_piece0 stored as bytes in memory (estimated size 44.1 KiB, free 8.6 GiB) -26/04/01 08:39:26 INFO BlockManagerInfo: Added broadcast_25_piece0 in memory on 10.0.0.133:59288 (size: 44.1 KiB, free: 8.6 GiB) -26/04/01 08:39:26 INFO SparkContext: Created broadcast 25 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:26 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 47 (MapPartitionsRDD[72] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:39:26 INFO TaskSchedulerImpl: Adding task set 47.0 with 1 tasks resource profile 0 -26/04/01 08:39:26 INFO TaskSetManager: Starting task 0.0 in stage 47.0 (TID 621) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8999 bytes) -26/04/01 08:39:26 INFO Executor: Running task 0.0 in stage 47.0 (TID 621) -26/04/01 08:39:26 INFO ShuffleBlockFetcherIterator: Getting 40 (595.2 KiB) non-empty blocks including 40 (595.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:26 INFO CodeGenerator: Code generated in 3.784167 ms -26/04/01 08:39:26 INFO Executor: Finished task 0.0 in stage 47.0 (TID 621). 29688 bytes result sent to driver -26/04/01 08:39:26 INFO TaskSetManager: Finished task 0.0 in stage 47.0 (TID 621) in 28 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:39:26 INFO TaskSchedulerImpl: Removed TaskSet 47.0, whose tasks have all completed, from pool -26/04/01 08:39:26 INFO DAGScheduler: ResultStage 47 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.031 s -26/04/01 08:39:26 INFO DAGScheduler: Job 18 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:39:26 INFO TaskSchedulerImpl: Killing all running tasks in stage 47: Stage finished -26/04/01 08:39:26 INFO DAGScheduler: Job 18 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.033149 s -26/04/01 08:39:26 INFO DAGScheduler: Registering RDD 73 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) as input to shuffle 9 -26/04/01 08:39:26 INFO DAGScheduler: Got map stage job 19 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 08:39:26 INFO DAGScheduler: Final stage: ShuffleMapStage 57 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:39:26 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 56) -26/04/01 08:39:26 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:26 INFO DAGScheduler: Submitting ShuffleMapStage 57 (MapPartitionsRDD[73] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:39:26 INFO MemoryStore: Block broadcast_26 stored as values in memory (estimated size 116.3 KiB, free 8.6 GiB) -26/04/01 08:39:26 INFO MemoryStore: Block broadcast_26_piece0 stored as bytes in memory (estimated size 45.9 KiB, free 8.6 GiB) -26/04/01 08:39:26 INFO BlockManagerInfo: Added broadcast_26_piece0 in memory on 10.0.0.133:59288 (size: 45.9 KiB, free: 8.6 GiB) -26/04/01 08:39:26 INFO SparkContext: Created broadcast 26 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:26 INFO DAGScheduler: Submitting 1 missing tasks from ShuffleMapStage 57 (MapPartitionsRDD[73] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:39:26 INFO TaskSchedulerImpl: Adding task set 57.0 with 1 tasks resource profile 0 -26/04/01 08:39:26 INFO TaskSetManager: Starting task 0.0 in stage 57.0 (TID 622) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8988 bytes) -26/04/01 08:39:26 INFO Executor: Running task 0.0 in stage 57.0 (TID 622) -26/04/01 08:39:26 INFO CodeGenerator: Code generated in 1.754625 ms -26/04/01 08:39:26 INFO ShuffleBlockFetcherIterator: Getting 40 (595.2 KiB) non-empty blocks including 40 (595.2 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:26 INFO Executor: Finished task 0.0 in stage 57.0 (TID 622). 20944 bytes result sent to driver -26/04/01 08:39:26 INFO TaskSetManager: Finished task 0.0 in stage 57.0 (TID 622) in 52 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:39:26 INFO TaskSchedulerImpl: Removed TaskSet 57.0, whose tasks have all completed, from pool -26/04/01 08:39:26 INFO DAGScheduler: ShuffleMapStage 57 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.057 s -26/04/01 08:39:26 INFO DAGScheduler: looking for newly runnable stages -26/04/01 08:39:26 INFO DAGScheduler: running: Set() -26/04/01 08:39:26 INFO DAGScheduler: waiting: Set() -26/04/01 08:39:26 INFO DAGScheduler: failed: Set() -26/04/01 08:39:26 INFO ShufflePartitionsUtil: For shuffle(9), advisory target size: 67108864, actual target size 1048576, minimum partition size: 1048576 -26/04/01 08:39:26 INFO CodeGenerator: Code generated in 2.3705 ms -26/04/01 08:39:26 INFO SparkContext: Starting job: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174 -26/04/01 08:39:26 INFO DAGScheduler: Got job 20 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) with 1 output partitions -26/04/01 08:39:26 INFO DAGScheduler: Final stage: ResultStage 68 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) -26/04/01 08:39:26 INFO DAGScheduler: Parents of final stage: List(ShuffleMapStage 67) -26/04/01 08:39:26 INFO DAGScheduler: Missing parents: List() -26/04/01 08:39:26 INFO DAGScheduler: Submitting ResultStage 68 (MapPartitionsRDD[76] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174), which has no missing parents -26/04/01 08:39:26 INFO MemoryStore: Block broadcast_27 stored as values in memory (estimated size 102.4 KiB, free 8.6 GiB) -26/04/01 08:39:26 INFO MemoryStore: Block broadcast_27_piece0 stored as bytes in memory (estimated size 42.7 KiB, free 8.6 GiB) -26/04/01 08:39:26 INFO BlockManagerInfo: Added broadcast_27_piece0 in memory on 10.0.0.133:59288 (size: 42.7 KiB, free: 8.6 GiB) -26/04/01 08:39:26 INFO SparkContext: Created broadcast 27 from broadcast at DAGScheduler.scala:1611 -26/04/01 08:39:26 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 68 (MapPartitionsRDD[76] at collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) (first 15 tasks are for partitions Vector(0)) -26/04/01 08:39:26 INFO TaskSchedulerImpl: Adding task set 68.0 with 1 tasks resource profile 0 -26/04/01 08:39:26 INFO TaskSetManager: Starting task 0.0 in stage 68.0 (TID 623) (10.0.0.133, executor driver, partition 0, NODE_LOCAL, 8999 bytes) -26/04/01 08:39:26 INFO Executor: Running task 0.0 in stage 68.0 (TID 623) -26/04/01 08:39:26 INFO ShuffleBlockFetcherIterator: Getting 1 (17.3 KiB) non-empty blocks including 1 (17.3 KiB) local and 0 (0.0 B) host-local and 0 (0.0 B) push-merged-local and 0 (0.0 B) remote blocks -26/04/01 08:39:26 INFO ShuffleBlockFetcherIterator: Started 0 remote fetches in 0 ms -26/04/01 08:39:26 INFO CodeGenerator: Code generated in 1.805 ms -26/04/01 08:39:26 INFO CodeGenerator: Code generated in 2.202917 ms -26/04/01 08:39:26 INFO Executor: Finished task 0.0 in stage 68.0 (TID 623). 25767 bytes result sent to driver -26/04/01 08:39:26 INFO TaskSetManager: Finished task 0.0 in stage 68.0 (TID 623) in 9 ms on 10.0.0.133 (executor driver) (1/1) -26/04/01 08:39:26 INFO TaskSchedulerImpl: Removed TaskSet 68.0, whose tasks have all completed, from pool -26/04/01 08:39:26 INFO DAGScheduler: ResultStage 68 (collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174) finished in 0.012 s -26/04/01 08:39:26 INFO DAGScheduler: Job 20 is finished. Cancelling potential speculative or zombie tasks for this job -26/04/01 08:39:26 INFO TaskSchedulerImpl: Killing all running tasks in stage 68: Stage finished -26/04/01 08:39:26 INFO DAGScheduler: Job 20 finished: collect at /Users/andy/git/apache/datafusion-comet/benchmarks/tpc/tpcbench.py:174, took 0.013881 s -26/04/01 08:39:26 INFO SparkContext: SparkContext is stopping with exitCode 0. -26/04/01 08:39:26 INFO MapOutputTrackerMasterEndpoint: MapOutputTrackerMasterEndpoint stopped! -26/04/01 08:39:29 INFO MemoryStore: MemoryStore cleared -26/04/01 08:39:29 INFO BlockManager: BlockManager stopped -26/04/01 08:39:29 INFO BlockManagerMaster: BlockManagerMaster stopped -26/04/01 08:39:29 INFO OutputCommitCoordinator$OutputCommitCoordinatorEndpoint: OutputCommitCoordinator stopped! -26/04/01 08:39:29 INFO SparkContext: Successfully stopped SparkContext -26/04/01 08:39:29 INFO ShutdownHookManager: Shutdown hook called -26/04/01 08:39:29 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-d176a20a-d434-4356-bca6-cae43ef66498 -26/04/01 08:39:29 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-d176a20a-d434-4356-bca6-cae43ef66498/pyspark-15bb5836-3e4b-4658-b524-ea35cdb6f558 -26/04/01 08:39:29 INFO ShutdownHookManager: Deleting directory /private/var/folders/14/l9q3x6g90dd91py352flf6s00000gn/T/spark-0b85ad85-88e5-4003-afbe-ce40c48d1d47 - 86.21 real 432.85 user 104.97 sys - 8887762944 maximum resident set size - 0 average shared memory size - 0 average unshared data size - 0 average unshared stack size - 3508536 page reclaims - 48 page faults - 0 swaps - 0 block input operations - 0 block output operations - 1016 messages sent - 1034 messages received - 251 signals received - 114132 voluntary context switches - 1088013 involuntary context switches - 5685978480737 instructions retired - 1928666289311 cycles elapsed - 8867438272 peak memory footprint diff --git a/benchmarks/tpc/memory-profile-results/results.csv b/benchmarks/tpc/memory-profile-results/results.csv deleted file mode 100644 index e0bc63b1e0..0000000000 --- a/benchmarks/tpc/memory-profile-results/results.csv +++ /dev/null @@ -1,4 +0,0 @@ -engine,offheap_size,query,peak_rss_mb,wall_time_sec,exit_code -spark,4g,9,8476,86.21,0 -comet,4g,9,8525,384.09,0 -comet,8g,9,8448,384.01,0 diff --git a/docs/superpowers/plans/2026-04-01-spill-callback.md b/docs/superpowers/plans/2026-04-01-spill-callback.md deleted file mode 100644 index f0770a5229..0000000000 --- a/docs/superpowers/plans/2026-04-01-spill-callback.md +++ /dev/null @@ -1,782 +0,0 @@ -# Spill Callback 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:** Enable Spark to reclaim memory from Comet's native operators via the `spill()` callback, so cross-task memory eviction works and Comet can run at lower off-heap memory settings. - -**Architecture:** Add a shared `SpillState` (atomics for pressure/freed) that bridges `NativeMemoryConsumer.spill()` on the JVM side with `CometUnifiedMemoryPool.try_grow()` on the Rust side. When Spark calls `spill()`, it JNI-calls into native to set spill pressure. The pool's `try_grow()` checks this flag and returns `ResourcesExhausted`, causing DataFusion's Sort/Aggregate/Shuffle operators to spill internally. As operators release memory via `shrink()`, the freed bytes are tracked and returned to Spark. - -**Tech Stack:** Rust (native memory pool), Java (JNI bridge, Spark MemoryConsumer), JNI - ---- - -## File Map - -| File | Action | Responsibility | -| ------------------------------------------------------------------ | --------- | ------------------------------------------------------------------------- | -| `native/core/src/execution/memory_pools/spill.rs` | Create | `SpillState` struct with atomics and wait/notify | -| `native/core/src/execution/memory_pools/unified_pool.rs` | Modify | Check `SpillState` in `try_grow`, track freed in `shrink` | -| `native/core/src/execution/memory_pools/mod.rs` | Modify | Export `spill` module, pass `SpillState` through `create_memory_pool` | -| `native/core/src/execution/memory_pools/config.rs` | No change | Pool config unchanged | -| `native/core/src/execution/jni_api.rs` | Modify | Store `SpillState` in `ExecutionContext`, add JNI function `requestSpill` | -| `spark/src/main/java/org/apache/spark/CometTaskMemoryManager.java` | Modify | Implement `spill()`, store native handle, add JNI call | -| `spark/src/main/scala/org/apache/comet/Native.scala` | Modify | Add `requestSpill` native method declaration | -| `spark/src/main/scala/org/apache/comet/CometExecIterator.scala` | Modify | Pass native plan handle back to `CometTaskMemoryManager` | - -## Scope - -This plan covers the `GreedyUnified` pool only (the default off-heap pool). The `FairUnified` pool (`fair_pool.rs`) uses the same Spark JNI path and can be updated in a follow-up using the same pattern. - ---- - -### Task 1: Create SpillState - -**Files:** - -- Create: `native/core/src/execution/memory_pools/spill.rs` - -- [ ] **Step 1: Create the SpillState struct** - -```rust -// native/core/src/execution/memory_pools/spill.rs - -// 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 std::sync::atomic::{AtomicUsize, Ordering}; -use std::sync::{Condvar, Mutex}; -use std::time::Duration; - -/// Shared state for coordinating spill requests between Spark's memory manager -/// (which calls `NativeMemoryConsumer.spill()` on a Spark thread) and DataFusion -/// operators (which call `try_grow()`/`shrink()` on tokio threads). -/// -/// When Spark needs to reclaim memory from Comet, it sets `pressure` via -/// `request_spill()`. The memory pool's `try_grow()` checks this and returns -/// `ResourcesExhausted`, causing operators to spill. As operators call `shrink()`, -/// freed bytes are accumulated and the waiting Spark thread is notified. -#[derive(Debug)] -pub struct SpillState { - /// Bytes requested to be freed. Set by Spark's spill() callback. - pressure: AtomicUsize, - /// Bytes actually freed since pressure was set. - freed: AtomicUsize, - /// Mutex + Condvar to allow the spill requester to wait for operators to react. - notify: (Mutex<()>, Condvar), -} - -impl SpillState { - pub fn new() -> Self { - Self { - pressure: AtomicUsize::new(0), - freed: AtomicUsize::new(0), - notify: (Mutex::new(()), Condvar::new()), - } - } - - /// Returns the current spill pressure in bytes. Called by the memory pool's - /// `try_grow()` to decide whether to deny allocations. - pub fn pressure(&self) -> usize { - self.pressure.load(Ordering::Acquire) - } - - /// Record that `size` bytes were freed (called from pool's `shrink()`). - /// Wakes the waiting spill requester. - pub fn record_freed(&self, size: usize) { - self.freed.fetch_add(size, Ordering::Release); - let (_lock, cvar) = &self.notify; - cvar.notify_all(); - } - - /// Called from JNI when Spark's `NativeMemoryConsumer.spill()` is invoked. - /// Sets spill pressure and waits (up to `timeout`) for operators to free memory. - /// Returns the actual number of bytes freed. - pub fn request_spill(&self, size: usize, timeout: Duration) -> usize { - // Reset freed counter and set pressure - self.freed.store(0, Ordering::Release); - self.pressure.store(size, Ordering::Release); - - // Wait for operators to react - let (lock, cvar) = &self.notify; - let mut guard = lock.lock().unwrap(); - let deadline = std::time::Instant::now() + timeout; - while self.freed.load(Ordering::Acquire) < size { - let remaining = deadline.saturating_duration_since(std::time::Instant::now()); - if remaining.is_zero() { - break; - } - let (new_guard, _timeout_result) = cvar.wait_timeout(guard, remaining).unwrap(); - guard = new_guard; - } - - // Clear pressure and return freed bytes - self.pressure.store(0, Ordering::Release); - self.freed.load(Ordering::Acquire) - } -} - -impl Default for SpillState { - fn default() -> Self { - Self::new() - } -} -``` - -- [ ] **Step 2: Add module to mod.rs** - -Add to the top of `native/core/src/execution/memory_pools/mod.rs`, after the existing module declarations: - -```rust -pub(crate) mod spill; -``` - -- [ ] **Step 3: Build to verify compilation** - -Run: `cargo build --manifest-path native/Cargo.toml` -Expected: Build succeeds with no errors. - -- [ ] **Step 4: Commit** - -```bash -git add native/core/src/execution/memory_pools/spill.rs native/core/src/execution/memory_pools/mod.rs -git commit -m "feat: add SpillState for cross-thread spill coordination" -``` - ---- - -### Task 2: Wire SpillState into CometUnifiedMemoryPool - -**Files:** - -- Modify: `native/core/src/execution/memory_pools/unified_pool.rs` -- Modify: `native/core/src/execution/memory_pools/mod.rs` - -- [ ] **Step 1: Add SpillState to CometUnifiedMemoryPool** - -In `unified_pool.rs`, add the `spill_state` field and update the constructor: - -```rust -use super::spill::SpillState; -use std::sync::Arc; -``` - -Change the struct definition: - -```rust -pub struct CometUnifiedMemoryPool { - task_memory_manager_handle: Arc, - used: AtomicUsize, - task_attempt_id: i64, - spill_state: Arc, -} -``` - -Update `Debug` impl to include the new field: - -```rust -impl Debug for CometUnifiedMemoryPool { - fn fmt(&self, f: &mut Formatter<'_>) -> FmtResult { - f.debug_struct("CometUnifiedMemoryPool") - .field("used", &self.used.load(Relaxed)) - .field("spill_pressure", &self.spill_state.pressure()) - .finish() - } -} -``` - -Update the constructor: - -```rust -pub fn new( - task_memory_manager_handle: Arc, - task_attempt_id: i64, - spill_state: Arc, -) -> CometUnifiedMemoryPool { - Self { - task_memory_manager_handle, - task_attempt_id, - used: AtomicUsize::new(0), - spill_state, - } -} -``` - -- [ ] **Step 2: Check spill pressure in try_grow** - -Replace the `try_grow` implementation: - -```rust -fn try_grow(&self, _: &MemoryReservation, additional: usize) -> Result<(), DataFusionError> { - if additional > 0 { - // If there is spill pressure, deny the allocation to trigger operator spill - if self.spill_state.pressure() > 0 { - return Err(resources_datafusion_err!( - "Task {} denied {} bytes due to spill pressure. Reserved: {}", - self.task_attempt_id, - additional, - self.reserved() - )); - } - - let acquired = self.acquire_from_spark(additional)?; - // If the number of bytes we acquired is less than the requested, return an error, - // and hopefully will trigger spilling from the caller side. - if acquired < additional as i64 { - // Release the acquired bytes before throwing error - self.release_to_spark(acquired as usize)?; - - return Err(resources_datafusion_err!( - "Task {} failed to acquire {} bytes, only got {}. Reserved: {}", - self.task_attempt_id, - additional, - acquired, - self.reserved() - )); - } - if let Err(prev) = self - .used - .fetch_update(Relaxed, Relaxed, |old| old.checked_add(acquired as usize)) - { - return Err(resources_datafusion_err!( - "Task {} failed to acquire {} bytes due to overflow. Reserved: {}", - self.task_attempt_id, - additional, - prev - )); - } - } - Ok(()) -} -``` - -- [ ] **Step 3: Record freed bytes in shrink** - -Replace the `shrink` implementation: - -```rust -fn shrink(&self, _: &MemoryReservation, size: usize) { - if let Err(e) = self.release_to_spark(size) { - panic!( - "Task {} failed to return {size} bytes to Spark: {e:?}", - self.task_attempt_id - ); - } - if let Err(prev) = self - .used - .fetch_update(Relaxed, Relaxed, |old| old.checked_sub(size)) - { - panic!( - "Task {} overflow when releasing {size} of {prev} bytes", - self.task_attempt_id - ); - } - // Notify the spill requester that memory was freed - if self.spill_state.pressure() > 0 { - self.spill_state.record_freed(size); - } -} -``` - -- [ ] **Step 4: Update create_memory_pool to accept and pass SpillState** - -In `native/core/src/execution/memory_pools/mod.rs`, update the function signature and the `GreedyUnified` arm: - -```rust -use super::memory_pools::spill::SpillState; -``` - -Change the signature of `create_memory_pool`: - -```rust -pub(crate) fn create_memory_pool( - memory_pool_config: &MemoryPoolConfig, - comet_task_memory_manager: Arc, - task_attempt_id: i64, - spill_state: Option>, -) -> Arc { -``` - -Update the `GreedyUnified` match arm: - -```rust -MemoryPoolType::GreedyUnified => { - let spill_state = spill_state - .unwrap_or_else(|| Arc::new(SpillState::new())); - let memory_pool = CometUnifiedMemoryPool::new( - comet_task_memory_manager, - task_attempt_id, - spill_state, - ); - Arc::new(TrackConsumersPool::new( - memory_pool, - NonZeroUsize::new(NUM_TRACKED_CONSUMERS).unwrap(), - )) -} -``` - -- [ ] **Step 5: Update call site in jni_api.rs** - -In `jni_api.rs`, update the call to `create_memory_pool` (around line 260): - -```rust -let memory_pool = - create_memory_pool(&memory_pool_config, task_memory_manager, task_attempt_id, None); -``` - -We pass `None` for now — Task 3 will wire in the real `SpillState`. - -- [ ] **Step 6: Build to verify compilation** - -Run: `cargo build --manifest-path native/Cargo.toml` -Expected: Build succeeds. - -- [ ] **Step 7: Commit** - -```bash -git add native/core/src/execution/memory_pools/unified_pool.rs \ - native/core/src/execution/memory_pools/mod.rs \ - native/core/src/execution/jni_api.rs -git commit -m "feat: wire SpillState into CometUnifiedMemoryPool" -``` - ---- - -### Task 3: Add JNI requestSpill Function and Store SpillState in ExecutionContext - -**Files:** - -- Modify: `native/core/src/execution/jni_api.rs` - -- [ ] **Step 1: Add SpillState to ExecutionContext and pass through create_memory_pool** - -Add field to `ExecutionContext` struct (around line 179): - -```rust -pub spill_state: Option>, -``` - -Add the import near the top of `jni_api.rs`: - -```rust -use crate::execution::memory_pools::spill::SpillState; -``` - -In the `createPlan` JNI function, create the `SpillState` before pool creation (around line 259): - -```rust -let spill_state = if off_heap_mode != JNI_FALSE - && matches!(memory_pool_config.pool_type, MemoryPoolType::GreedyUnified) -{ - Some(Arc::new(SpillState::new())) -} else { - None -}; - -let memory_pool = create_memory_pool( - &memory_pool_config, - task_memory_manager, - task_attempt_id, - spill_state.clone(), -); -``` - -Add the field to the `ExecutionContext` initialization (around line 328): - -```rust -spill_state, -``` - -- [ ] **Step 2: Add the JNI requestSpill function** - -Add this function at module level in `jni_api.rs` (after the existing JNI functions): - -```rust -/// Called from `CometTaskMemoryManager.spill()` via JNI to request that native -/// operators free memory. Returns the number of bytes actually freed. -/// # 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_requestSpill( - e: JNIEnv, - _class: JClass, - exec_context: jlong, - size: jlong, -) -> jlong { - try_unwrap_or_throw(&e, |_env| { - let exec_context = get_execution_context(exec_context); - if let Some(ref spill_state) = exec_context.spill_state { - let timeout = std::time::Duration::from_secs(10); - let freed = spill_state.request_spill(size as usize, timeout); - Ok(freed as jlong) - } else { - // No spill state (not using unified pool) — can't spill - Ok(0i64) - } - }) -} -``` - -- [ ] **Step 3: Build to verify compilation** - -Run: `cargo build --manifest-path native/Cargo.toml` -Expected: Build succeeds. - -- [ ] **Step 4: Commit** - -```bash -git add native/core/src/execution/jni_api.rs -git commit -m "feat: add JNI requestSpill function with SpillState in ExecutionContext" -``` - ---- - -### Task 4: Update JVM Side — CometTaskMemoryManager and Native - -**Files:** - -- Modify: `spark/src/main/java/org/apache/spark/CometTaskMemoryManager.java` -- Modify: `spark/src/main/scala/org/apache/comet/Native.scala` - -- [ ] **Step 1: Add requestSpill to Native.scala** - -In `spark/src/main/scala/org/apache/comet/Native.scala`, add after the `releasePlan` method (around line 104): - -```scala - /** - * Request that native operators spill memory. Called from CometTaskMemoryManager.spill(). - * - * @param nativePlanHandle - * the native ExecutionContext pointer - * @param size - * bytes requested to free - * @return - * actual bytes freed - */ - @native def requestSpill(nativePlanHandle: Long, size: Long): Long -``` - -- [ ] **Step 2: Update CometTaskMemoryManager to store native handle and implement spill** - -Replace the contents of `CometTaskMemoryManager.java`: - -```java -package org.apache.spark; - -import java.io.IOException; -import java.util.concurrent.atomic.AtomicLong; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.comet.Native; -import org.apache.spark.memory.MemoryConsumer; -import org.apache.spark.memory.MemoryMode; -import org.apache.spark.memory.TaskMemoryManager; - -/** - * A adapter class that is used by Comet native to acquire & release memory through Spark's unified - * memory manager. This assumes Spark's off-heap memory mode is enabled. - */ -public class CometTaskMemoryManager { - - private static final Logger logger = LoggerFactory.getLogger(CometTaskMemoryManager.class); - - /** The id uniquely identifies the native plan this memory manager is associated to */ - private final long id; - - private final long taskAttemptId; - - public final TaskMemoryManager internal; - private final NativeMemoryConsumer nativeMemoryConsumer; - private final AtomicLong used = new AtomicLong(); - - /** - * The native ExecutionContext handle. Set after native plan creation. Used to route spill() - * requests to the native memory pool. - */ - private volatile long nativePlanHandle = 0; - - public CometTaskMemoryManager(long id, long taskAttemptId) { - this.id = id; - this.taskAttemptId = taskAttemptId; - this.internal = TaskContext$.MODULE$.get().taskMemoryManager(); - this.nativeMemoryConsumer = new NativeMemoryConsumer(); - } - - /** - * Set the native plan handle after plan creation. This enables the spill() callback to route - * requests to the native memory pool. - */ - public void setNativePlanHandle(long handle) { - this.nativePlanHandle = handle; - } - - // Called by Comet native through JNI. - // Returns the actual amount of memory (in bytes) granted. - public long acquireMemory(long size) { - if (logger.isTraceEnabled()) { - logger.trace("Task {} requested {} bytes", taskAttemptId, size); - } - long acquired = internal.acquireExecutionMemory(size, nativeMemoryConsumer); - long newUsed = used.addAndGet(acquired); - if (acquired < size) { - logger.warn( - "Task {} requested {} bytes but only received {} bytes. Current allocation is {} and " - + "the total memory consumption is {} bytes.", - taskAttemptId, - size, - acquired, - newUsed, - internal.getMemoryConsumptionForThisTask()); - // If memory manager is not able to acquire the requested size, log memory usage - internal.showMemoryUsage(); - } - return acquired; - } - - // Called by Comet native through JNI - public void releaseMemory(long size) { - if (logger.isTraceEnabled()) { - logger.trace("Task {} released {} bytes", taskAttemptId, size); - } - long newUsed = used.addAndGet(-size); - if (newUsed < 0) { - logger.error( - "Task {} used memory is negative ({}) after releasing {} bytes", - taskAttemptId, - newUsed, - size); - } - internal.releaseExecutionMemory(size, nativeMemoryConsumer); - } - - public long getUsed() { - return used.get(); - } - - /** - * A memory consumer that routes spill requests to the native memory pool. When Spark's memory - * manager needs to reclaim memory (e.g., for another task), it calls spill() which signals the - * native pool to apply backpressure. DataFusion operators (Sort, Aggregate, Shuffle) react by - * spilling their internal state to disk. - */ - private class NativeMemoryConsumer extends MemoryConsumer { - protected NativeMemoryConsumer() { - super(CometTaskMemoryManager.this.internal, 0, MemoryMode.OFF_HEAP); - } - - @Override - public long spill(long size, MemoryConsumer trigger) throws IOException { - long handle = nativePlanHandle; - if (handle == 0) { - // Native plan not yet created or already destroyed - return 0; - } - logger.info( - "Task {} received spill request for {} bytes, forwarding to native", - taskAttemptId, - size); - try { - long freed = new Native().requestSpill(handle, size); - logger.info("Task {} native spill freed {} bytes", taskAttemptId, freed); - return freed; - } catch (Exception e) { - logger.warn("Task {} native spill failed: {}", taskAttemptId, e.getMessage()); - return 0; - } - } - - @Override - public String toString() { - return String.format("NativeMemoryConsumer(id=%d, taskAttemptId=%d)", id, taskAttemptId); - } - } -} -``` - -- [ ] **Step 3: Build JVM to verify compilation** - -Run: `./mvnw compile -DskipTests` -Expected: BUILD SUCCESS - -- [ ] **Step 4: Commit** - -```bash -git add spark/src/main/java/org/apache/spark/CometTaskMemoryManager.java \ - spark/src/main/scala/org/apache/comet/Native.scala -git commit -m "feat: implement spill() callback in CometTaskMemoryManager" -``` - ---- - -### Task 5: Wire Native Plan Handle Back to CometTaskMemoryManager - -**Files:** - -- Modify: `spark/src/main/scala/org/apache/comet/CometExecIterator.scala` - -- [ ] **Step 1: Set the native plan handle after creation** - -In `CometExecIterator.scala`, after the `nativeLib.createPlan(...)` call (around line 131), add: - -```scala - private val plan = { - // ... existing code that calls nativeLib.createPlan ... - - val nativePlan = nativeLib.createPlan( - id, - inputIterators, - protobufQueryPlan, - protobufSparkConfigs, - numParts, - nativeMetrics, - metricsUpdateInterval = COMET_METRICS_UPDATE_INTERVAL.get(), - cometTaskMemoryManager, - localDiskDirs, - batchSize = COMET_BATCH_SIZE.get(), - memoryConfig.offHeapMode, - memoryConfig.memoryPoolType, - memoryConfig.memoryLimit, - memoryConfig.memoryLimitPerTask, - taskAttemptId, - taskCPUs, - keyUnwrapper) - - // Enable spill callback by giving the memory manager a handle to the native plan - cometTaskMemoryManager.setNativePlanHandle(nativePlan) - - nativePlan - } -``` - -The existing code calls `nativeLib.createPlan(...)` and that's the last expression in the block (making it the return value). Restructure so we can call `setNativePlanHandle` and still return the handle: - -```scala - val handle = nativeLib.createPlan( - id, - inputIterators, - // ... all existing args unchanged ... - keyUnwrapper) - - // Enable spill callback by giving the memory manager a handle to the native plan - cometTaskMemoryManager.setNativePlanHandle(handle) - - handle - } -``` - -- [ ] **Step 2: Clear the handle on close** - -In the `close()` method (line 231), add `cometTaskMemoryManager.setNativePlanHandle(0)` -before the `nativeLib.releasePlan(plan)` call to prevent spill callbacks after the -native plan is destroyed: - -```scala - def close(): Unit = synchronized { - if (!closed) { - if (currentBatch != null) { - currentBatch.close() - currentBatch = null - } - nativeUtil.close() - shuffleBlockIterators.values.foreach(_.close()) - cometTaskMemoryManager.setNativePlanHandle(0) - nativeLib.releasePlan(plan) -``` - -- [ ] **Step 3: Build everything** - -Run: `make` -Expected: Both native and JVM build succeed. - -- [ ] **Step 4: Commit** - -```bash -git add spark/src/main/scala/org/apache/comet/CometExecIterator.scala -git commit -m "feat: wire native plan handle to CometTaskMemoryManager for spill routing" -``` - ---- - -### Task 6: Test with TPC-H Q9 - -- [ ] **Step 1: Run the memory profile script for Q9** - -```bash -cd benchmarks/tpc -./memory-profile.sh --queries "9" --offheap-sizes "4g 8g" --cores 4 -``` - -Compare peak RSS against the baseline results: - -- Spark 4g: 4580 MB -- Comet 4g (before): 5911 MB -- Comet 8g (before): 6359 MB - -**Expected:** The elastic growth (4g→8g delta) should be reduced because Spark can -now reclaim memory from Comet's shuffle writers when other tasks need it. The absolute -numbers may also decrease slightly. - -- [ ] **Step 2: Verify spill logging** - -Check the logs for spill activity: - -```bash -grep -i "spill" benchmarks/tpc/memory-profile-results/logs/comet-offheap4g-q9.log -``` - -Expected: Log lines showing "received spill request" and "native spill freed N bytes" -from `CometTaskMemoryManager`. - -- [ ] **Step 3: Run full test suite for regressions** - -```bash -./mvnw test -DwildcardSuites="CometExec" -``` - -Expected: All existing tests pass. - -- [ ] **Step 4: Commit results and update analysis** - -Update `docs/source/contributor-guide/memory-management.md` with the new results. - -```bash -git add docs/source/contributor-guide/memory-management.md -git commit -m "docs: add spill callback benchmark results" -``` - ---- - -## Notes - -- **Timeout**: The 10-second timeout in `request_spill()` is conservative. If no operators - are actively allocating, the spill won't trigger until one does. In practice, during - TPC-H execution, operators allocate frequently so the response should be fast. - -- **FairUnified pool**: Not covered by this plan. It uses a different pool struct - (`CometFairMemoryPool`) but the same pattern applies — add `SpillState`, check in - `try_grow`, record in `shrink`. - -- **Thread safety**: `spill()` is called from Spark's memory manager thread. - `try_grow()`/`shrink()` are called from tokio threads. The `AtomicUsize` + `Condvar` - design handles this safely without locks on the hot path (`pressure()` is a single - atomic load). - -- **Multiple native plans per task**: A single Spark task may create multiple - `CometExecIterator` instances (e.g., for subqueries). Each has its own - `CometTaskMemoryManager` and `NativeMemoryConsumer`. Spark will call `spill()` on each - independently, which is correct — each routes to its own pool. diff --git a/docs/superpowers/plans/2026-04-02-shuffle-ipc-stream-per-partition.md b/docs/superpowers/plans/2026-04-02-shuffle-ipc-stream-per-partition.md deleted file mode 100644 index a6fae86cff..0000000000 --- a/docs/superpowers/plans/2026-04-02-shuffle-ipc-stream-per-partition.md +++ /dev/null @@ -1,1438 +0,0 @@ -# Shuffle IPC Stream Per Partition — 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:** Reduce shuffle format overhead by writing one Arrow IPC stream per partition (schema once, N batches) instead of one IPC stream per batch, and move IPC stream reading to Rust via JNI InputStream callbacks. - -**Architecture:** Replace the custom header format + per-batch IPC stream with standard Arrow IPC streams using built-in body compression (LZ4_FRAME/ZSTD). On the read side, Rust pulls bytes from JVM's InputStream via a `JniInputStream` adapter and uses Arrow's `StreamReader` to decode batches. A native handle pattern (`open`/`next`/`close`) manages the reader lifecycle across JNI calls. - -**Tech Stack:** Arrow IPC (`StreamWriter`/`StreamReader` with `IpcWriteOptions`), JNI (`jni` crate), Arrow FFI - -**Spec:** `docs/superpowers/specs/2026-04-02-shuffle-ipc-stream-per-partition-design.md` - ---- - -## File Structure - -### Rust — Modified - -| File | Responsibility | -|---|---| -| `native/Cargo.toml` | Add `ipc_compression` feature to arrow dependency | -| `native/shuffle/Cargo.toml` | Remove `snap` dependency | -| `native/shuffle/src/lib.rs` | Update exports (remove `read_ipc_compressed`, `ShuffleBlockWriter`) | -| `native/shuffle/src/writers/mod.rs` | Remove `shuffle_block_writer` module, remove `ShuffleBlockWriter` export | -| `native/shuffle/src/writers/shuffle_block_writer.rs` | **Delete** — replaced by persistent `StreamWriter` | -| `native/shuffle/src/writers/buf_batch_writer.rs` | Remove `ShuffleBlockWriter` dependency, write directly via `StreamWriter` | -| `native/shuffle/src/writers/spill.rs` | Remove `ShuffleBlockWriter` dependency, write via `StreamWriter` | -| `native/shuffle/src/ipc.rs` | Replace `read_ipc_compressed` with `JniInputStream` and stream reader handle | -| `native/shuffle/src/partitioners/immediate_mode.rs` | `PartitionOutputStream`: persistent `StreamWriter`, Arrow-native compression | -| `native/shuffle/src/partitioners/single_partition.rs` | Use persistent `StreamWriter` directly | -| `native/shuffle/src/partitioners/multi_partition.rs` | Use persistent `StreamWriter` per partition | -| `native/shuffle/src/shuffle_writer.rs` | Update `CompressionCodec` → Arrow IPC `CompressionType`, update tests | -| `native/core/src/execution/jni_api.rs` | Replace `decodeShuffleBlock` with `openShuffleStream`/`nextShuffleStreamBatch`/`closeShuffleStream` | - -### Scala — Modified - -| File | Responsibility | -|---|---| -| `spark/src/main/scala/org/apache/comet/Native.scala` | Replace `decodeShuffleBlock` with new JNI method declarations | -| `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala` | Use native handle pattern, remove header parsing | -| `common/src/main/scala/org/apache/comet/CometConf.scala` | Remove `snappy` from codec options | - ---- - -## Task 1: Enable Arrow IPC Compression Feature - -**Files:** -- Modify: `native/Cargo.toml:37` - -- [ ] **Step 1: Add `ipc_compression` feature to arrow dependency** - -In `native/Cargo.toml`, change line 37 from: -```toml -arrow = { version = "57.3.0", features = ["prettyprint", "ffi", "chrono-tz"] } -``` -to: -```toml -arrow = { version = "57.3.0", features = ["prettyprint", "ffi", "chrono-tz", "ipc_compression"] } -``` - -- [ ] **Step 2: Verify it compiles** - -Run: `cargo check --manifest-path native/Cargo.toml` -Expected: Compilation succeeds with no errors. - -- [ ] **Step 3: Commit** - -```bash -git add native/Cargo.toml -git commit -m "feat: enable Arrow IPC compression feature for shuffle format" -``` - ---- - -## Task 2: Replace `CompressionCodec` with Arrow IPC Compression - -The existing `CompressionCodec` enum in `shuffle_block_writer.rs` manages Snappy/LZ4/Zstd/None with external compression wrappers. Replace it with a thin wrapper around Arrow IPC's `CompressionType` so the `StreamWriter` handles compression internally. - -**Files:** -- Modify: `native/shuffle/src/writers/shuffle_block_writer.rs` (will be replaced with just the codec enum) -- Modify: `native/shuffle/src/writers/mod.rs` -- Modify: `native/shuffle/src/lib.rs` - -- [ ] **Step 1: Replace `CompressionCodec` in `shuffle_block_writer.rs`** - -Replace the entire contents of `native/shuffle/src/writers/shuffle_block_writer.rs` with: - -```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 arrow::ipc::writer::IpcWriteOptions; -use arrow::ipc::CompressionType; - -/// Compression codec for shuffle IPC streams. -/// -/// Maps to Arrow IPC's built-in body compression. Each record batch's data -/// buffers are individually compressed by the `StreamWriter`. -#[derive(Debug, Clone)] -pub enum CompressionCodec { - None, - Lz4Frame, - Zstd(i32), -} - -impl CompressionCodec { - /// Create `IpcWriteOptions` with the appropriate compression setting. - pub fn ipc_write_options(&self) -> datafusion::error::Result { - let compression = match self { - CompressionCodec::None => None, - CompressionCodec::Lz4Frame => Some(CompressionType::LZ4_FRAME), - CompressionCodec::Zstd(_) => Some(CompressionType::ZSTD), - }; - IpcWriteOptions::try_with_compression(8, false, arrow::ipc::MetadataVersion::V5, compression) - .map_err(|e| datafusion::common::DataFusionError::ArrowError(Box::from(e), None)) - } -} -``` - -- [ ] **Step 2: Update `writers/mod.rs` exports** - -Replace the contents of `native/shuffle/src/writers/mod.rs`: - -```rust -// (keep license header) - -mod buf_batch_writer; -mod checksum; -mod shuffle_block_writer; -mod spill; - -pub(crate) use buf_batch_writer::BufBatchWriter; -pub(crate) use checksum::Checksum; -pub use shuffle_block_writer::CompressionCodec; -pub(crate) use spill::PartitionWriter; -``` - -Note: `ShuffleBlockWriter` is no longer exported — only `CompressionCodec` remains. - -- [ ] **Step 3: Update `lib.rs` exports** - -In `native/shuffle/src/lib.rs`, change: -```rust -pub use writers::{CompressionCodec, ShuffleBlockWriter}; -``` -to: -```rust -pub use writers::CompressionCodec; -``` - -Also remove the `read_ipc_compressed` export: -```rust -pub use ipc::read_ipc_compressed; -``` -becomes: -```rust -// (remove this line entirely — will be replaced in Task 6) -``` - -- [ ] **Step 4: Verify it compiles (expect errors in dependent code)** - -Run: `cargo check --manifest-path native/Cargo.toml 2>&1 | head -50` -Expected: Errors in files that still reference `ShuffleBlockWriter` and `read_ipc_compressed`. This is expected — we'll fix them in subsequent tasks. - -- [ ] **Step 5: Commit** - -```bash -git add native/shuffle/src/writers/shuffle_block_writer.rs native/shuffle/src/writers/mod.rs native/shuffle/src/lib.rs -git commit -m "refactor: replace ShuffleBlockWriter with CompressionCodec wrapper for Arrow IPC" -``` - ---- - -## Task 3: Update `PartitionOutputStream` (Immediate Mode) to Persistent StreamWriter - -Currently `PartitionOutputStream` creates a new `StreamWriter` per `write_ipc_block()` call. Change it to hold a persistent `StreamWriter` that writes the schema once and appends record batch messages for subsequent writes. - -**Files:** -- Modify: `native/shuffle/src/partitioners/immediate_mode.rs:284-369` (the `PartitionOutputStream` struct and impl) - -- [ ] **Step 1: Replace `PartitionOutputStream`** - -Replace the `PartitionOutputStream` struct and its `impl` block (lines 284-369) with: - -```rust -pub(crate) struct PartitionOutputStream { - /// Buffer that the StreamWriter writes into. - buffer: Vec, - /// Persistent Arrow IPC stream writer. Created lazily on first write. - writer: Option>>, - /// IPC write options with compression settings. - write_options: IpcWriteOptions, - /// Schema for creating the writer on first use. - schema: SchemaRef, -} - -impl PartitionOutputStream { - pub(crate) fn try_new(schema: SchemaRef, codec: CompressionCodec) -> Result { - let write_options = codec.ipc_write_options()?; - Ok(Self { - buffer: Vec::new(), - writer: None, - write_options, - schema, - }) - } - - fn write_batch(&mut self, batch: &RecordBatch) -> Result { - let start_pos = self.buffer.len(); - - if self.writer.is_none() { - // SAFETY: We ensure the buffer outlives the writer by owning both - // in this struct and only dropping the writer before the buffer. - let buf_ptr: *mut Vec = &mut self.buffer; - let buf_ref: &'static mut Vec = unsafe { &mut *buf_ptr }; - let w = StreamWriter::try_new_with_options(buf_ref, &self.schema, self.write_options.clone())?; - self.writer = Some(w); - } - - self.writer.as_mut().unwrap().write(batch)?; - - Ok(self.buffer.len() - start_pos) - } - - /// Finish the IPC stream (writes EOS marker) and return the buffer. - fn finish(&mut self) -> Result> { - if let Some(mut w) = self.writer.take() { - w.finish()?; - } - Ok(std::mem::take(&mut self.buffer)) - } - - fn drain_buffer(&mut self) -> Vec { - // When draining mid-stream (for spilling), we take the buffer contents - // but keep the writer alive. The writer's internal reference still points - // to self.buffer, which is now empty but still valid. - std::mem::take(&mut self.buffer) - } - - fn buffer_len(&self) -> usize { - self.buffer.len() - } -} -``` - -**Important:** The unsafe lifetime trick with the buffer pointer is needed because `StreamWriter` borrows the `Write` target. We need the writer to persist across calls while the buffer is drained. An alternative is to use `Cursor>` or a shared buffer wrapper — the implementer should evaluate which is cleanest. If the unsafe approach is concerning, consider using `StreamWriter>` and calling `into_inner()` when draining. - -**Simpler alternative using `StreamWriter>`:** - -Actually, a cleaner approach: since `Vec` implements `Write`, we can have the `StreamWriter` own the `Vec`. When we need to drain, we `finish()` the writer, take the buffer, and set `writer = None` so the next write creates a fresh stream. This means each spill creates a separate IPC stream. On read, the reader sees multiple concatenated IPC streams per partition (from spills), which Arrow `StreamReader` handles — it reads until EOS, then the JVM-side iterator moves to the next stream. - -Replace the above with this cleaner version: - -```rust -pub(crate) struct PartitionOutputStream { - /// Persistent Arrow IPC stream writer. Owns the buffer. - /// None before first write or after a drain (spill). - writer: Option>>, - /// IPC write options with compression settings. - write_options: IpcWriteOptions, - /// Schema for creating the writer. - schema: SchemaRef, - /// Accumulated bytes from finished (drained) streams. - spilled_bytes: Vec, -} - -impl PartitionOutputStream { - pub(crate) fn try_new(schema: SchemaRef, codec: CompressionCodec) -> Result { - let write_options = codec.ipc_write_options()?; - Ok(Self { - writer: None, - write_options, - schema, - spilled_bytes: Vec::new(), - }) - } - - fn ensure_writer(&mut self) -> Result<()> { - if self.writer.is_none() { - let w = StreamWriter::try_new_with_options( - Vec::new(), - &self.schema, - self.write_options.clone(), - )?; - self.writer = Some(w); - } - Ok(()) - } - - fn write_batch(&mut self, batch: &RecordBatch) -> Result { - self.ensure_writer()?; - let w = self.writer.as_mut().unwrap(); - // Get position before write to calculate bytes written - // StreamWriter writes to Vec which grows, so track via get_ref().len() - let before = w.get_ref().len(); - w.write(batch)?; - let after = w.get_ref().len(); - Ok(after - before) - } - - /// Finish the current IPC stream and drain all accumulated bytes. - /// After this call, the next write_batch creates a new IPC stream. - fn drain_buffer(&mut self) -> Result> { - let mut result = std::mem::take(&mut self.spilled_bytes); - if let Some(mut w) = self.writer.take() { - w.finish()?; - result.extend(w.into_inner()?); - } - Ok(result) - } - - /// Finish the IPC stream and return all bytes (spilled + in-memory). - fn finish(mut self) -> Result> { - self.drain_buffer() - } - - fn buffer_len(&self) -> usize { - self.spilled_bytes.len() - + self.writer.as_ref().map_or(0, |w| w.get_ref().len()) - } -} -``` - -- [ ] **Step 2: Update `flush_partition` to use new API** - -In `ImmediateModePartitioner::flush_partition` (around line 600), change: -```rust -let ipc_bytes = self.streams[pid].write_ipc_block(&output_batch)?; -``` -to: -```rust -let ipc_bytes = self.streams[pid].write_batch(&output_batch)?; -``` - -- [ ] **Step 3: Update `spill_all` to use `drain_buffer`** - -In `ImmediateModePartitioner::spill_all` (around line 622), the drain call changes from: -```rust -let buf = self.streams[pid].drain_buffer(); -``` -to: -```rust -let buf = self.streams[pid].drain_buffer()?; -``` -(It now returns `Result>` because it finishes the IPC stream.) - -- [ ] **Step 4: Update `shuffle_write` to use `drain_buffer`** - -In `ImmediateModePartitioner::shuffle_write` (around line 741), change: -```rust -let buf = self.streams[pid].drain_buffer(); -``` -to: -```rust -let buf = self.streams[pid].drain_buffer()?; -``` - -- [ ] **Step 5: Remove old imports** - -In `immediate_mode.rs`, remove the `CompressionCodec` import from `crate::` (if it was used for the codec tag matching) and remove `use arrow::ipc::writer::StreamWriter;` at the top — it's now used inside `PartitionOutputStream` via the `StreamWriter` from `arrow::ipc::writer`. Actually, keep the `StreamWriter` import since `PartitionOutputStream` uses it. Remove imports that are no longer needed: -- Remove the old `CompressionCodec` matching arms for `Snappy` codec tags (`b"SNAP"` etc.) - -Add new import: -```rust -use arrow::ipc::writer::{IpcWriteOptions, StreamWriter}; -``` - -- [ ] **Step 6: Update test `test_partition_output_stream_write_and_read`** - -Replace the test (around line 856) with: - -```rust -#[test] -#[cfg_attr(miri, ignore)] -fn test_partition_output_stream_write_and_read() { - use arrow::ipc::reader::StreamReader; - - let batch = make_test_batch(&[1, 2, 3, 4, 5]); - let schema = batch.schema(); - - for codec in [ - CompressionCodec::None, - CompressionCodec::Lz4Frame, - CompressionCodec::Zstd(1), - ] { - let mut stream = - PartitionOutputStream::try_new(Arc::clone(&schema), codec).unwrap(); - stream.write_batch(&batch).unwrap(); - stream.write_batch(&batch).unwrap(); // write 2 batches - - let buf = stream.finish().unwrap(); - assert!(!buf.is_empty()); - - // Read back using standard Arrow IPC StreamReader - let cursor = std::io::Cursor::new(&buf); - let reader = StreamReader::try_new(cursor, None).unwrap(); - let batches: Vec = reader.map(|r| r.unwrap()).collect(); - assert_eq!(batches.len(), 2); - assert_eq!(batches[0].num_rows(), 5); - assert_eq!(batches[1].num_rows(), 5); - } -} -``` - -- [ ] **Step 7: Run tests** - -Run: `cd native && cargo test -p datafusion-comet-shuffle test_partition_output_stream` -Expected: `test_partition_output_stream_write_and_read` passes. - -- [ ] **Step 8: Commit** - -```bash -git add native/shuffle/src/partitioners/immediate_mode.rs -git commit -m "feat: persistent StreamWriter per partition in immediate mode shuffle" -``` - ---- - -## Task 4: Update Buffered Mode (`MultiPartitionShuffleRepartitioner`) Writers - -The buffered mode uses `ShuffleBlockWriter` via `PartitionWriter` and `BufBatchWriter`. Replace these with persistent `StreamWriter` per partition. - -**Files:** -- Modify: `native/shuffle/src/writers/spill.rs` -- Modify: `native/shuffle/src/writers/buf_batch_writer.rs` -- Modify: `native/shuffle/src/partitioners/multi_partition.rs` - -- [ ] **Step 1: Rewrite `PartitionWriter` in `spill.rs`** - -Replace the entire contents of `native/shuffle/src/writers/spill.rs` with: - -```rust -// (keep license header) - -use arrow::array::RecordBatch; -use arrow::datatypes::SchemaRef; -use arrow::ipc::writer::{IpcWriteOptions, StreamWriter}; -use datafusion::common::DataFusionError; -use datafusion::execution::disk_manager::RefCountedTempFile; -use datafusion::execution::runtime_env::RuntimeEnv; -use datafusion::physical_plan::metrics::Time; -use std::fs::{File, OpenOptions}; -use std::io::Write; - -use crate::partitioners::PartitionedBatchIterator; -use crate::metrics::ShufflePartitionerMetrics; - -/// A temporary disk file for spilling a partition's intermediate shuffle data. -struct SpillFile { - temp_file: RefCountedTempFile, - file: File, -} - -/// Manages encoding and optional disk spilling for a single shuffle partition. -/// Each partition gets one IPC stream (schema written once), with spills creating -/// additional IPC streams that are concatenated in the final output. -pub(crate) struct PartitionWriter { - spill_file: Option, - schema: SchemaRef, - write_options: IpcWriteOptions, -} - -impl PartitionWriter { - pub(crate) fn try_new( - schema: SchemaRef, - write_options: IpcWriteOptions, - ) -> datafusion::common::Result { - Ok(Self { - spill_file: None, - schema, - write_options, - }) - } - - fn ensure_spill_file_created( - &mut self, - runtime: &RuntimeEnv, - ) -> datafusion::common::Result<()> { - if self.spill_file.is_none() { - let spill_file = runtime - .disk_manager - .create_tmp_file("shuffle writer spill")?; - let spill_data = OpenOptions::new() - .write(true) - .create(true) - .truncate(true) - .open(spill_file.path()) - .map_err(|e| { - DataFusionError::Execution(format!("Error occurred while spilling {e}")) - })?; - self.spill_file = Some(SpillFile { - temp_file: spill_file, - file: spill_data, - }); - } - Ok(()) - } - - pub(crate) fn spill( - &mut self, - iter: &mut PartitionedBatchIterator, - runtime: &RuntimeEnv, - metrics: &ShufflePartitionerMetrics, - ) -> datafusion::common::Result { - if let Some(batch) = iter.next() { - self.ensure_spill_file_created(runtime)?; - - let file = &mut self.spill_file.as_mut().unwrap().file; - let mut writer = StreamWriter::try_new_with_options( - file, - &self.schema, - self.write_options.clone(), - )?; - - let batch = batch?; - let mut encode_timer = metrics.encode_time.timer(); - writer.write(&batch)?; - encode_timer.stop(); - - for batch in iter { - let batch = batch?; - let mut encode_timer = metrics.encode_time.timer(); - writer.write(&batch)?; - encode_timer.stop(); - } - - let mut encode_timer = metrics.encode_time.timer(); - writer.finish()?; - let file = writer.into_inner()?; - file.flush()?; - encode_timer.stop(); - - let bytes_written = file.metadata() - .map(|m| m.len() as usize) - .unwrap_or(0); - - Ok(bytes_written) - } else { - Ok(0) - } - } - - pub(crate) fn path(&self) -> Option<&std::path::Path> { - self.spill_file - .as_ref() - .map(|spill_file| spill_file.temp_file.path()) - } - - #[cfg(test)] - pub(crate) fn has_spill_file(&self) -> bool { - self.spill_file.is_some() - } -} -``` - -- [ ] **Step 2: Rewrite `BufBatchWriter` in `buf_batch_writer.rs`** - -The `BufBatchWriter` now wraps a `StreamWriter` directly instead of a `ShuffleBlockWriter`. Replace the entire contents of `native/shuffle/src/writers/buf_batch_writer.rs` with: - -```rust -// (keep license header) - -use arrow::array::RecordBatch; -use arrow::compute::kernels::coalesce::BatchCoalescer; -use arrow::datatypes::SchemaRef; -use arrow::ipc::writer::{IpcWriteOptions, StreamWriter}; -use datafusion::physical_plan::metrics::Time; -use std::io::Write; - -/// Write batches to an Arrow IPC stream with coalescing. -/// -/// Small batches are coalesced using Arrow's [`BatchCoalescer`] before serialization, -/// producing exactly `batch_size`-row output batches to reduce per-batch overhead. -/// Uses a persistent `StreamWriter` — schema is written once. -pub(crate) struct BufBatchWriter { - writer: StreamWriter, - /// Coalesces small batches into target_batch_size before serialization. - coalescer: Option, - schema: SchemaRef, - batch_size: usize, -} - -impl BufBatchWriter { - pub(crate) fn new( - inner: W, - schema: &SchemaRef, - write_options: &IpcWriteOptions, - batch_size: usize, - ) -> datafusion::common::Result { - let writer = StreamWriter::try_new_with_options( - inner, - schema, - write_options.clone(), - )?; - Ok(Self { - writer, - coalescer: None, - schema: schema.clone(), - batch_size, - }) - } - - pub(crate) fn write( - &mut self, - batch: &RecordBatch, - encode_time: &Time, - _write_time: &Time, - ) -> datafusion::common::Result { - let coalescer = self - .coalescer - .get_or_insert_with(|| BatchCoalescer::new(self.schema.clone(), self.batch_size)); - coalescer.push_batch(batch.clone())?; - - let mut completed = Vec::new(); - while let Some(batch) = coalescer.next_completed_batch() { - completed.push(batch); - } - - let mut bytes_written = 0; - for batch in &completed { - let mut timer = encode_time.timer(); - let before = self.writer.get_ref().len(); - self.writer.write(batch)?; - let after = self.writer.get_ref().len(); - bytes_written += after - before; - timer.stop(); - } - Ok(bytes_written) - } - - pub(crate) fn flush( - &mut self, - encode_time: &Time, - _write_time: &Time, - ) -> datafusion::common::Result<()> { - let mut remaining = Vec::new(); - if let Some(coalescer) = &mut self.coalescer { - coalescer.finish_buffered_batch()?; - while let Some(batch) = coalescer.next_completed_batch() { - remaining.push(batch); - } - } - for batch in &remaining { - let mut timer = encode_time.timer(); - self.writer.write(batch)?; - timer.stop(); - } - - let mut timer = encode_time.timer(); - self.writer.finish()?; - timer.stop(); - Ok(()) - } -} -``` - -**Note:** The `write()` return value (bytes written) is only used for metrics. Since the `StreamWriter` writes directly to the underlying `W`, and for file-backed writers we track offsets via `stream_position()`, the return value from `write()` can be dropped. Change the signature to return `Result<()>` and remove the byte counting logic. Callers that previously used the return value for offset tracking should use `stream_position()` on the underlying file instead. - -- [ ] **Step 3: Update `writers/mod.rs`** - -```rust -// (keep license header) - -mod buf_batch_writer; -mod checksum; -mod shuffle_block_writer; -mod spill; - -pub(crate) use buf_batch_writer::BufBatchWriter; -pub(crate) use checksum::Checksum; -pub use shuffle_block_writer::CompressionCodec; -pub(crate) use spill::PartitionWriter; -``` - -(No change needed — `BufBatchWriter` and `PartitionWriter` are still exported with the same names.) - -- [ ] **Step 4: Update `MultiPartitionShuffleRepartitioner`** - -In `native/shuffle/src/partitioners/multi_partition.rs`: - -Change the struct fields — remove `shuffle_block_writer: ShuffleBlockWriter` and add `write_options: IpcWriteOptions`: - -At approximately line 113, replace: -```rust - shuffle_block_writer: ShuffleBlockWriter, -``` -with: -```rust - write_options: IpcWriteOptions, -``` - -In `try_new`, change the construction of `partition_writers` from: -```rust -let shuffle_block_writer = ShuffleBlockWriter::try_new(schema.as_ref(), codec.clone())?; -// ... -.map(|_| PartitionWriter::try_new(shuffle_block_writer.clone())) -``` -to: -```rust -let write_options = codec.ipc_write_options()?; -// ... -.map(|_| PartitionWriter::try_new(schema.clone(), write_options.clone())) -``` - -Update `shuffle_write_partition` to create a `BufBatchWriter` with the new API: -```rust -fn shuffle_write_partition( - partition_iter: &mut PartitionedBatchIterator, - output_data: &mut BufWriter, - schema: &SchemaRef, - write_options: &IpcWriteOptions, - encode_time: &Time, - write_time: &Time, - batch_size: usize, -) -> datafusion::common::Result<()> { - let mut buf_batch_writer = BufBatchWriter::new( - output_data, - schema, - write_options, - batch_size, - )?; - for batch in partition_iter { - let batch = batch?; - buf_batch_writer.write(&batch, encode_time, write_time)?; - } - buf_batch_writer.flush(encode_time, write_time)?; - Ok(()) -} -``` - -Update the call site in `shuffle_write` (around line 595): -```rust -Self::shuffle_write_partition( - &mut partition_iter, - &mut output_data, - &self.schema, - &self.write_options, - &self.metrics.encode_time, - &self.metrics.write_time, - self.batch_size, -)?; -``` - -Update `spill` to pass the new args: -```rust -spilled_bytes += partition_writer.spill( - &mut iter, - &self.runtime, - &self.metrics, -)?; -``` -(Remove `write_buffer_size` and `batch_size` args from `spill` — the spill writer now creates its own `StreamWriter`.) - -- [ ] **Step 5: Update `SinglePartitionShufflePartitioner`** - -In `native/shuffle/src/partitioners/single_partition.rs`, change the struct to hold a `BufBatchWriter` created with the new API: - -```rust -pub(crate) struct SinglePartitionShufflePartitioner { - output_data_writer: BufBatchWriter, - output_index_path: String, - metrics: ShufflePartitionerMetrics, - batch_size: usize, -} -``` - -Update `try_new`: -```rust -pub(crate) fn try_new( - output_data_path: String, - output_index_path: String, - schema: SchemaRef, - metrics: ShufflePartitionerMetrics, - batch_size: usize, - codec: CompressionCodec, - _write_buffer_size: usize, -) -> datafusion::common::Result { - let write_options = codec.ipc_write_options()?; - let output_data_file = OpenOptions::new() - .write(true) - .create(true) - .truncate(true) - .open(output_data_path)?; - - let output_data_writer = BufBatchWriter::new( - output_data_file, - &schema, - &write_options, - batch_size, - )?; - - Ok(Self { - output_data_writer, - output_index_path, - metrics, - batch_size, - }) -} -``` - -Remove `buffered_batches`, `num_buffered_rows`, `add_buffered_batch`, and `concat_buffered_batches` — the `BufBatchWriter` handles coalescing internally. Simplify `insert_batch` to just call `self.output_data_writer.write(...)`. - -- [ ] **Step 6: Verify Rust compilation** - -Run: `cargo check --manifest-path native/Cargo.toml` -Expected: Compiles with only remaining errors in `jni_api.rs` (the read side, Task 6). - -- [ ] **Step 7: Run existing shuffle writer tests** - -Run: `cd native && cargo test -p datafusion-comet-shuffle` -Expected: All write-side tests pass. Read-side tests (`read_ipc_compressed`) will fail — expected. - -- [ ] **Step 8: Commit** - -```bash -git add native/shuffle/src/writers/ native/shuffle/src/partitioners/ -git commit -m "feat: persistent StreamWriter per partition in buffered mode shuffle" -``` - ---- - -## Task 5: Remove Snappy Support from Config and Dependencies - -**Files:** -- Modify: `native/shuffle/Cargo.toml` -- Modify: `common/src/main/scala/org/apache/comet/CometConf.scala:433-442` - -- [ ] **Step 1: Remove `snap` dependency from Cargo.toml** - -In `native/shuffle/Cargo.toml`, remove line: -```toml -snap = "1.1" -``` - -- [ ] **Step 2: Update CometConf to remove snappy** - -In `common/src/main/scala/org/apache/comet/CometConf.scala`, change line 441: -```scala - .checkValues(Set("zstd", "lz4", "snappy")) -``` -to: -```scala - .checkValues(Set("zstd", "lz4")) -``` - -And update the doc string (line 437-438): -```scala - "The codec of Comet native shuffle used to compress shuffle data. lz4, zstd, and " + - "snappy are supported. Compression can be disabled by setting " + -``` -to: -```scala - "The codec of Comet native shuffle used to compress shuffle data. " + - "lz4 and zstd are supported. Compression can be disabled by setting " + -``` - -- [ ] **Step 3: Remove any remaining Snappy references in Rust code** - -Search for and remove any `snap::` or `Snappy` references in the shuffle crate. After Tasks 2-4, there should be none left in the writer code. Check with: - -Run: `grep -rn "snap\|Snappy\|SNAP" native/shuffle/src/` - -Remove any remaining references. - -- [ ] **Step 4: Commit** - -```bash -git add native/shuffle/Cargo.toml common/src/main/scala/org/apache/comet/CometConf.scala -git commit -m "chore: remove Snappy codec support from shuffle (Arrow IPC supports LZ4/ZSTD)" -``` - ---- - -## Task 6: Implement `JniInputStream` and Stream Reader Handle (Rust Read Side) - -Create the Rust-side JNI infrastructure for reading Arrow IPC streams from JVM InputStreams. - -**Files:** -- Modify: `native/shuffle/src/ipc.rs` — replace `read_ipc_compressed` with `JniInputStream` and `ShuffleStreamReader` -- Modify: `native/core/src/execution/jni_api.rs` — replace `decodeShuffleBlock` with new JNI functions - -- [ ] **Step 1: Rewrite `ipc.rs` with `JniInputStream` and `ShuffleStreamReader`** - -Replace the entire contents of `native/shuffle/src/ipc.rs` with: - -```rust -// (keep license header) - -use arrow::array::RecordBatch; -use arrow::ipc::reader::StreamReader; -use jni::objects::{GlobalRef, JByteArray, JObject, JValue}; -use jni::sys::jint; -use jni::{JNIEnv, JavaVM}; -use std::io::{self, Read}; - -/// Buffer size for JNI read-ahead. Minimizes JNI boundary crossings. -const JNI_READ_BUFFER_SIZE: usize = 64 * 1024; // 64KB - -/// A `Read` adapter that pulls bytes from a JVM `InputStream` via JNI callbacks. -/// -/// Uses an internal read-ahead buffer to minimize JNI call overhead. -pub struct JniInputStream { - jvm: JavaVM, - stream: GlobalRef, - jbuf: JByteArray, - buf: Vec, - pos: usize, - len: usize, -} - -impl JniInputStream { - /// Create a new `JniInputStream` wrapping a JVM `InputStream`. - /// - /// The `input_stream` object must be a valid `java.io.InputStream`. - pub fn new(env: &mut JNIEnv, input_stream: &JObject) -> jni::errors::Result { - let jvm = env.get_java_vm()?; - let stream = env.new_global_ref(input_stream)?; - let jbuf = env.new_byte_array(JNI_READ_BUFFER_SIZE as jint)?; - let jbuf = JByteArray::from(env.new_global_ref(jbuf)?.as_ref().as_raw()); - Ok(Self { - jvm, - stream, - jbuf, - buf: vec![0u8; JNI_READ_BUFFER_SIZE], - pos: 0, - len: 0, - }) - } - - fn refill(&mut self) -> io::Result<()> { - let mut env = self.jvm.attach_current_thread().map_err(|e| { - io::Error::new(io::ErrorKind::Other, format!("JNI attach failed: {e}")) - })?; - - let n = env - .call_method( - &self.stream, - "read", - "([BII)I", - &[ - JValue::Object(self.jbuf.as_ref().into()), - JValue::Int(0), - JValue::Int(JNI_READ_BUFFER_SIZE as jint), - ], - ) - .and_then(|v| v.i()) - .map_err(|e| { - io::Error::new(io::ErrorKind::Other, format!("JNI read failed: {e}")) - })?; - - if n <= 0 { - self.pos = 0; - self.len = 0; - return Ok(()); - } - - let n = n as usize; - env.get_byte_array_region(&self.jbuf, 0, &mut self.buf[..n].iter().map(|b| *b as i8).collect::>()) - .map_err(|e| { - io::Error::new(io::ErrorKind::Other, format!("JNI get_byte_array_region failed: {e}")) - })?; - - // Actually, get_byte_array_region writes into an &mut [i8], not returns. - // Let's use a simpler approach: get_byte_array_elements or copy into our buffer. - // The cleanest approach: - let mut i8_buf = vec![0i8; n]; - env.get_byte_array_region(&self.jbuf, 0, &mut i8_buf) - .map_err(|e| { - io::Error::new(io::ErrorKind::Other, format!("JNI copy failed: {e}")) - })?; - // Reinterpret i8 as u8 - self.buf[..n].copy_from_slice(unsafe { - std::slice::from_raw_parts(i8_buf.as_ptr() as *const u8, n) - }); - - self.pos = 0; - self.len = n; - Ok(()) - } -} - -impl Read for JniInputStream { - fn read(&mut self, buf: &mut [u8]) -> io::Result { - if self.pos >= self.len { - self.refill()?; - if self.len == 0 { - return Ok(0); // EOF - } - } - - let available = self.len - self.pos; - let to_copy = available.min(buf.len()); - buf[..to_copy].copy_from_slice(&self.buf[self.pos..self.pos + to_copy]); - self.pos += to_copy; - Ok(to_copy) - } -} - -/// Holds a persistent Arrow IPC `StreamReader` across JNI calls. -/// -/// The reader is created once when the shuffle stream is opened, and -/// `next_batch()` is called repeatedly to read individual batches. -pub struct ShuffleStreamReader { - reader: StreamReader, - num_fields: usize, -} - -impl ShuffleStreamReader { - /// Create a new reader wrapping a JVM InputStream. - pub fn new(env: &mut JNIEnv, input_stream: &JObject) -> Result { - let jni_stream = JniInputStream::new(env, input_stream) - .map_err(|e| format!("Failed to create JniInputStream: {e}"))?; - - let reader = unsafe { - StreamReader::try_new(jni_stream, None) - .map_err(|e| format!("Failed to create StreamReader: {e}"))? - .with_skip_validation(true) - }; - - let num_fields = reader.schema().fields().len(); - - Ok(Self { reader, num_fields }) - } - - /// Read the next batch from the stream. - /// Returns `None` when the stream is exhausted. - pub fn next_batch(&mut self) -> Result, String> { - match self.reader.next() { - Some(Ok(batch)) => Ok(Some(batch)), - Some(Err(e)) => Err(format!("Error reading batch: {e}")), - None => Ok(None), - } - } - - /// Number of fields (columns) in the schema. - pub fn num_fields(&self) -> usize { - self.num_fields - } -} -``` - -**Note to implementer:** The `JByteArray` handling in `refill()` above is pseudocode — the exact JNI API for byte array handling with the `jni` crate needs careful implementation. The `jni` crate's `get_byte_array_region` takes `&mut [jbyte]` (which is `&mut [i8]`). The key pattern is: -1. Call `inputStream.read(byte[], 0, len)` via `call_method` -2. Copy the Java byte[] contents to the Rust buffer via `get_byte_array_region` -3. Reinterpret `i8` as `u8` - -Also, the `JByteArray` from a `GlobalRef` conversion needs care — the implementer should verify this compiles. A `GlobalRef` to the byte array must be stored, and a local ref obtained for each JNI call. - -- [ ] **Step 2: Update `lib.rs` exports** - -In `native/shuffle/src/lib.rs`, replace: -```rust -pub use ipc::read_ipc_compressed; -``` -with: -```rust -pub use ipc::{JniInputStream, ShuffleStreamReader}; -``` - -- [ ] **Step 3: Add JNI functions in `jni_api.rs`** - -In `native/core/src/execution/jni_api.rs`, replace the `decodeShuffleBlock` function (lines 878-900) with three new functions: - -```rust -#[no_mangle] -/// Open a shuffle stream reader over a JVM InputStream. -/// Returns an opaque handle (pointer) to a ShuffleStreamReader. -/// # Safety -/// This function is inherently unsafe since it deals with JNI objects. -pub unsafe extern "system" fn Java_org_apache_comet_Native_openShuffleStream( - e: JNIEnv, - _class: JClass, - input_stream: JObject, -) -> jlong { - try_unwrap_or_throw(&e, |mut env| { - let reader = datafusion_comet_shuffle::ShuffleStreamReader::new(&mut env, &input_stream) - .map_err(|e| CometError::Internal(e))?; - let boxed = Box::new(reader); - Ok(Box::into_raw(boxed) as jlong) - }) -} - -#[no_mangle] -/// Read the next batch from a shuffle stream. -/// Returns the number of rows, or -1 if the stream is exhausted. -/// # Safety -/// This function is inherently unsafe since it deals with raw pointers from JNI. -pub unsafe extern "system" fn Java_org_apache_comet_Native_nextShuffleStreamBatch( - e: JNIEnv, - _class: JClass, - handle: jlong, - array_addrs: JLongArray, - schema_addrs: JLongArray, -) -> jlong { - try_unwrap_or_throw(&e, |mut env| { - let reader = unsafe { &mut *(handle as *mut datafusion_comet_shuffle::ShuffleStreamReader) }; - match reader.next_batch() { - Ok(Some(batch)) => { - prepare_output(&mut env, array_addrs, schema_addrs, batch, false) - } - Ok(None) => Ok(-1), - Err(e) => Err(CometError::Internal(e)), - } - }) -} - -#[no_mangle] -/// Close and drop a shuffle stream reader. -/// # Safety -/// This function is inherently unsafe since it deals with raw pointers from JNI. -pub unsafe extern "system" fn Java_org_apache_comet_Native_closeShuffleStream( - e: JNIEnv, - _class: JClass, - handle: jlong, -) { - try_unwrap_or_throw(&e, |_env| { - if handle != 0 { - unsafe { drop(Box::from_raw(handle as *mut datafusion_comet_shuffle::ShuffleStreamReader)) }; - } - Ok(()) - }); -} -``` - -Remove the old `decodeShuffleBlock` function. - -- [ ] **Step 4: Verify Rust compilation** - -Run: `cargo check --manifest-path native/Cargo.toml` -Expected: Compiles (possibly with warnings about unused code). JNI type details may need adjustment. - -- [ ] **Step 5: Commit** - -```bash -git add native/shuffle/src/ipc.rs native/shuffle/src/lib.rs native/core/src/execution/jni_api.rs -git commit -m "feat: JniInputStream and stream reader handle for shuffle read path" -``` - ---- - -## Task 7: Update JVM Read Side (Native.scala + NativeBatchDecoderIterator) - -**Files:** -- Modify: `spark/src/main/scala/org/apache/comet/Native.scala` -- Modify: `spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala` - -- [ ] **Step 1: Update `Native.scala` JNI declarations** - -Replace the `decodeShuffleBlock` declaration (lines 175-180) with: - -```scala - /** - * Open a shuffle stream reader over a JVM InputStream. - * @param inputStream the InputStream to read from - * @return an opaque handle to the native stream reader - */ - @native def openShuffleStream(inputStream: java.io.InputStream): Long - - /** - * Read the next batch from a shuffle stream. - * @param handle the native stream reader handle - * @param arrayAddrs Arrow Array addresses for FFI export - * @param schemaAddrs Arrow Schema addresses for FFI export - * @return the number of rows in the batch, or -1 if stream is exhausted - */ - @native def nextShuffleStreamBatch( - handle: Long, - arrayAddrs: Array[Long], - schemaAddrs: Array[Long]): Long - - /** - * Close and release a native shuffle stream reader. - * @param handle the native stream reader handle - */ - @native def closeShuffleStream(handle: Long): Unit -``` - -Also remove the `import java.nio.ByteBuffer` if it's no longer needed (check other usages first). - -- [ ] **Step 2: Rewrite `NativeBatchDecoderIterator.scala`** - -Replace the entire file with: - -```scala -/* - * (keep license header) - */ - -package org.apache.spark.sql.comet.execution.shuffle - -import java.io.InputStream - -import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.vectorized.ColumnarBatch - -import org.apache.comet.Native -import org.apache.comet.vector.NativeUtil - -/** - * Iterator that reads shuffle blocks from a JVM InputStream using native Arrow IPC - * stream decoding. The native side pulls bytes from the InputStream via JNI callbacks. - */ -case class NativeBatchDecoderIterator( - in: InputStream, - decodeTime: SQLMetric, - nativeLib: Native, - nativeUtil: NativeUtil, - tracingEnabled: Boolean) - extends Iterator[ColumnarBatch] { - - private var isClosed = false - private var currentBatch: ColumnarBatch = null - - // Open the native stream reader - private val handle: Long = if (in != null) { - nativeLib.openShuffleStream(in) - } else { - 0L - } - - private var batch = fetchNext() - - def hasNext(): Boolean = { - if (handle == 0L || isClosed) { - return false - } - if (batch.isDefined) { - return true - } - - if (currentBatch != null) { - currentBatch.close() - currentBatch = null - } - - batch = fetchNext() - if (batch.isEmpty) { - close() - return false - } - true - } - - def next(): ColumnarBatch = { - if (!hasNext) { - throw new NoSuchElementException - } - val nextBatch = batch.get - currentBatch = nextBatch - batch = None - currentBatch - } - - private def fetchNext(): Option[ColumnarBatch] = { - if (handle == 0L || isClosed) { - return None - } - - val startTime = System.nanoTime() - - // Query the field count from the native reader (it parsed the schema on open). - val batch = nativeUtil.getNextBatch( - nativeLib.shuffleStreamNumFields(handle).toInt, - (arrayAddrs, schemaAddrs) => { - nativeLib.nextShuffleStreamBatch(handle, arrayAddrs, schemaAddrs) - }) - - decodeTime.add(System.nanoTime() - startTime) - batch - } - - def close(): Unit = { - synchronized { - if (!isClosed) { - if (currentBatch != null) { - currentBatch.close() - currentBatch = null - } - if (handle != 0L) { - nativeLib.closeShuffleStream(handle) - } - if (in != null) { - in.close() - } - isClosed = true - } - } - } -} -``` - -**Note:** This requires adding a `shuffleStreamNumFields` JNI method. Add to `Native.scala`: - -```scala - /** - * Get the number of fields (columns) in a shuffle stream's schema. - * @param handle the native stream reader handle - * @return the number of fields - */ - @native def shuffleStreamNumFields(handle: Long): Long -``` - -And add the corresponding Rust JNI function in `jni_api.rs`: - -```rust -#[no_mangle] -/// Get the number of fields in the shuffle stream's schema. -/// # Safety -/// This function is inherently unsafe since it deals with raw pointers from JNI. -pub unsafe extern "system" fn Java_org_apache_comet_Native_shuffleStreamNumFields( - e: JNIEnv, - _class: JClass, - handle: jlong, -) -> jlong { - try_unwrap_or_throw(&e, |_env| { - let reader = unsafe { &*(handle as *const datafusion_comet_shuffle::ShuffleStreamReader) }; - Ok(reader.num_fields() as jlong) - }) -} -``` - -- [ ] **Step 3: Remove the companion object with thread-local buffer** - -The old `NativeBatchDecoderIterator` companion object (lines 185-198) with `threadLocalDataBuf` and `resetDataBuf` is no longer needed. Remove it entirely. The new version has no companion object. - -- [ ] **Step 4: Commit** - -```bash -git add spark/src/main/scala/org/apache/comet/Native.scala \ - spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/NativeBatchDecoderIterator.scala \ - native/core/src/execution/jni_api.rs -git commit -m "feat: JVM shuffle reader uses native IPC stream decoding via JNI" -``` - ---- - -## Task 8: Build and Run End-to-End Tests - -Verify the full stack works together. - -**Files:** No new files — testing existing test suites. - -- [ ] **Step 1: Build native code** - -Run: `make core` -Expected: Native Rust code compiles successfully. - -- [ ] **Step 2: Run Rust shuffle tests** - -Run: `cd native && cargo test -p datafusion-comet-shuffle` -Expected: All tests pass. Tests using `read_ipc_compressed` have been updated in previous tasks. - -- [ ] **Step 3: Run clippy** - -Run: `cd native && cargo clippy --all-targets --workspace -- -D warnings` -Expected: No warnings or errors. - -- [ ] **Step 4: Build JVM** - -Run: `make` -Expected: Full build succeeds (native + JVM). - -- [ ] **Step 5: Run JVM shuffle tests** - -Run: `./mvnw test -Dsuites="org.apache.comet.exec.CometShuffleSuite"` -Expected: All shuffle tests pass with the new format. - -- [ ] **Step 6: Run broader test suite to check for regressions** - -Run: `./mvnw test -DwildcardSuites="CometShuffle"` -Expected: All shuffle-related tests pass. - -- [ ] **Step 7: Commit any test fixes** - -If any tests needed adjustments (e.g., tests that checked the old custom header format), commit them: - -```bash -git add -A -git commit -m "test: update shuffle tests for IPC stream-per-partition format" -``` - ---- - -## Task 9: Clean Up Removed Code - -Remove any dead code left over from the migration. - -**Files:** -- Potentially: `native/shuffle/src/writers/shuffle_block_writer.rs` (if not already reduced to just `CompressionCodec`) -- Any remaining references to `read_ipc_compressed`, `decodeShuffleBlock`, Snappy - -- [ ] **Step 1: Search for dead code** - -Run: -```bash -grep -rn "read_ipc_compressed\|decodeShuffleBlock\|ShuffleBlockWriter\|snap::\|Snappy\|SNAP\|CompressionCodec::Snappy" native/shuffle/src/ spark/src/ common/src/ -``` - -- [ ] **Step 2: Remove any remaining references** - -Remove or update any files still referencing the old format. - -- [ ] **Step 3: Rename `shuffle_block_writer.rs` to `codec.rs`** - -Since the file now only contains the `CompressionCodec` enum, rename it: -- Rename `native/shuffle/src/writers/shuffle_block_writer.rs` to `native/shuffle/src/writers/codec.rs` -- Update `native/shuffle/src/writers/mod.rs`: change `mod shuffle_block_writer;` to `mod codec;` and update the use statement. - -- [ ] **Step 4: Run full test suite** - -Run: `cd native && cargo test -p datafusion-comet-shuffle` -Expected: All tests pass. - -- [ ] **Step 5: Format code** - -Run: `make format` -Expected: All code formatted. - -- [ ] **Step 6: Commit** - -```bash -git add -A -git commit -m "chore: clean up dead code from shuffle format migration" -``` diff --git a/docs/superpowers/specs/2026-04-02-shuffle-ipc-stream-per-partition-design.md b/docs/superpowers/specs/2026-04-02-shuffle-ipc-stream-per-partition-design.md deleted file mode 100644 index 4707cf42ca..0000000000 --- a/docs/superpowers/specs/2026-04-02-shuffle-ipc-stream-per-partition-design.md +++ /dev/null @@ -1,252 +0,0 @@ -# Shuffle Format: One IPC Stream Per Partition - -**Issue:** [#3882](https://github.com/apache/datafusion-comet/issues/3882) — Current shuffle format has too much overhead with default batch size - -**Date:** 2026-04-02 - -## Problem - -The current shuffle format writes each batch as an independent Arrow IPC stream, repeating the schema for every batch. Each block also has a custom 20-byte header (8-byte length + 8-byte field count + 4-byte codec tag) and creates a new compression codec instance. With the default batch size of 8192 rows, Comet shuffle files are ~50% larger than Spark shuffle files and overall query performance is ~10% slower than Spark. - -## Solution - -Write one Arrow IPC stream per partition instead of one per batch. Use Arrow IPC's built-in per-buffer body compression instead of wrapping the entire IPC stream in an external compression codec. Move IPC stream parsing to Rust, with Rust pulling bytes from JVM's `InputStream` via JNI callbacks. - -## Current Format (Per Partition) - -``` -[block 1: header(20 bytes) | compressed(schema + batch)] -[block 2: header(20 bytes) | compressed(schema + batch)] -... -[block N: header(20 bytes) | compressed(schema + batch)] -``` - -Each block is a self-contained compressed Arrow IPC stream with a custom header: -- 8 bytes: IPC block length (little-endian u64) -- 8 bytes: field count (little-endian u64) -- 4 bytes: compression codec tag (ASCII: `SNAP`, `LZ4_`, `ZSTD`, `NONE`) -- Variable: compressed Arrow IPC stream bytes (schema message + one record batch message) - -The JVM reader parses the custom header, reads the exact byte blob, and passes it to Rust via JNI for decompression and decoding. - -## New Format (Per Partition) - -``` -[standard Arrow IPC stream: schema | batch | batch | ... | EOS] -``` - -Each partition's data is a single standard Arrow IPC stream: -- One schema message (written once when the stream is created) -- N record batch messages (each batch's buffers individually compressed via Arrow IPC body compression) -- End-of-stream marker - -No custom header. No external compression wrapper. Standard Arrow IPC framing handles message boundaries. - -## Architecture Changes - -### Write Side (Rust) - -#### Immediate Mode (`ImmediateModePartitioner`) - -`PartitionOutputStream` currently creates a new `StreamWriter` per `write_ipc_block()` call. Change to hold a persistent `StreamWriter` per partition: - -- On first flush: create `StreamWriter` with `IpcWriteOptions::try_with_compression(...)`, which writes the schema message once -- On subsequent flushes: call `writer.write(batch)` which appends only the record batch message -- On `shuffle_write()`: call `writer.finish()` to write the EOS marker, then drain the buffer to the output file - -The `StreamWriter` writes into the existing `Vec` buffer. Spill behavior is unchanged — when memory pressure triggers a spill, the buffer (now containing partial IPC stream bytes) is written to a spill file. On final write, spill files are concatenated before the remaining in-memory bytes. Since all bytes belong to the same IPC stream, concatenation produces a valid stream. - -#### Buffered Mode (`MultiPartitionShuffleRepartitioner`) - -`PartitionWriter` wraps `ShuffleBlockWriter`, which creates a new `StreamWriter` per `write_batch()` call. Change to hold a persistent `StreamWriter` per partition: - -- Each `PartitionWriter` creates a `StreamWriter` on first batch write -- Subsequent batches append record batch messages only -- On final write: finish the stream and write to the output file - -#### `ShuffleBlockWriter` - -This struct is eliminated. Its responsibilities (compression codec selection, header writing, per-batch IPC encoding) are replaced by: -- `IpcWriteOptions` for compression codec selection -- Arrow `StreamWriter` for IPC encoding -- No custom headers needed - -#### `SinglePartitionShufflePartitioner` - -Same change — use a persistent `StreamWriter` for the single output partition. - -### Read Side - -#### New: `JniInputStream` (Rust) - -A Rust struct that wraps a JVM `InputStream` object reference and implements `std::io::Read`: - -```rust -struct JniInputStream { - jvm: JavaVM, - stream: GlobalRef, // reference to JVM InputStream - buffer: Vec, // internal read-ahead buffer (e.g., 64KB) - pos: usize, - len: usize, -} - -impl Read for JniInputStream { - fn read(&mut self, buf: &mut [u8]) -> io::Result { - // If internal buffer is exhausted, refill via JNI: - // byte[] jbuf = new byte[capacity]; - // int n = inputStream.read(jbuf, 0, capacity); - // Copy bytes from jbuf into self.buffer - // Then copy from self.buffer into buf - } -} -``` - -The internal buffer minimizes JNI boundary crossings. A 64KB buffer means one JNI call per 64KB of data, which keeps overhead negligible. - -#### New: JNI Function `decodeShuffleStream` - -```rust -pub unsafe extern "system" fn Java_org_apache_comet_Native_decodeShuffleStream( - e: JNIEnv, - _class: JClass, - input_stream: JObject, // JVM InputStream - array_addrs: JLongArray, - schema_addrs: JLongArray, - tracing_enabled: jboolean, -) -> jlong { - // 1. Wrap input_stream in JniInputStream - // 2. Create Arrow StreamReader over it - // 3. Read next batch from the stream - // 4. Export batch via Arrow FFI (prepare_output) - // 5. Return row count (or 0 if stream exhausted) -} -``` - -The `StreamReader` must be created with validation disabled (`with_skip_validation(true)`) since we control both the write and read sides and validation adds unnecessary overhead. - -This function is called repeatedly by the JVM iterator until it returns 0 (stream exhausted). The `StreamReader` must persist across calls — either by storing it in a native handle that the JVM holds, or by restructuring the JVM iterator to make a single JNI call that iterates all batches. - -**Recommended approach:** Use a native handle pattern: -1. `openShuffleStream(inputStream) -> long handle` — creates `JniInputStream` + `StreamReader`, returns opaque handle -2. `nextShuffleStreamBatch(handle, arrayAddrs, schemaAddrs) -> long rowCount` — reads next batch, exports via FFI -3. `closeShuffleStream(handle)` — drops the reader and stream - -This avoids recreating the `StreamReader` per batch and cleanly manages the lifecycle. - -#### JVM: `NativeBatchDecoderIterator` - -Simplify to use the native handle pattern: - -```scala -case class NativeBatchDecoderIterator(in: InputStream, ...) extends Iterator[ColumnarBatch] { - // On construction: call nativeLib.openShuffleStream(in) to get handle - private val handle = nativeLib.openShuffleStream(in) - - private def fetchNext(): Option[ColumnarBatch] = { - // Call nativeLib.nextShuffleStreamBatch(handle, arrayAddrs, schemaAddrs) - // Returns batch or None if stream exhausted - } - - def close(): Unit = { - nativeLib.closeShuffleStream(handle) - in.close() - } -} -``` - -Remove all manual header parsing (length bytes, field count, codec tag). - -#### Removed - -- `read_ipc_compressed()` in `ipc.rs` — no longer needed -- `decodeShuffleBlock` JNI function — replaced by the stream-based API -- Custom header format parsing in `NativeBatchDecoderIterator` - -### Compression - -#### Arrow IPC Body Compression - -Arrow IPC supports per-buffer compression via `IpcWriteOptions`: - -```rust -let options = IpcWriteOptions::try_with_compression( - Some(CompressionType::LZ4_FRAME) // or CompressionType::ZSTD -)?; -let writer = StreamWriter::try_new_with_options(output, &schema, options)?; -``` - -Each record batch's data buffers are individually compressed. The schema message and IPC framing metadata are not compressed (they're small). The `StreamReader` handles decompression transparently. - -#### Supported Codecs - -- **LZ4_FRAME** — fast compression/decompression, moderate ratio -- **ZSTD** — better compression ratio, slightly slower -- **None** — no compression - -#### Dropped Codec - -- **Snappy** — not supported by Arrow IPC body compression. This is acceptable because LZ4 provides similar speed characteristics with better compression ratios. The `CompressionCodec::Snappy` variant and all Snappy-related code paths are removed. - -### Configuration - -Map the existing `spark.comet.exec.shuffle.compression.codec` config (`COMET_EXEC_SHUFFLE_COMPRESSION_CODEC`) to Arrow IPC compression types: - -| Config Value | Arrow IPC CompressionType | -|---|---| -| `lz4` | `CompressionType::LZ4_FRAME` | -| `zstd` | `CompressionType::ZSTD` | -| `none` | `None` | -| `snappy` | Error or fall back to `LZ4_FRAME` with warning | - -### Format Compatibility - -This is a **breaking change** to the shuffle format. The new format is not readable by old readers and vice versa. This is acceptable because: - -- Shuffle data is ephemeral — it exists only for the duration of a job -- There is no cross-version shuffle data exchange -- All writers and readers within a single Comet deployment use the same version - -### What Stays the Same - -- **Index file format** — partition offset table written at end of shuffle write -- **Block fetching** — `ShuffleBlockFetcherIterator` and Spark's shuffle block resolution -- **Partition assignment** — hash/range partitioning logic in both modes -- **Spill file handling** — immediate mode spill/restore behavior (though spill files now contain partial IPC stream bytes) -- **`readAsRawStream()`** — still concatenates partition InputStreams; now each stream is a valid Arrow IPC stream - -## Files Modified - -### Rust (native/) - -| File | Change | -|---|---| -| `shuffle/src/partitioners/immediate_mode.rs` | `PartitionOutputStream`: persistent `StreamWriter`, remove custom headers, use `IpcWriteOptions` compression | -| `shuffle/src/partitioners/multi_partition.rs` | `PartitionWriter`: persistent `StreamWriter` per partition | -| `shuffle/src/partitioners/single_partition.rs` | Same persistent `StreamWriter` change | -| `shuffle/src/writers/shuffle_block_writer.rs` | **Remove** — replaced by direct `StreamWriter` usage | -| `shuffle/src/ipc.rs` | **Remove** `read_ipc_compressed` — no longer needed | -| `core/src/execution/jni_api.rs` | Replace `decodeShuffleBlock` with `openShuffleStream`, `nextShuffleStreamBatch`, `closeShuffleStream` | -| `shuffle/src/lib.rs` | New `JniInputStream` struct, update exports | - -### Scala (spark/) - -| File | Change | -|---|---| -| `NativeBatchDecoderIterator.scala` | Replace header parsing with native handle pattern (`open`/`next`/`close`) | -| `Native.scala` (or equivalent) | Add new JNI method declarations | -| `CometConf.scala` | Update codec config to map to Arrow IPC types, deprecate Snappy | - -### Removed - -| File | Reason | -|---|---| -| `shuffle/src/writers/shuffle_block_writer.rs` | Replaced by persistent `StreamWriter` with `IpcWriteOptions` | -| `shuffle/src/ipc.rs` (partially) | `read_ipc_compressed` no longer needed | - -## Testing - -- Existing shuffle tests should pass with the new format (they test end-to-end behavior, not wire format) -- Add unit test for `JniInputStream` — mock a JVM `InputStream` and verify `Read` impl -- Add integration test verifying a partition with multiple batches produces a valid Arrow IPC stream -- Verify shuffle benchmark shows reduced file size and improved performance with default batch size -- Test with LZ4, ZSTD, and no compression From c264d314d0e98213e786905e3d450726f01443fb Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 2 Apr 2026 11:06:41 -0600 Subject: [PATCH 46/46] format --- native/core/src/execution/operators/shuffle_scan.rs | 13 ++++--------- native/shuffle/src/ipc.rs | 4 +--- native/shuffle/src/writers/buf_batch_writer.rs | 5 +---- 3 files changed, 6 insertions(+), 16 deletions(-) diff --git a/native/core/src/execution/operators/shuffle_scan.rs b/native/core/src/execution/operators/shuffle_scan.rs index 385319c11a..9a5d41af79 100644 --- a/native/core/src/execution/operators/shuffle_scan.rs +++ b/native/core/src/execution/operators/shuffle_scan.rs @@ -18,9 +18,7 @@ use crate::{ errors::CometError, execution::{ - operators::ExecutionError, - planner::TEST_EXEC_CONTEXT_ID, - shuffle::ShuffleStreamReader, + operators::ExecutionError, planner::TEST_EXEC_CONTEXT_ID, shuffle::ShuffleStreamReader, }, jvm_bridge::JVMClasses, }; @@ -468,12 +466,9 @@ mod tests { // Write as Arrow IPC stream with compression let write_options = CompressionCodec::Zstd(1).ipc_write_options().unwrap(); let mut buf = Vec::new(); - let mut writer = StreamWriter::try_new_with_options( - &mut buf, - &dict_batch.schema(), - write_options, - ) - .unwrap(); + let mut writer = + StreamWriter::try_new_with_options(&mut buf, &dict_batch.schema(), write_options) + .unwrap(); writer.write(&dict_batch).unwrap(); writer.finish().unwrap(); diff --git a/native/shuffle/src/ipc.rs b/native/shuffle/src/ipc.rs index bf9dd2cc3e..bd41deb4d8 100644 --- a/native/shuffle/src/ipc.rs +++ b/native/shuffle/src/ipc.rs @@ -251,9 +251,7 @@ impl ShuffleStreamReader { match StreamReader::try_new(self.jni_stream.reader(), None) { Ok(new_reader) => { - self.reader = Some(unsafe { - new_reader.with_skip_validation(true) - }); + self.reader = Some(unsafe { new_reader.with_skip_validation(true) }); // Loop back to read from the new reader } Err(_) => { diff --git a/native/shuffle/src/writers/buf_batch_writer.rs b/native/shuffle/src/writers/buf_batch_writer.rs index 3c188de7ef..0ca1b9a1d7 100644 --- a/native/shuffle/src/writers/buf_batch_writer.rs +++ b/native/shuffle/src/writers/buf_batch_writer.rs @@ -63,10 +63,7 @@ impl BufBatchWriter { Ok(()) } - pub(crate) fn flush( - &mut self, - encode_time: &Time, - ) -> datafusion::common::Result<()> { + pub(crate) fn flush(&mut self, encode_time: &Time) -> datafusion::common::Result<()> { // Finish any remaining buffered rows in the coalescer self.coalescer.finish_buffered_batch()?; while let Some(batch) = self.coalescer.next_completed_batch() {